-
Notifications
You must be signed in to change notification settings - Fork 1.8k
/
MySqlSourceReader.java
361 lines (332 loc) · 16.9 KB
/
MySqlSourceReader.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
/*
* Copyright 2022 Ververica Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package com.ververica.cdc.connectors.mysql.source.reader;
import org.apache.flink.api.connector.source.SourceEvent;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.base.source.reader.RecordEmitter;
import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds;
import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
import org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager;
import org.apache.flink.connector.base.source.reader.synchronization.FutureCompletingBlockingQueue;
import org.apache.flink.util.FlinkRuntimeException;
import com.ververica.cdc.connectors.mysql.debezium.DebeziumUtils;
import com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfig;
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaEvent;
import com.ververica.cdc.connectors.mysql.source.events.BinlogSplitMetaRequestEvent;
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsAckEvent;
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsReportEvent;
import com.ververica.cdc.connectors.mysql.source.events.FinishedSnapshotSplitsRequestEvent;
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsSizeEvent;
import com.ververica.cdc.connectors.mysql.source.events.LatestFinishedSplitsSizeRequestEvent;
import com.ververica.cdc.connectors.mysql.source.events.SuspendBinlogReaderAckEvent;
import com.ververica.cdc.connectors.mysql.source.events.SuspendBinlogReaderEvent;
import com.ververica.cdc.connectors.mysql.source.events.WakeupReaderEvent;
import com.ververica.cdc.connectors.mysql.source.offset.BinlogOffset;
import com.ververica.cdc.connectors.mysql.source.split.FinishedSnapshotSplitInfo;
import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit;
import com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplitState;
import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplit;
import com.ververica.cdc.connectors.mysql.source.split.MySqlSnapshotSplitState;
import com.ververica.cdc.connectors.mysql.source.split.MySqlSplit;
import com.ververica.cdc.connectors.mysql.source.split.MySqlSplitState;
import com.ververica.cdc.connectors.mysql.source.split.SourceRecords;
import com.ververica.cdc.connectors.mysql.source.utils.TableDiscoveryUtils;
import io.debezium.connector.mysql.MySqlConnection;
import io.debezium.relational.TableId;
import io.debezium.relational.history.TableChanges;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.sql.SQLException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.function.Supplier;
import java.util.stream.Collectors;
import static com.ververica.cdc.connectors.mysql.source.events.WakeupReaderEvent.WakeUpTarget.SNAPSHOT_READER;
import static com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit.toNormalBinlogSplit;
import static com.ververica.cdc.connectors.mysql.source.split.MySqlBinlogSplit.toSuspendedBinlogSplit;
import static com.ververica.cdc.connectors.mysql.source.utils.ChunkUtils.getNextMetaGroupId;
/** The source reader for MySQL source splits. */
public class MySqlSourceReader<T>
extends SingleThreadMultiplexSourceReaderBase<
SourceRecords, T, MySqlSplit, MySqlSplitState> {
private static final Logger LOG = LoggerFactory.getLogger(MySqlSourceReader.class);
private final MySqlSourceConfig sourceConfig;
private final Map<String, MySqlSnapshotSplit> finishedUnackedSplits;
private final Map<String, MySqlBinlogSplit> uncompletedBinlogSplits;
private final int subtaskId;
private final MySqlSourceReaderContext mySqlSourceReaderContext;
private MySqlBinlogSplit suspendedBinlogSplit;
public MySqlSourceReader(
FutureCompletingBlockingQueue<RecordsWithSplitIds<SourceRecords>> elementQueue,
Supplier<MySqlSplitReader> splitReaderSupplier,
RecordEmitter<SourceRecords, T, MySqlSplitState> recordEmitter,
Configuration config,
MySqlSourceReaderContext context,
MySqlSourceConfig sourceConfig) {
super(
elementQueue,
new SingleThreadFetcherManager<>(elementQueue, splitReaderSupplier::get),
recordEmitter,
config,
context.getSourceReaderContext());
this.sourceConfig = sourceConfig;
this.finishedUnackedSplits = new HashMap<>();
this.uncompletedBinlogSplits = new HashMap<>();
this.subtaskId = context.getSourceReaderContext().getIndexOfSubtask();
this.mySqlSourceReaderContext = context;
this.suspendedBinlogSplit = null;
}
@Override
public void start() {
if (getNumberOfCurrentlyAssignedSplits() == 0) {
context.sendSplitRequest();
}
}
@Override
protected MySqlSplitState initializedState(MySqlSplit split) {
if (split.isSnapshotSplit()) {
return new MySqlSnapshotSplitState(split.asSnapshotSplit());
} else {
return new MySqlBinlogSplitState(split.asBinlogSplit());
}
}
@Override
public List<MySqlSplit> snapshotState(long checkpointId) {
List<MySqlSplit> stateSplits = super.snapshotState(checkpointId);
// unfinished splits
List<MySqlSplit> unfinishedSplits =
stateSplits.stream()
.filter(split -> !finishedUnackedSplits.containsKey(split.splitId()))
.collect(Collectors.toList());
// add finished snapshot splits that didn't receive ack yet
unfinishedSplits.addAll(finishedUnackedSplits.values());
// add binlog splits who are uncompleted
unfinishedSplits.addAll(uncompletedBinlogSplits.values());
// add suspended BinlogSplit
if (suspendedBinlogSplit != null) {
unfinishedSplits.add(suspendedBinlogSplit);
}
logCurrentBinlogOffsets(unfinishedSplits, checkpointId);
return unfinishedSplits;
}
@Override
protected void onSplitFinished(Map<String, MySqlSplitState> finishedSplitIds) {
boolean requestNextSplit = true;
for (MySqlSplitState mySqlSplitState : finishedSplitIds.values()) {
MySqlSplit mySqlSplit = mySqlSplitState.toMySqlSplit();
if (mySqlSplit.isBinlogSplit()) {
LOG.info(
"binlog split reader suspended due to newly added table, offset {}",
mySqlSplitState.asBinlogSplitState().getStartingOffset());
mySqlSourceReaderContext.resetStopBinlogSplitReader();
suspendedBinlogSplit = toSuspendedBinlogSplit(mySqlSplit.asBinlogSplit());
context.sendSourceEventToCoordinator(new SuspendBinlogReaderAckEvent());
// do not request next split when the reader is suspended, the suspended reader will
// automatically request the next split after it has been wakeup
requestNextSplit = false;
} else {
finishedUnackedSplits.put(mySqlSplit.splitId(), mySqlSplit.asSnapshotSplit());
}
}
reportFinishedSnapshotSplitsIfNeed();
if (requestNextSplit) {
context.sendSplitRequest();
}
}
@Override
public void addSplits(List<MySqlSplit> splits) {
// restore for finishedUnackedSplits
List<MySqlSplit> unfinishedSplits = new ArrayList<>();
for (MySqlSplit split : splits) {
LOG.info("Add Split: " + split);
if (split.isSnapshotSplit()) {
MySqlSnapshotSplit snapshotSplit = split.asSnapshotSplit();
if (snapshotSplit.isSnapshotReadFinished()) {
finishedUnackedSplits.put(snapshotSplit.splitId(), snapshotSplit);
} else {
unfinishedSplits.add(split);
}
} else {
MySqlBinlogSplit binlogSplit = split.asBinlogSplit();
// the binlog split is suspended
if (binlogSplit.isSuspended()) {
suspendedBinlogSplit = binlogSplit;
} else if (!binlogSplit.isCompletedSplit()) {
uncompletedBinlogSplits.put(split.splitId(), split.asBinlogSplit());
requestBinlogSplitMetaIfNeeded(split.asBinlogSplit());
} else {
uncompletedBinlogSplits.remove(split.splitId());
MySqlBinlogSplit mySqlBinlogSplit =
discoverTableSchemasForBinlogSplit(split.asBinlogSplit());
unfinishedSplits.add(mySqlBinlogSplit);
}
}
}
// notify split enumerator again about the finished unacked snapshot splits
reportFinishedSnapshotSplitsIfNeed();
// add all un-finished splits (including binlog split) to SourceReaderBase
if (!unfinishedSplits.isEmpty()) {
super.addSplits(unfinishedSplits);
}
}
private MySqlBinlogSplit discoverTableSchemasForBinlogSplit(MySqlBinlogSplit split) {
final String splitId = split.splitId();
if (split.getTableSchemas().isEmpty()) {
try (MySqlConnection jdbc = DebeziumUtils.createMySqlConnection(sourceConfig)) {
Map<TableId, TableChanges.TableChange> tableSchemas =
TableDiscoveryUtils.discoverCapturedTableSchemas(sourceConfig, jdbc);
LOG.info("The table schema discovery for binlog split {} success", splitId);
return MySqlBinlogSplit.fillTableSchemas(split, tableSchemas);
} catch (SQLException e) {
LOG.error("Failed to obtains table schemas due to {}", e.getMessage());
throw new FlinkRuntimeException(e);
}
} else {
LOG.warn(
"The binlog split {} has table schemas yet, skip the table schema discovery",
split);
return split;
}
}
@Override
public void handleSourceEvents(SourceEvent sourceEvent) {
if (sourceEvent instanceof FinishedSnapshotSplitsAckEvent) {
FinishedSnapshotSplitsAckEvent ackEvent = (FinishedSnapshotSplitsAckEvent) sourceEvent;
LOG.debug(
"The subtask {} receives ack event for {} from enumerator.",
subtaskId,
ackEvent.getFinishedSplits());
for (String splitId : ackEvent.getFinishedSplits()) {
this.finishedUnackedSplits.remove(splitId);
}
} else if (sourceEvent instanceof FinishedSnapshotSplitsRequestEvent) {
// report finished snapshot splits
LOG.debug(
"The subtask {} receives request to report finished snapshot splits.",
subtaskId);
reportFinishedSnapshotSplitsIfNeed();
} else if (sourceEvent instanceof BinlogSplitMetaEvent) {
LOG.debug(
"The subtask {} receives binlog meta with group id {}.",
subtaskId,
((BinlogSplitMetaEvent) sourceEvent).getMetaGroupId());
fillMetaDataForBinlogSplit((BinlogSplitMetaEvent) sourceEvent);
} else if (sourceEvent instanceof SuspendBinlogReaderEvent) {
mySqlSourceReaderContext.setStopBinlogSplitReader();
} else if (sourceEvent instanceof WakeupReaderEvent) {
WakeupReaderEvent wakeupReaderEvent = (WakeupReaderEvent) sourceEvent;
if (wakeupReaderEvent.getTarget() == SNAPSHOT_READER) {
context.sendSplitRequest();
} else {
if (suspendedBinlogSplit != null) {
context.sendSourceEventToCoordinator(
new LatestFinishedSplitsSizeRequestEvent());
}
}
} else if (sourceEvent instanceof LatestFinishedSplitsSizeEvent) {
if (suspendedBinlogSplit != null) {
final int finishedSplitsSize =
((LatestFinishedSplitsSizeEvent) sourceEvent).getLatestFinishedSplitsSize();
final MySqlBinlogSplit binlogSplit =
toNormalBinlogSplit(suspendedBinlogSplit, finishedSplitsSize);
suspendedBinlogSplit = null;
this.addSplits(Collections.singletonList(binlogSplit));
}
} else {
super.handleSourceEvents(sourceEvent);
}
}
private void reportFinishedSnapshotSplitsIfNeed() {
if (!finishedUnackedSplits.isEmpty()) {
final Map<String, BinlogOffset> finishedOffsets = new HashMap<>();
for (MySqlSnapshotSplit split : finishedUnackedSplits.values()) {
finishedOffsets.put(split.splitId(), split.getHighWatermark());
}
FinishedSnapshotSplitsReportEvent reportEvent =
new FinishedSnapshotSplitsReportEvent(finishedOffsets);
context.sendSourceEventToCoordinator(reportEvent);
LOG.debug(
"The subtask {} reports offsets of finished snapshot splits {}.",
subtaskId,
finishedOffsets);
}
}
private void requestBinlogSplitMetaIfNeeded(MySqlBinlogSplit binlogSplit) {
final String splitId = binlogSplit.splitId();
if (!binlogSplit.isCompletedSplit()) {
final int nextMetaGroupId =
getNextMetaGroupId(
binlogSplit.getFinishedSnapshotSplitInfos().size(),
sourceConfig.getSplitMetaGroupSize());
BinlogSplitMetaRequestEvent splitMetaRequestEvent =
new BinlogSplitMetaRequestEvent(splitId, nextMetaGroupId);
context.sendSourceEventToCoordinator(splitMetaRequestEvent);
} else {
LOG.info("The meta of binlog split {} has been collected success", splitId);
this.addSplits(Collections.singletonList(binlogSplit));
}
}
private void fillMetaDataForBinlogSplit(BinlogSplitMetaEvent metadataEvent) {
MySqlBinlogSplit binlogSplit = uncompletedBinlogSplits.get(metadataEvent.getSplitId());
if (binlogSplit != null) {
final int receivedMetaGroupId = metadataEvent.getMetaGroupId();
final int expectedMetaGroupId =
getNextMetaGroupId(
binlogSplit.getFinishedSnapshotSplitInfos().size(),
sourceConfig.getSplitMetaGroupSize());
if (receivedMetaGroupId == expectedMetaGroupId) {
List<FinishedSnapshotSplitInfo> metaDataGroup =
metadataEvent.getMetaGroup().stream()
.map(FinishedSnapshotSplitInfo::deserialize)
.collect(Collectors.toList());
uncompletedBinlogSplits.put(
binlogSplit.splitId(),
MySqlBinlogSplit.appendFinishedSplitInfos(binlogSplit, metaDataGroup));
LOG.info("Fill meta data of group {} to binlog split", metaDataGroup.size());
} else {
LOG.warn(
"Received out of oder binlog meta event for split {}, the received meta group id is {}, but expected is {}, ignore it",
metadataEvent.getSplitId(),
receivedMetaGroupId,
expectedMetaGroupId);
}
requestBinlogSplitMetaIfNeeded(uncompletedBinlogSplits.get(binlogSplit.splitId()));
} else {
LOG.warn(
"Received binlog meta event for split {}, but the uncompleted split map does not contain it",
metadataEvent.getSplitId());
}
}
private void logCurrentBinlogOffsets(List<MySqlSplit> splits, long checkpointId) {
if (!LOG.isInfoEnabled()) {
return;
}
for (MySqlSplit split : splits) {
if (!split.isBinlogSplit()) {
return;
}
BinlogOffset offset = split.asBinlogSplit().getStartingOffset();
LOG.info("Binlog offset on checkpoint {}: {}", checkpointId, offset);
}
}
@Override
protected MySqlSplit toSplitType(String splitId, MySqlSplitState splitState) {
return splitState.toMySqlSplit();
}
}