Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,8 @@ public MySqlPipelineRecordEmitter(
super(
debeziumDeserializationSchema,
sourceReaderMetrics,
sourceConfig.isIncludeSchemaChanges());
sourceConfig.isIncludeSchemaChanges(),
false); // Explicitly disable transaction metadata events
this.debeziumDeserializationSchema = debeziumDeserializationSchema;
this.sourceConfig = sourceConfig;
this.alreadySendCreateTableTables = new HashSet<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -132,7 +132,8 @@ public static <T> MySqlSourceBuilder<T> builder() {
new MySqlRecordEmitter<>(
deserializationSchema,
sourceReaderMetrics,
sourceConfig.isIncludeSchemaChanges()));
sourceConfig.isIncludeSchemaChanges(),
sourceConfig.isIncludeTransactionMetadataEvents()));
}

MySqlSource(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,6 +198,12 @@ public MySqlSourceBuilder<T> includeSchemaChanges(boolean includeSchemaChanges)
return this;
}

/** Whether the {@link MySqlSource} should output the transaction metadata events or not. */
public MySqlSourceBuilder<T> includeTransactionMetadataEvents(boolean includeTransactionMetadataEvents) {
this.configFactory.includeTransactionMetadataEvents(includeTransactionMetadataEvents);
return this;
}

