|
| 1 | +package com.bytedance.bitsail.connector.cdc.jdbc.source.reader; |
| 2 | + |
| 3 | +import com.bytedance.bitsail.common.configuration.BitSailConfiguration; |
| 4 | +import com.bytedance.bitsail.connector.cdc.jdbc.source.config.AbstractJdbcDebeziumConfig; |
| 5 | +import com.bytedance.bitsail.connector.cdc.jdbc.source.streaming.SplitChangeEventStreamingTaskContext; |
| 6 | +import com.bytedance.bitsail.connector.cdc.jdbc.source.streaming.SplitChangeEventStreamingTaskController; |
| 7 | +import com.bytedance.bitsail.connector.cdc.source.reader.BinlogSplitReader; |
| 8 | +import com.bytedance.bitsail.connector.cdc.source.split.BinlogSplit; |
| 9 | +import com.bytedance.bitsail.common.row.Row; |
| 10 | +import io.debezium.connector.base.ChangeEventQueue; |
| 11 | +import io.debezium.connector.common.CdcSourceTaskContext; |
| 12 | +import io.debezium.pipeline.DataChangeEvent; |
| 13 | +import io.debezium.relational.RelationalDatabaseConnectorConfig; |
| 14 | +import org.apache.kafka.connect.source.SourceRecord; |
| 15 | +import org.slf4j.Logger; |
| 16 | +import org.slf4j.LoggerFactory; |
| 17 | + |
| 18 | +import java.io.IOException; |
| 19 | +import java.util.ArrayList; |
| 20 | +import java.util.HashMap; |
| 21 | +import java.util.Iterator; |
| 22 | +import java.util.List; |
| 23 | +import java.util.Map; |
| 24 | +import java.util.concurrent.TimeUnit; |
| 25 | + |
| 26 | +public abstract class AbstractJdbcChangeEventReader implements BinlogSplitReader<Row> { |
| 27 | + private static final Logger LOG = LoggerFactory.getLogger(AbstractJdbcChangeEventReader.class); |
| 28 | + |
| 29 | + private final AbstractJdbcDebeziumConfig jdbcDebeziumConfig; |
| 30 | + |
| 31 | + private ChangeEventQueue<DataChangeEvent> queue; |
| 32 | + |
| 33 | + private RelationalDatabaseConnectorConfig connectorConfig; |
| 34 | + |
| 35 | + private List<SourceRecord> batch; |
| 36 | + |
| 37 | + private Iterator<SourceRecord> recordIterator; |
| 38 | + |
| 39 | + private CdcSourceTaskContext taskContext; |
| 40 | + |
| 41 | + private Map<String, ?> offset; |
| 42 | + |
| 43 | + private SplitChangeEventStreamingTaskController splitChangeEventStreamingTaskController; |
| 44 | + |
| 45 | + private SplitChangeEventStreamingTaskContext splitChangeEventStreamingTaskContext; |
| 46 | + |
| 47 | + private final int subtaskId; |
| 48 | + |
| 49 | + public AbstractJdbcChangeEventReader(BitSailConfiguration jobConf, int subtaskId) { |
| 50 | + jdbcDebeziumConfig = getJdbcDebeziumConfig(jobConf); |
| 51 | + connectorConfig = jdbcDebeziumConfig.getConnectorConfig(); |
| 52 | + this.subtaskId = subtaskId; |
| 53 | + this.offset = new HashMap<>(); |
| 54 | + } |
| 55 | + |
| 56 | + public AbstractJdbcDebeziumConfig getJdbcDebeziumConfig(BitSailConfiguration jobConf) { |
| 57 | + return AbstractJdbcDebeziumConfig.fromBitSailConf(jobConf); |
| 58 | + } |
| 59 | + |
| 60 | + public abstract SplitChangeEventStreamingTaskContext getSplitReaderTaskContext(); |
| 61 | + |
| 62 | + public abstract void testConnectionAndValidBinlogConfiguration(RelationalDatabaseConnectorConfig connectorConfig) throws IOException; |
| 63 | + |
| 64 | + public void inititialzeSplitReader(BinlogSplit split) { |
| 65 | + splitChangeEventStreamingTaskContext = getSplitReaderTaskContext(); |
| 66 | + this.offset = new HashMap<>(); |
| 67 | + this.queue = new ChangeEventQueue.Builder<DataChangeEvent>() |
| 68 | + .pollInterval(connectorConfig.getPollInterval()) |
| 69 | + .maxBatchSize(connectorConfig.getMaxBatchSize()) |
| 70 | + .maxQueueSize(connectorConfig.getMaxQueueSize()) |
| 71 | + .maxQueueSizeInBytes(connectorConfig.getMaxQueueSizeInBytes()) |
| 72 | + .loggingContextSupplier(() -> taskContext.configureLoggingContext(splitChangeEventStreamingTaskContext.threadNamePrefix())) |
| 73 | + .buffering() |
| 74 | + .build(); |
| 75 | + this.batch = new ArrayList<>(); |
| 76 | + this.recordIterator = this.batch.iterator(); |
| 77 | + splitChangeEventStreamingTaskContext.initializeSplitReaderTaskContext(connectorConfig, this.queue); |
| 78 | + splitChangeEventStreamingTaskController = new SplitChangeEventStreamingTaskController(splitChangeEventStreamingTaskContext, this.subtaskId); |
| 79 | + } |
| 80 | + |
| 81 | + /** |
| 82 | + * Try to start streaming task to drain change event into target queue |
| 83 | + * @param split |
| 84 | + * @throws IOException |
| 85 | + * @throws InterruptedException |
| 86 | + */ |
| 87 | + @Override |
| 88 | + public void readSplit(BinlogSplit split) throws IOException, InterruptedException { |
| 89 | + inititialzeSplitReader(split); |
| 90 | + splitChangeEventStreamingTaskContext.testConnectionAndValidBinlogConfiguration(); |
| 91 | + splitChangeEventStreamingTaskController.launchSplitReaderTask(); |
| 92 | + } |
| 93 | + |
| 94 | + /** |
| 95 | + * get the binlog offset being processed |
| 96 | + * @return |
| 97 | + */ |
| 98 | + @Override |
| 99 | + public Map<String, String> getOffset() { |
| 100 | + Map<String, String> offsetToStore = new HashMap<>(); |
| 101 | + this.offset.forEach((k, v) -> offsetToStore.put(k, v.toString())); |
| 102 | + return offsetToStore; |
| 103 | + } |
| 104 | + |
| 105 | + /** |
| 106 | + * close task and resources |
| 107 | + */ |
| 108 | + @Override |
| 109 | + public void close() { |
| 110 | + try { |
| 111 | + splitChangeEventStreamingTaskController.closeTask(); |
| 112 | + } catch (Exception e) { |
| 113 | + LOG.error("Failed to close change event streaming task: {}", e.getMessage()); |
| 114 | + } |
| 115 | + |
| 116 | + try { |
| 117 | + splitChangeEventStreamingTaskContext.closeContextResources(); |
| 118 | + } catch (Exception e) { |
| 119 | + LOG.error("Failed to close resources of streaming task context: {}", e.getMessage()); |
| 120 | + } |
| 121 | + } |
| 122 | + |
| 123 | + @Override |
| 124 | + public Row poll() { |
| 125 | + SourceRecord record = this.recordIterator.next(); |
| 126 | + this.offset = record.sourceOffset(); |
| 127 | + LOG.info("OFFSET:" + record.sourceOffset()); |
| 128 | + LOG.info("poll one record {}", record.value()); |
| 129 | + // TODO: Build BitSail row and return |
| 130 | + return null; |
| 131 | + } |
| 132 | + |
| 133 | + /** |
| 134 | + * To judge whether current split has next record |
| 135 | + * @return |
| 136 | + * @throws Exception |
| 137 | + */ |
| 138 | + @Override |
| 139 | + public boolean hasNext() throws Exception { |
| 140 | + if (this.recordIterator.hasNext()) { |
| 141 | + return true; |
| 142 | + } else { |
| 143 | + return pollNextBatch(); |
| 144 | + } |
| 145 | + } |
| 146 | + |
| 147 | + @Override |
| 148 | + public boolean isCompleted() { |
| 149 | + return !splitChangeEventStreamingTaskController.isRunning(); |
| 150 | + } |
| 151 | + |
| 152 | + private boolean pollNextBatch() throws InterruptedException { |
| 153 | + 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()); |
| 164 | + } |
| 165 | + this.recordIterator = this.batch.iterator(); |
| 166 | + return true; |
| 167 | + } |
| 168 | + return false; |
| 169 | + } |
| 170 | +} |
0 commit comments