18
18
19
19
package com .ververica .cdc .connectors .mysql .debezium .task .context ;
20
20
21
- import org .apache .flink .configuration .Configuration ;
22
-
23
21
import com .github .shyiko .mysql .binlog .BinaryLogClient ;
24
22
import com .ververica .cdc .connectors .mysql .debezium .EmbeddedFlinkDatabaseHistory ;
25
23
import com .ververica .cdc .connectors .mysql .debezium .dispatcher .EventDispatcherImpl ;
26
- import com .ververica .cdc .connectors .mysql .source .MySqlSourceOptions ;
24
+ import com .ververica .cdc .connectors .mysql .source .MySqlParallelSourceConfig ;
27
25
import com .ververica .cdc .connectors .mysql .source .offset .BinlogOffset ;
28
26
import com .ververica .cdc .connectors .mysql .source .split .MySqlSplit ;
29
27
import io .debezium .connector .AbstractSourceInfo ;
36
34
import io .debezium .connector .mysql .MySqlOffsetContext ;
37
35
import io .debezium .connector .mysql .MySqlStreamingChangeEventSourceMetrics ;
38
36
import io .debezium .connector .mysql .MySqlTopicSelector ;
39
- import io .debezium .connector .mysql .MySqlValueConverters ;
40
37
import io .debezium .data .Envelope ;
41
- import io .debezium .jdbc .JdbcValueConverters ;
42
- import io .debezium .jdbc .TemporalPrecisionMode ;
43
38
import io .debezium .pipeline .DataChangeEvent ;
44
39
import io .debezium .pipeline .ErrorHandler ;
45
40
import io .debezium .pipeline .metrics .SnapshotChangeEventSourceMetrics ;
46
41
import io .debezium .pipeline .metrics .StreamingChangeEventSourceMetrics ;
47
42
import io .debezium .pipeline .source .spi .EventMetadataProvider ;
48
43
import io .debezium .pipeline .spi .OffsetContext ;
49
44
import io .debezium .relational .TableId ;
50
- import io .debezium .relational .history .AbstractDatabaseHistory ;
51
45
import io .debezium .schema .DataCollectionId ;
52
46
import io .debezium .schema .TopicSelector ;
53
47
import io .debezium .util .Clock ;
62
56
import java .util .Map ;
63
57
64
58
import static com .ververica .cdc .connectors .mysql .source .offset .BinlogOffset .BINLOG_FILENAME_OFFSET_KEY ;
65
- import static io .debezium .config .CommonConnectorConfig .TOMBSTONES_ON_DELETE ;
66
59
67
60
/**
68
61
* A stateful task context that contains entries the debezium mysql connector task required.
@@ -75,7 +68,7 @@ public class StatefulTaskContext {
75
68
private static final Logger LOG = LoggerFactory .getLogger (StatefulTaskContext .class );
76
69
private static final Clock clock = Clock .SYSTEM ;
77
70
78
- private final io . debezium . config . Configuration dezConf ;
71
+ private final MySqlParallelSourceConfig sourceConfig ;
79
72
private final MySqlConnectorConfig connectorConfig ;
80
73
private final MySqlEventMetadataProvider metadataProvider ;
81
74
private final SchemaNameAdjuster schemaNameAdjuster ;
@@ -93,11 +86,11 @@ public class StatefulTaskContext {
93
86
private ErrorHandler errorHandler ;
94
87
95
88
public StatefulTaskContext (
96
- Configuration configuration ,
89
+ MySqlParallelSourceConfig sourceConfig ,
97
90
BinaryLogClient binaryLogClient ,
98
91
MySqlConnection connection ) {
99
- this .dezConf = toDebeziumConfig ( configuration ) ;
100
- this .connectorConfig = new MySqlConnectorConfig ( dezConf );
92
+ this .sourceConfig = sourceConfig ;
93
+ this .connectorConfig = sourceConfig . getMySqlConnectorConfig ( );
101
94
this .schemaNameAdjuster = SchemaNameAdjuster .create ();
102
95
this .metadataProvider = new MySqlEventMetadataProvider ();
103
96
this .binaryLogClient = binaryLogClient ;
@@ -106,19 +99,14 @@ public StatefulTaskContext(
106
99
107
100
public void configure (MySqlSplit mySqlSplit ) {
108
101
// initial stateful objects
109
- final boolean tableIdCaseInsensitive = connection .isTableIdCaseSensitive ();
110
102
this .topicSelector = MySqlTopicSelector .defaultSelector (connectorConfig );
111
- final MySqlValueConverters valueConverters = getValueConverters (connectorConfig );
112
103
EmbeddedFlinkDatabaseHistory .registerHistory (
113
- dezConf .getString (EmbeddedFlinkDatabaseHistory .DATABASE_HISTORY_INSTANCE_NAME ),
104
+ sourceConfig
105
+ .getDbzConfig ()
106
+ .getString (EmbeddedFlinkDatabaseHistory .DATABASE_HISTORY_INSTANCE_NAME ),
114
107
mySqlSplit .getTableSchemas ().values ());
115
108
this .databaseSchema =
116
- new MySqlDatabaseSchema (
117
- connectorConfig ,
118
- valueConverters ,
119
- topicSelector ,
120
- schemaNameAdjuster ,
121
- tableIdCaseInsensitive );
109
+ MySqlParallelSourceConfig .getMySqlDatabaseSchema (connectorConfig , connection );
122
110
this .offsetContext =
123
111
loadStartingOffsetState (new MySqlOffsetContext .Loader (connectorConfig ), mySqlSplit );
124
112
validateAndLoadDatabaseHistory (offsetContext , databaseSchema );
@@ -139,7 +127,7 @@ public void configure(MySqlSplit mySqlSplit) {
139
127
() ->
140
128
taskContext .configureLoggingContext (
141
129
"mysql-cdc-connector-task" ))
142
- // no buffer any more , we use signal event
130
+ // do not buffer any element , we use signal event
143
131
// .buffering()
144
132
.build ();
145
133
this .dispatcher =
@@ -218,30 +206,6 @@ private boolean isBinlogAvailable(MySqlOffsetContext offset) {
218
206
return found ;
219
207
}
220
208
221
- private static MySqlValueConverters getValueConverters (MySqlConnectorConfig configuration ) {
222
- TemporalPrecisionMode timePrecisionMode = configuration .getTemporalPrecisionMode ();
223
- JdbcValueConverters .DecimalMode decimalMode = configuration .getDecimalMode ();
224
- String bigIntUnsignedHandlingModeStr =
225
- configuration
226
- .getConfig ()
227
- .getString (MySqlConnectorConfig .BIGINT_UNSIGNED_HANDLING_MODE );
228
- MySqlConnectorConfig .BigIntUnsignedHandlingMode bigIntUnsignedHandlingMode =
229
- MySqlConnectorConfig .BigIntUnsignedHandlingMode .parse (
230
- bigIntUnsignedHandlingModeStr );
231
- JdbcValueConverters .BigIntUnsignedMode bigIntUnsignedMode =
232
- bigIntUnsignedHandlingMode .asBigIntUnsignedMode ();
233
-
234
- final boolean timeAdjusterEnabled =
235
- configuration .getConfig ().getBoolean (MySqlConnectorConfig .ENABLE_TIME_ADJUSTER );
236
- return new MySqlValueConverters (
237
- decimalMode ,
238
- timePrecisionMode ,
239
- bigIntUnsignedMode ,
240
- configuration .binaryHandlingMode (),
241
- timeAdjusterEnabled ? MySqlValueConverters ::adjustTemporal : x -> x ,
242
- MySqlValueConverters ::defaultParsingErrorHandler );
243
- }
244
-
245
209
/** Copied from debezium for accessing here. */
246
210
public static class MySqlEventMetadataProvider implements EventMetadataProvider {
247
211
public static final String SERVER_ID_KEY = "server_id" ;
@@ -297,8 +261,8 @@ public static Clock getClock() {
297
261
return clock ;
298
262
}
299
263
300
- public io . debezium . config . Configuration getDezConf () {
301
- return dezConf ;
264
+ public MySqlParallelSourceConfig getSourceConfig () {
265
+ return sourceConfig ;
302
266
}
303
267
304
268
public MySqlConnectorConfig getConnectorConfig () {
@@ -354,48 +318,4 @@ public StreamingChangeEventSourceMetrics getStreamingChangeEventSourceMetrics()
354
318
public SchemaNameAdjuster getSchemaNameAdjuster () {
355
319
return schemaNameAdjuster ;
356
320
}
357
-
358
- // ------------ utils ---------
359
- public static BinaryLogClient getBinaryClient (Configuration configuration ) {
360
- final MySqlConnectorConfig connectorConfig =
361
- new MySqlConnectorConfig (toDebeziumConfig (configuration ));
362
- return new BinaryLogClient (
363
- connectorConfig .hostname (),
364
- connectorConfig .port (),
365
- connectorConfig .username (),
366
- connectorConfig .password ());
367
- }
368
-
369
- public static MySqlConnection getConnection (Configuration configuration ) {
370
- return new MySqlConnection (
371
- new MySqlConnection .MySqlConnectionConfiguration (toDebeziumConfig (configuration )));
372
- }
373
-
374
- public static MySqlDatabaseSchema getMySqlDatabaseSchema (
375
- Configuration configuration , MySqlConnection connection ) {
376
- io .debezium .config .Configuration dezConf = toDebeziumConfig (configuration );
377
- MySqlConnectorConfig connectorConfig = new MySqlConnectorConfig (dezConf );
378
- boolean tableIdCaseInsensitive = connection .isTableIdCaseSensitive ();
379
- TopicSelector <TableId > topicSelector = MySqlTopicSelector .defaultSelector (connectorConfig );
380
- SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster .create ();
381
- MySqlValueConverters valueConverters = getValueConverters (connectorConfig );
382
- return new MySqlDatabaseSchema (
383
- connectorConfig ,
384
- valueConverters ,
385
- topicSelector ,
386
- schemaNameAdjuster ,
387
- tableIdCaseInsensitive );
388
- }
389
-
390
- public static io .debezium .config .Configuration toDebeziumConfig (Configuration configuration ) {
391
- return io .debezium .config .Configuration .from (configuration .toMap ())
392
- .edit ()
393
- .with (AbstractDatabaseHistory .INTERNAL_PREFER_DDL , true )
394
- .with (TOMBSTONES_ON_DELETE , false )
395
- .with ("database.responseBuffering" , "adaptive" )
396
- .with (
397
- "database.fetchSize" ,
398
- configuration .getInteger (MySqlSourceOptions .SCAN_SNAPSHOT_FETCH_SIZE ))
399
- .build ();
400
- }
401
321
}
0 commit comments