/** Whether the {@link MySqlSource} should scan the newly added tables or not. */
public MySqlSourceBuilder<T> scanNewlyAddedTableEnabled(boolean scanNewlyAddedTableEnabled) {
this.configFactory.scanNewlyAddedTableEnabled(scanNewlyAddedTableEnabled);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,6 +62,7 @@ public class MySqlSourceConfig implements Serializable {
private final double distributionFactorUpper;
private final double distributionFactorLower;
private final boolean includeSchemaChanges;
private final boolean includeTransactionMetadataEvents;
private final boolean scanNewlyAddedTableEnabled;
private final boolean closeIdleReaders;
private final Properties jdbcProperties;
Expand Down Expand Up @@ -99,6 +100,7 @@ public class MySqlSourceConfig implements Serializable {
double distributionFactorUpper,
double distributionFactorLower,
boolean includeSchemaChanges,
boolean includeTransactionMetadataEvents,
boolean scanNewlyAddedTableEnabled,
boolean closeIdleReaders,
Properties dbzProperties,
Expand Down Expand Up @@ -128,6 +130,7 @@ public class MySqlSourceConfig implements Serializable {
this.distributionFactorUpper = distributionFactorUpper;
this.distributionFactorLower = distributionFactorLower;
this.includeSchemaChanges = includeSchemaChanges;
this.includeTransactionMetadataEvents = includeTransactionMetadataEvents;
this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled;
this.closeIdleReaders = closeIdleReaders;
this.dbzProperties = checkNotNull(dbzProperties);
Expand Down Expand Up @@ -227,6 +230,10 @@ public boolean isIncludeSchemaChanges() {
return includeSchemaChanges;
}

public boolean isIncludeTransactionMetadataEvents() {
return includeTransactionMetadataEvents;
}

public boolean isScanNewlyAddedTableEnabled() {
return scanNewlyAddedTableEnabled;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@ public class MySqlSourceConfigFactory implements Serializable {
private double distributionFactorLower =
MySqlSourceOptions.CHUNK_KEY_EVEN_DISTRIBUTION_FACTOR_LOWER_BOUND.defaultValue();
private boolean includeSchemaChanges = false;
private boolean includeTransactionMetadataEvents = false;
private boolean scanNewlyAddedTableEnabled = false;
private boolean closeIdleReaders = false;
private Properties jdbcProperties;
Expand Down Expand Up @@ -235,6 +236,12 @@ public MySqlSourceConfigFactory includeSchemaChanges(boolean includeSchemaChange
return this;
}

/** Whether the {@link MySqlSource} should output the transaction metadata events or not. */
public MySqlSourceConfigFactory includeTransactionMetadataEvents(boolean includeTransactionMetadataEvents) {
this.includeTransactionMetadataEvents = includeTransactionMetadataEvents;
return this;
}

/** Whether the {@link MySqlSource} should scan the newly added tables or not. */
public MySqlSourceConfigFactory scanNewlyAddedTableEnabled(boolean scanNewlyAddedTableEnabled) {
this.scanNewlyAddedTableEnabled = scanNewlyAddedTableEnabled;
Expand Down Expand Up @@ -359,6 +366,8 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) {
// Note: the includeSchemaChanges parameter is used to control emitting the schema record,
// only DataStream API program need to emit the schema record, the Table API need not
props.setProperty("include.schema.changes", String.valueOf(true));
// enable transaction metadata if includeTransactionMetadataEvents is true
props.setProperty("provide.transaction.metadata", String.valueOf(includeTransactionMetadataEvents));
// disable the offset flush totally
props.setProperty("offset.flush.interval.ms", String.valueOf(Long.MAX_VALUE));
// disable tombstones
Expand Down Expand Up @@ -412,6 +421,7 @@ public MySqlSourceConfig createConfig(int subtaskId, String serverName) {
distributionFactorUpper,
distributionFactorLower,
includeSchemaChanges,
includeTransactionMetadataEvents,
scanNewlyAddedTableEnabled,
closeIdleReaders,
props,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,15 +53,18 @@ public class MySqlRecordEmitter<T> implements RecordEmitter<SourceRecords, T, My
private final DebeziumDeserializationSchema<T> debeziumDeserializationSchema;
private final MySqlSourceReaderMetrics sourceReaderMetrics;
private final boolean includeSchemaChanges;
private final boolean includeTransactionMetadataEvents;
private final OutputCollector<T> outputCollector;

public MySqlRecordEmitter(
DebeziumDeserializationSchema<T> debeziumDeserializationSchema,
MySqlSourceReaderMetrics sourceReaderMetrics,
boolean includeSchemaChanges) {
boolean includeSchemaChanges,
boolean includeTransactionMetadataEvents) {
this.debeziumDeserializationSchema = debeziumDeserializationSchema;
this.sourceReaderMetrics = sourceReaderMetrics;
this.includeSchemaChanges = includeSchemaChanges;
this.includeTransactionMetadataEvents = includeTransactionMetadataEvents;
this.outputCollector = new OutputCollector<>();
}

Expand Down Expand Up @@ -102,6 +105,11 @@ protected void processElement(
emitElement(element, output);
} else if (RecordUtils.isHeartbeatEvent(element)) {
updateStartingOffsetForSplit(splitState, element);
} else if (RecordUtils.isTransactionMetadataEvent(element)) {
updateStartingOffsetForSplit(splitState, element);
if (includeTransactionMetadataEvents) {
emitElement(element, output);
}
} else {
// unknown element
LOG.info("Meet unknown element {}, just skip.", element);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -73,6 +73,8 @@ private RecordUtils() {}
"io.debezium.connector.mysql.SchemaChangeKey";
public static final String SCHEMA_HEARTBEAT_EVENT_KEY_NAME =
"io.debezium.connector.common.Heartbeat";
public static final String SCHEMA_TRANSACTION_METADATA_EVENT_KEY_NAME =
"io.debezium.connector.common.TransactionMetadataKey";
private static final DocumentReader DOCUMENT_READER = DocumentReader.defaultReader();

/** Converts a {@link ResultSet} row to an array of Objects. */
Expand Down Expand Up @@ -339,6 +341,18 @@ public static boolean isHeartbeatEvent(SourceRecord record) {
&& SCHEMA_HEARTBEAT_EVENT_KEY_NAME.equalsIgnoreCase(valueSchema.name());
}

/**
* Check whether the given source record is a transaction metadata event (BEGIN or END).
*
* <p>Transaction events are emitted by Debezium to mark transaction boundaries when
* provide.transaction.metadata is enabled.
*/
public static boolean isTransactionMetadataEvent(SourceRecord record) {
Schema keySchema = record.keySchema();
return keySchema != null
&& SCHEMA_TRANSACTION_METADATA_EVENT_KEY_NAME.equalsIgnoreCase(keySchema.name());
}

/**
* Return the finished snapshot split information.
*
Expand Down
Loading