1
+ /*
2
+ * Copyright 2022-2023 Bytedance Ltd. and/or its affiliates.
3
+ *
4
+ * Licensed under the Apache License, Version 2.0 (the "License");
5
+ * you may not use this file except in compliance with the License.
6
+ * You may obtain a copy of the License at
7
+ *
8
+ * http://www.apache.org/licenses/LICENSE-2.0
9
+ *
10
+ * Unless required by applicable law or agreed to in writing, software
11
+ * distributed under the License is distributed on an "AS IS" BASIS,
12
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13
+ * See the License for the specific language governing permissions and
14
+ * limitations under the License.
15
+ */
1
16
package com .bytedance .bitsail .connector .cdc .jdbc .source .reader ;
2
17
3
18
import com .bytedance .bitsail .common .configuration .BitSailConfiguration ;
4
19
import com .bytedance .bitsail .connector .cdc .jdbc .source .config .AbstractJdbcDebeziumConfig ;
5
- import com .bytedance .bitsail .connector .cdc .jdbc .source .streaming .SplitChangeEventStreamingTaskContext ;
20
+ import com .bytedance .bitsail .connector .cdc .jdbc .source .streaming .AbstractSplitChangeEventStreamingTaskContext ;
6
21
import com .bytedance .bitsail .connector .cdc .jdbc .source .streaming .SplitChangeEventStreamingTaskController ;
7
22
import com .bytedance .bitsail .connector .cdc .source .reader .BinlogSplitReader ;
8
23
import com .bytedance .bitsail .connector .cdc .source .split .BinlogSplit ;
9
24
import com .bytedance .bitsail .common .row .Row ;
10
25
import io .debezium .connector .base .ChangeEventQueue ;
11
- import io .debezium .connector .common .CdcSourceTaskContext ;
12
26
import io .debezium .pipeline .DataChangeEvent ;
13
27
import io .debezium .relational .RelationalDatabaseConnectorConfig ;
28
+ import lombok .Getter ;
14
29
import org .apache .kafka .connect .source .SourceRecord ;
15
30
import org .slf4j .Logger ;
16
31
import org .slf4j .LoggerFactory ;
23
38
import java .util .Map ;
24
39
import java .util .concurrent .TimeUnit ;
25
40
26
- public abstract class AbstractJdbcChangeEventReader implements BinlogSplitReader <Row > {
27
- private static final Logger LOG = LoggerFactory .getLogger (AbstractJdbcChangeEventReader .class );
41
+ @ Getter
42
+ public abstract class AbstractJdbcChangeEventSplitReader implements BinlogSplitReader <Row > {
43
+ private static final Logger LOG = LoggerFactory .getLogger (AbstractJdbcChangeEventSplitReader .class );
28
44
29
- private final AbstractJdbcDebeziumConfig jdbcDebeziumConfig ;
45
+ protected final AbstractJdbcDebeziumConfig jdbcDebeziumConfig ;
30
46
31
- private ChangeEventQueue <DataChangeEvent > queue ;
47
+ protected ChangeEventQueue <DataChangeEvent > queue ;
32
48
33
- private RelationalDatabaseConnectorConfig connectorConfig ;
49
+ protected RelationalDatabaseConnectorConfig connectorConfig ;
34
50
35
- private List <SourceRecord > batch ;
51
+ protected List <SourceRecord > batch ;
36
52
37
- private Iterator <SourceRecord > recordIterator ;
53
+ protected Iterator <SourceRecord > recordIterator ;
38
54
39
- private CdcSourceTaskContext taskContext ;
55
+ protected Map < String , ?> offset ;
40
56
41
- private Map < String , ?> offset ;
57
+ protected SplitChangeEventStreamingTaskController splitChangeEventStreamingTaskController ;
42
58
43
- private SplitChangeEventStreamingTaskController splitChangeEventStreamingTaskController ;
44
-
45
- private SplitChangeEventStreamingTaskContext splitChangeEventStreamingTaskContext ;
59
+ protected AbstractSplitChangeEventStreamingTaskContext splitChangeEventStreamingTaskContext ;
46
60
47
61
private final int subtaskId ;
48
62
49
- public AbstractJdbcChangeEventReader (BitSailConfiguration jobConf , int subtaskId ) {
63
+ public AbstractJdbcChangeEventSplitReader (BitSailConfiguration jobConf , int subtaskId ) {
50
64
jdbcDebeziumConfig = getJdbcDebeziumConfig (jobConf );
51
- connectorConfig = jdbcDebeziumConfig .getConnectorConfig ();
65
+ connectorConfig = jdbcDebeziumConfig .getDbzJdbcConnectorConfig ();
52
66
this .subtaskId = subtaskId ;
53
67
this .offset = new HashMap <>();
54
68
}
55
69
56
- public AbstractJdbcDebeziumConfig getJdbcDebeziumConfig (BitSailConfiguration jobConf ) {
57
- return AbstractJdbcDebeziumConfig .fromBitSailConf (jobConf );
58
- }
59
-
60
- public abstract SplitChangeEventStreamingTaskContext getSplitReaderTaskContext ();
70
+ public abstract AbstractJdbcDebeziumConfig getJdbcDebeziumConfig (BitSailConfiguration jobConf );
61
71
62
- public abstract void testConnectionAndValidBinlogConfiguration ( RelationalDatabaseConnectorConfig connectorConfig ) throws IOException ;
72
+ public abstract AbstractSplitChangeEventStreamingTaskContext getSplitReaderTaskContext ( BinlogSplit split , RelationalDatabaseConnectorConfig connectorConfig );
63
73
64
74
public void inititialzeSplitReader (BinlogSplit split ) {
65
- splitChangeEventStreamingTaskContext = getSplitReaderTaskContext ();
75
+ splitChangeEventStreamingTaskContext = getSplitReaderTaskContext (split , connectorConfig );
66
76
this .offset = new HashMap <>();
67
77
this .queue = new ChangeEventQueue .Builder <DataChangeEvent >()
68
78
.pollInterval (connectorConfig .getPollInterval ())
69
79
.maxBatchSize (connectorConfig .getMaxBatchSize ())
70
80
.maxQueueSize (connectorConfig .getMaxQueueSize ())
71
81
.maxQueueSizeInBytes (connectorConfig .getMaxQueueSizeInBytes ())
72
- .loggingContextSupplier (() -> taskContext .configureLoggingContext (splitChangeEventStreamingTaskContext .threadNamePrefix ()))
82
+ .loggingContextSupplier (() -> splitChangeEventStreamingTaskContext .getDbzTaskContext ()
83
+ .configureLoggingContext (splitChangeEventStreamingTaskContext .threadNamePrefix ()))
73
84
.buffering ()
74
85
.build ();
75
86
this .batch = new ArrayList <>();
76
87
this .recordIterator = this .batch .iterator ();
77
- splitChangeEventStreamingTaskContext .initializeSplitReaderTaskContext ( connectorConfig , this .queue );
88
+ splitChangeEventStreamingTaskContext .attachStreamingToQueue ( this .queue );
78
89
splitChangeEventStreamingTaskController = new SplitChangeEventStreamingTaskController (splitChangeEventStreamingTaskContext , this .subtaskId );
79
90
}
80
91
@@ -136,7 +147,7 @@ public Row poll() {
136
147
* @throws Exception
137
148
*/
138
149
@ Override
139
- public boolean hasNext () throws Exception {
150
+ public boolean hasNext () {
140
151
if (this .recordIterator .hasNext ()) {
141
152
return true ;
142
153
} else {
@@ -149,20 +160,24 @@ public boolean isCompleted() {
149
160
return !splitChangeEventStreamingTaskController .isRunning ();
150
161
}
151
162
152
- private boolean pollNextBatch () throws InterruptedException {
163
+ private boolean pollNextBatch () {
153
164
if (splitChangeEventStreamingTaskController .isRunning ()) {
154
- List <DataChangeEvent > dbzRecords = queue .poll ();
155
- while (dbzRecords .isEmpty ()) {
156
- //sleep 10s
157
- LOG .info ("No record found, sleep for 5s in reader" );
158
- TimeUnit .SECONDS .sleep (5 );
159
- dbzRecords = queue .poll ();
160
- }
161
- this .batch = new ArrayList <>();
162
- for (DataChangeEvent event : dbzRecords ) {
163
- this .batch .add (event .getRecord ());
165
+ try {
166
+ List <DataChangeEvent > dbzRecords = queue .poll ();
167
+ while (dbzRecords .isEmpty ()) {
168
+ //sleep 10s
169
+ LOG .info ("No record found, sleep for 5s in reader" );
170
+ TimeUnit .SECONDS .sleep (5 );
171
+ dbzRecords = queue .poll ();
172
+ }
173
+ this .batch = new ArrayList <>();
174
+ for (DataChangeEvent event : dbzRecords ) {
175
+ this .batch .add (event .getRecord ());
176
+ }
177
+ this .recordIterator = this .batch .iterator ();
178
+ } catch (InterruptedException e ) {
179
+ throw new RuntimeException (e );
164
180
}
165
- this .recordIterator = this .batch .iterator ();
166
181
return true ;
167
182
}
168
183
return false ;
0 commit comments