2525import org .apache .doris .job .cdc .request .FetchRecordRequest ;
2626import org .apache .doris .job .cdc .request .WriteRecordRequest ;
2727import org .apache .doris .job .cdc .split .BinlogSplit ;
28- import org .apache .doris .job .cdc .split .SnapshotSplit ;
2928
3029import org .apache .commons .collections .CollectionUtils ;
3130import org .apache .flink .api .connector .source .SourceSplit ;
@@ -125,32 +124,19 @@ private RecordWithMeta buildRecordResponse(
125124 sourceReader .extractSnapshotStateOffset (readResult .getSplitState ());
126125 offsetRes .put (SPLIT_ID , split .splitId ());
127126 recordResponse .setMeta (offsetRes );
128- return recordResponse ;
129127 }
128+
130129 // set meta for binlog event
131130 if (sourceReader .isBinlogSplit (split )) {
132131 Map <String , String > offsetRes =
133132 sourceReader .extractBinlogStateOffset (readResult .getSplitState ());
134133 offsetRes .put (SPLIT_ID , BinlogSplit .BINLOG_SPLIT_ID );
135- }
136- }
137-
138- // no data in this split, set meta info
139- if (CollectionUtils .isEmpty (recordResponse .getRecords ())) {
140- if (sourceReader .isBinlogSplit (split )) {
141- Map <String , String > offsetRes =
142- sourceReader .extractBinlogOffset (readResult .getSplit ());
143- offsetRes .put (SPLIT_ID , BinlogSplit .BINLOG_SPLIT_ID );
144134 recordResponse .setMeta (offsetRes );
145- } else {
146- SnapshotSplit snapshotSplit =
147- objectMapper .convertValue (fetchRecord .getMeta (), SnapshotSplit .class );
148- Map <String , String > meta = new HashMap <>();
149- meta .put (SPLIT_ID , snapshotSplit .getSplitId ());
150- // chunk no data
151- recordResponse .setMeta (meta );
152135 }
136+ } else {
137+ throw new RuntimeException ("split state is null" );
153138 }
139+
154140 sourceReader .commitSourceOffset (fetchRecord .getJobId (), readResult .getSplit ());
155141 return recordResponse ;
156142 }
@@ -188,7 +174,6 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception
188174 SourceReader sourceReader = Env .getCurrentEnv ().getReader (writeRecordRequest );
189175 DorisBatchStreamLoad batchStreamLoad = null ;
190176 Map <String , String > metaResponse = new HashMap <>();
191- boolean hasData = false ;
192177 long scannedRows = 0L ;
193178 long scannedBytes = 0L ;
194179 SplitReadResult readResult = null ;
@@ -215,21 +200,12 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception
215200 if (!CollectionUtils .isEmpty (serializedRecords )) {
216201 String database = writeRecordRequest .getTargetDb ();
217202 String table = extractTable (element );
218- hasData = true ;
219203 for (String record : serializedRecords ) {
220204 scannedRows ++;
221205 byte [] dataBytes = record .getBytes ();
222206 scannedBytes += dataBytes .length ;
223207 batchStreamLoad .writeRecord (database , table , dataBytes );
224208 }
225-
226- if (sourceReader .isBinlogSplit (readResult .getSplit ())) {
227- // put offset for event
228- Map <String , String > lastMeta =
229- sourceReader .extractBinlogStateOffset (readResult .getSplitState ());
230- lastMeta .put (SPLIT_ID , BinlogSplit .BINLOG_SPLIT_ID );
231- metaResponse = lastMeta ;
232- }
233209 }
234210 // Check if maxInterval has been exceeded
235211 long elapsedTime = System .currentTimeMillis () - startTime ;
@@ -245,29 +221,29 @@ public void writeRecords(WriteRecordRequest writeRecordRequest) throws Exception
245221 }
246222
247223 try {
248- if (!hasData ) {
249- // todo: need return the lastest heartbeat offset, means the maximum offset that the
250- // current job can recover.
224+ if (readResult .getSplitState () != null ) {
225+ // Set meta information for hw
226+ if (sourceReader .isSnapshotSplit (readResult .getSplit ())) {
227+ Map <String , String > offsetRes =
228+ sourceReader .extractSnapshotStateOffset (readResult .getSplitState ());
229+ offsetRes .put (SPLIT_ID , readResult .getSplit ().splitId ());
230+ metaResponse = offsetRes ;
231+ }
232+
233+ // set meta for binlog event
251234 if (sourceReader .isBinlogSplit (readResult .getSplit ())) {
252235 Map <String , String > offsetRes =
253- sourceReader .extractBinlogOffset (readResult .getSplit ());
236+ sourceReader .extractBinlogStateOffset (readResult .getSplitState ());
254237 offsetRes .put (SPLIT_ID , BinlogSplit .BINLOG_SPLIT_ID );
255- batchStreamLoad .commitOffset (offsetRes , scannedRows , scannedBytes );
256- return ;
257- } else {
258- throw new RuntimeException ("should not happen" );
238+ metaResponse = offsetRes ;
259239 }
240+ } else {
241+ throw new RuntimeException ("split state is null" );
260242 }
261243
262244 // wait all stream load finish
263245 batchStreamLoad .forceFlush ();
264- // update offset meta
265- if (sourceReader .isSnapshotSplit (readResult .getSplit ())) {
266- Map <String , String > offsetRes =
267- sourceReader .extractSnapshotStateOffset (readResult .getSplitState ());
268- offsetRes .put (SPLIT_ID , readResult .getSplit ().splitId ());
269- metaResponse = offsetRes ;
270- }
246+
271247 // request fe api
272248 batchStreamLoad .commitOffset (metaResponse , scannedRows , scannedBytes );
273249
0 commit comments