@@ -108,8 +108,6 @@ public class MySqlSourceReader implements SourceReader {
108108 private static final FlinkJsonTableChangeSerializer TABLE_CHANGE_SERIALIZER =
109109 new FlinkJsonTableChangeSerializer ();
110110 private SourceRecordDeserializer <SourceRecord , List <String >> serializer ;
111- private SnapshotSplitReader snapshotReader ;
112- private BinlogSplitReader binlogReader ;
113111 private DebeziumReader <SourceRecords , MySqlSplit > currentReader ;
114112 private Map <TableId , TableChanges .TableChange > tableSchemas ;
115113 private SplitRecords currentSplitRecords ;
@@ -180,7 +178,7 @@ public SplitReadResult readSplitRecords(JobBaseRecordRequest baseReq) throws Exc
180178 SplitRecords currentSplitRecords = this .getCurrentSplitRecords ();
181179 if (currentSplitRecords == null ) {
182180 DebeziumReader <SourceRecords , MySqlSplit > currentReader = this .getCurrentReader ();
183- if (currentReader == null || baseReq . isReload () ) {
181+ if (baseReq . isReload () || currentReader == null ) {
184182 LOG .info (
185183 "No current reader or reload {}, create new split reader" ,
186184 baseReq .isReload ());
@@ -446,7 +444,7 @@ private SplitRecords pollSplitRecordsWithSplit(MySqlSplit split, JobBaseConfig j
446444 sourceRecords =
447445 pollUntilDataAvailable (currentReader , Constants .POLL_SPLIT_RECORDS_TIMEOUTS , 500 );
448446 if (currentReader instanceof SnapshotSplitReader ) {
449- closeSnapshotReader ();
447+ closeCurrentReader ();
450448 }
451449 return new SplitRecords (currentSplitId , sourceRecords .iterator ());
452450 }
@@ -514,61 +512,33 @@ private SourceRecords pollUntilDataAvailable(
514512
515513 private SnapshotSplitReader getSnapshotSplitReader (JobBaseConfig config ) {
516514 MySqlSourceConfig sourceConfig = getSourceConfig (config );
517- SnapshotSplitReader snapshotReader = this .getSnapshotReader ();
518- if (snapshotReader == null ) {
519- final MySqlConnection jdbcConnection =
520- DebeziumUtils .createMySqlConnection (sourceConfig );
521- final BinaryLogClient binaryLogClient =
522- DebeziumUtils .createBinaryClient (sourceConfig .getDbzConfiguration ());
523- final StatefulTaskContext statefulTaskContext =
524- new StatefulTaskContext (sourceConfig , binaryLogClient , jdbcConnection );
525- snapshotReader = new SnapshotSplitReader (statefulTaskContext , 0 );
526- this .setSnapshotReader (snapshotReader );
527- }
515+ final MySqlConnection jdbcConnection = DebeziumUtils .createMySqlConnection (sourceConfig );
516+ final BinaryLogClient binaryLogClient =
517+ DebeziumUtils .createBinaryClient (sourceConfig .getDbzConfiguration ());
518+ final StatefulTaskContext statefulTaskContext =
519+ new StatefulTaskContext (sourceConfig , binaryLogClient , jdbcConnection );
520+ SnapshotSplitReader snapshotReader = new SnapshotSplitReader (statefulTaskContext , 0 );
528521 return snapshotReader ;
529522 }
530523
531524 private BinlogSplitReader getBinlogSplitReader (JobBaseConfig config ) {
532525 MySqlSourceConfig sourceConfig = getSourceConfig (config );
533- BinlogSplitReader binlogReader = this .getBinlogReader ();
534- if (binlogReader == null ) {
535- final MySqlConnection jdbcConnection =
536- DebeziumUtils .createMySqlConnection (sourceConfig );
537- final BinaryLogClient binaryLogClient =
538- DebeziumUtils .createBinaryClient (sourceConfig .getDbzConfiguration ());
539- final StatefulTaskContext statefulTaskContext =
540- new StatefulTaskContext (sourceConfig , binaryLogClient , jdbcConnection );
541- binlogReader = new BinlogSplitReader (statefulTaskContext , 0 );
542- this .setBinlogReader (binlogReader );
543- }
526+ final MySqlConnection jdbcConnection = DebeziumUtils .createMySqlConnection (sourceConfig );
527+ final BinaryLogClient binaryLogClient =
528+ DebeziumUtils .createBinaryClient (sourceConfig .getDbzConfiguration ());
529+ final StatefulTaskContext statefulTaskContext =
530+ new StatefulTaskContext (sourceConfig , binaryLogClient , jdbcConnection );
531+ BinlogSplitReader binlogReader = new BinlogSplitReader (statefulTaskContext , 0 );
544532 return binlogReader ;
545533 }
546534
547- private void closeSnapshotReader () {
548- SnapshotSplitReader reusedSnapshotReader = this .getSnapshotReader ();
549- if (reusedSnapshotReader != null ) {
550- LOG .info (
551- "Close snapshot reader {}" , reusedSnapshotReader .getClass ().getCanonicalName ());
552- reusedSnapshotReader .close ();
553- DebeziumReader <SourceRecords , MySqlSplit > currentReader = this .getCurrentReader ();
554- if (reusedSnapshotReader == currentReader ) {
555- this .setCurrentReader (null );
556- }
557- this .setSnapshotReader (null );
558- }
559- }
560-
561- private void closeBinlogReader () {
562- BinlogSplitReader reusedBinlogReader = this .getBinlogReader ();
563- if (reusedBinlogReader != null ) {
564- LOG .info ("Close binlog reader {}" , reusedBinlogReader .getClass ().getCanonicalName ());
565- reusedBinlogReader .close ();
566- DebeziumReader <SourceRecords , MySqlSplit > currentReader = this .getCurrentReader ();
567- if (reusedBinlogReader == currentReader ) {
568- this .setCurrentReader (null );
569- }
570- this .setBinlogReader (null );
535+ private void closeCurrentReader () {
536+ DebeziumReader <SourceRecords , MySqlSplit > currentReader = this .getCurrentReader ();
537+ if (currentReader != null ) {
538+ LOG .info ("Close current reader {}" , currentReader .getClass ().getCanonicalName ());
539+ currentReader .close ();
571540 }
541+ this .setCurrentReader (null );
572542 }
573543
574544 private MySqlSourceConfig getSourceConfig (JobBaseConfig config ) {
@@ -719,8 +689,7 @@ public boolean isSnapshotSplit(SourceSplit split) {
719689 public void finishSplitRecords () {
720690 this .setCurrentSplitRecords (null );
721691 // Close after each read, the binlog client will occupy the connection.
722- closeBinlogReader ();
723- this .setCurrentReader (null );
692+ closeCurrentReader ();
724693 }
725694
726695 @ Override
@@ -777,8 +746,7 @@ private Map<TableId, TableChanges.TableChange> discoverTableSchemas(JobBaseConfi
777746 @ Override
778747 public void close (JobBaseConfig jobConfig ) {
779748 LOG .info ("Close source reader for job {}" , jobConfig .getJobId ());
780- closeSnapshotReader ();
781- closeBinlogReader ();
749+ closeCurrentReader ();
782750 currentReader = null ;
783751 currentSplitRecords = null ;
784752 if (tableSchemas != null ) {
0 commit comments