From 2fae5cd9a906426571f6a76c43da68edb10e82d9 Mon Sep 17 00:00:00 2001 From: twthorn Date: Thu, 28 Mar 2024 15:42:47 -0500 Subject: [PATCH 01/15] DBZ-7698 Add ordered transaction metadata for Vitess connector --- .../io/debezium/connector/vitess/Vgtid.java | 9 ++ .../vitess/VitessConnectorConfig.java | 20 ++++ .../vitess/VitessDatabaseSchema.java | 1 + .../connector/vitess/VitessOffsetContext.java | 13 ++- .../connector/vitess/VitessSchemaFactory.java | 75 ++++++++++++ .../VitessStreamingChangeEventSource.java | 4 +- .../connection/TransactionalMessage.java | 6 +- .../VStreamOutputMessageDecoder.java | 4 +- .../vitess/pipeline/txmetadata/Gtid.java | 82 +++++++++++++ .../txmetadata/VitessEpochProvider.java | 85 ++++++++++++++ .../VitessOrderedTransactionStructMaker.java | 42 +++++++ .../txmetadata/VitessRankProvider.java | 20 ++++ .../txmetadata/VitessTransactionInfo.java | 28 +++++ .../VitessTransactionOrderMetadata.java | 54 +++++++++ .../vitess/VitessChangeRecordEmitterTest.java | 6 +- .../connector/vitess/VitessConnectorIT.java | 72 +++++++++++- .../vitess/VitessOffsetContextTest.java | 27 +++++ .../vitess/pipeline/txmetadata/GtidTest.java | 26 +++++ .../txmetadata/VitessEpochProviderTest.java | 61 ++++++++++ .../txmetadata/VitessRankProviderTest.java | 24 ++++ .../VitessTransactionOrderMetadataTest.java | 109 ++++++++++++++++++ 21 files changed, 756 insertions(+), 12 deletions(-) create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfo.java create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/GtidTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java diff --git a/src/main/java/io/debezium/connector/vitess/Vgtid.java b/src/main/java/io/debezium/connector/vitess/Vgtid.java index cbbaf9c7..8cce24f8 100644 --- a/src/main/java/io/debezium/connector/vitess/Vgtid.java +++ b/src/main/java/io/debezium/connector/vitess/Vgtid.java @@ -87,6 +87,15 @@ public List getShardGtids() { return shardGtids; } + public ShardGtid getShardGtid(String shard) { + for (ShardGtid shardGtid : shardGtids) { + if (shardGtid.shard.equals(shard)) { + return shardGtid; + } + } + throw new RuntimeException("Gtid for shard missing, shard: " + shard + "vgtid: " + this.rawVgtid.toString()); + } + public boolean isSingleShard() { return rawVgtid.getShardGtidsCount() == 1; } diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index b5628eb4..786f9de8 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -31,8 +31,12 @@ import io.debezium.config.Field.ValidationOutput; import io.debezium.connector.SourceInfoStructMaker; import io.debezium.connector.vitess.connection.VitessTabletType; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionOrderMetadata; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.TemporalPrecisionMode; +import io.debezium.pipeline.txmetadata.TransactionOrderMetadata; +import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.relational.ColumnFilterMode; import io.debezium.relational.RelationalDatabaseConnectorConfig; @@ -417,6 +421,12 @@ private static int validateTimePrecisionMode(Configuration config, Field field, public static final Field SOURCE_INFO_STRUCT_MAKER = CommonConnectorConfig.SOURCE_INFO_STRUCT_MAKER .withDefault(VitessSourceInfoStructMaker.class.getName()); + public static final Field TRANSACTION_ORDER_METADATA_FIELD = CommonConnectorConfig.TRANSACTION_ORDER_METADATA_FIELD + .withDefault(VitessTransactionOrderMetadata.class.getName()); + + public static final Field TRANSACTION_STRUCT_MAKER = CommonConnectorConfig.TRANSACTION_STRUCT_MAKER + .withDefault(VitessOrderedTransactionStructMaker.class.getName()); + protected static final ConfigDefinition CONFIG_DEFINITION = RelationalDatabaseConnectorConfig.CONFIG_DEFINITION .edit() .name("Vitess") @@ -514,6 +524,16 @@ protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { return getSourceInfoStructMaker(SOURCE_INFO_STRUCT_MAKER, Module.name(), Module.version(), this); } + @Override + public TransactionOrderMetadata getTransactionOrderMetadata() { + return getTransactionOrderMetadata(TRANSACTION_ORDER_METADATA_FIELD); + } + + @Override + public TransactionStructMaker getTransactionStructMaker() { + return getTransactionStructMaker(TRANSACTION_STRUCT_MAKER); + } + public String getKeyspace() { return getConfig().getString(KEYSPACE); } diff --git a/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java b/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java index e747aeff..0ec23f44 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java +++ b/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java @@ -45,6 +45,7 @@ public VitessDatabaseSchema( schemaNameAdjuster, config.customConverterRegistry(), config.getSourceInfoStructMaker().schema(), + config.getTransactionStructMaker().getTransactionBlockSchema(), config.getFieldNamer(), false), false, diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 0573ce46..4797699e 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -19,6 +19,7 @@ import io.debezium.pipeline.CommonOffsetContext; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.pipeline.txmetadata.TransactionContextSupplier; import io.debezium.relational.TableId; import io.debezium.spi.schema.DataCollectionId; import io.debezium.util.Clock; @@ -50,8 +51,12 @@ public static VitessOffsetContext initialContext( VitessConnectorConfig connectorConfig, Clock clock) { LOGGER.info("No previous offset exists. Use default VGTID."); final Vgtid defaultVgtid = VitessReplicationConnection.defaultVgtid(connectorConfig); - return new VitessOffsetContext( - connectorConfig, defaultVgtid, clock.currentTimeAsInstant(), new TransactionContext()); + // use the other transaction context + TransactionContextSupplier supplier = new TransactionContextSupplier(connectorConfig); + TransactionContext transactionContext = supplier.newTransactionContext(); + VitessOffsetContext context = new VitessOffsetContext( + connectorConfig, defaultVgtid, clock.currentTimeAsInstant(), transactionContext); + return context; } /** @@ -144,11 +149,13 @@ public Loader(VitessConnectorConfig connectorConfig) { @Override public VitessOffsetContext load(Map offset) { final String vgtid = (String) offset.get(SourceInfo.VGTID_KEY); + TransactionContextSupplier supplier = new TransactionContextSupplier(connectorConfig); + TransactionContext transactionContext = supplier.loadTransactionContext(offset); return new VitessOffsetContext( connectorConfig, Vgtid.of(vgtid), null, - TransactionContext.load(offset)); + transactionContext); } } } diff --git a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java index 283d14ad..1e80de2c 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java +++ b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java @@ -5,6 +5,15 @@ */ package io.debezium.connector.vitess; +import java.util.HashSet; +import java.util.Set; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionOrderMetadata; +import io.debezium.data.Envelope; +import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.schema.SchemaFactory; public class VitessSchemaFactory extends SchemaFactory { @@ -18,4 +27,70 @@ public VitessSchemaFactory() { public static VitessSchemaFactory get() { return vitessSchemaFactoryObject; } + + public Schema getOrderedTransactionBlockSchema() { + return SchemaBuilder.struct().optional() + .name(TRANSACTION_BLOCK_SCHEMA_NAME) + .version(TRANSACTION_BLOCK_SCHEMA_VERSION) + .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_ID_KEY, Schema.STRING_SCHEMA) + .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, Schema.INT64_SCHEMA) + .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, Schema.INT64_SCHEMA) + .field(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) + .field(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) + .build(); + } + + @Override + public Envelope.Builder datatypeEnvelopeSchema() { + return new Envelope.Builder() { + private final SchemaBuilder builder = SchemaBuilder.struct() + .version(Envelope.SCHEMA_VERSION); + + private final Set missingFields = new HashSet<>(); + + @Override + public Envelope.Builder withSchema(Schema fieldSchema, String... fieldNames) { + for (String fieldName : fieldNames) { + builder.field(fieldName, fieldSchema); + } + return this; + } + + @Override + public Envelope.Builder withName(String name) { + builder.name(name); + return this; + } + + @Override + public Envelope.Builder withDoc(String doc) { + builder.doc(doc); + return this; + } + + @Override + public Envelope build() { + builder.field(Envelope.FieldName.OPERATION, Envelope.OPERATION_REQUIRED ? Schema.STRING_SCHEMA : Schema.OPTIONAL_STRING_SCHEMA); + builder.field(Envelope.FieldName.TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA); + builder.field(Envelope.FieldName.TIMESTAMP_US, Schema.OPTIONAL_INT64_SCHEMA); + builder.field(Envelope.FieldName.TIMESTAMP_NS, Schema.OPTIONAL_INT64_SCHEMA); + builder.field(Envelope.FieldName.TRANSACTION, getOrderedTransactionBlockSchema()); + checkFieldIsDefined(Envelope.FieldName.OPERATION); + checkFieldIsDefined(Envelope.FieldName.BEFORE); + checkFieldIsDefined(Envelope.FieldName.AFTER); + checkFieldIsDefined(Envelope.FieldName.SOURCE); + checkFieldIsDefined(Envelope.FieldName.TRANSACTION); + if (!missingFields.isEmpty()) { + throw new IllegalStateException("The envelope schema is missing field(s) " + String.join(", ", missingFields)); + } + return new Envelope(builder.build()); + } + + private void checkFieldIsDefined(String fieldName) { + if (builder.field(fieldName) == null) { + missingFields.add(fieldName); + } + } + }; + } } diff --git a/src/main/java/io/debezium/connector/vitess/VitessStreamingChangeEventSource.java b/src/main/java/io/debezium/connector/vitess/VitessStreamingChangeEventSource.java index 87d7b655..24a6cbac 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessStreamingChangeEventSource.java +++ b/src/main/java/io/debezium/connector/vitess/VitessStreamingChangeEventSource.java @@ -14,6 +14,7 @@ import io.debezium.connector.vitess.connection.ReplicationConnection; import io.debezium.connector.vitess.connection.ReplicationMessage; import io.debezium.connector.vitess.connection.ReplicationMessageProcessor; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionInfo; import io.debezium.pipeline.ErrorHandler; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.source.spi.StreamingChangeEventSource; @@ -96,7 +97,8 @@ private ReplicationMessageProcessor newReplicationMessageProcessor(VitessPartiti offsetContext.rotateVgtid(newVgtid, message.getCommitTime()); if (message.getOperation() == ReplicationMessage.Operation.BEGIN) { // send to transaction topic - dispatcher.dispatchTransactionStartedEvent(partition, message.getTransactionId(), offsetContext, message.getCommitTime()); + VitessTransactionInfo transactionInfo = new VitessTransactionInfo(message.getTransactionId(), message.getShard()); + dispatcher.dispatchTransactionStartedEvent(partition, transactionInfo, offsetContext, message.getCommitTime()); } else if (message.getOperation() == ReplicationMessage.Operation.COMMIT) { // send to transaction topic diff --git a/src/main/java/io/debezium/connector/vitess/connection/TransactionalMessage.java b/src/main/java/io/debezium/connector/vitess/connection/TransactionalMessage.java index 0b368858..76c38796 100644 --- a/src/main/java/io/debezium/connector/vitess/connection/TransactionalMessage.java +++ b/src/main/java/io/debezium/connector/vitess/connection/TransactionalMessage.java @@ -14,11 +14,13 @@ public class TransactionalMessage implements ReplicationMessage { private final String transactionId; private final Instant commitTime; private final Operation operation; + private final String shard; - public TransactionalMessage(Operation operation, String transactionId, Instant commitTime) { + public TransactionalMessage(Operation operation, String transactionId, Instant commitTime, String shard) { this.transactionId = transactionId; this.commitTime = commitTime; this.operation = operation; + this.shard = shard; } @Override @@ -43,7 +45,7 @@ public String getTable() { @Override public String getShard() { - throw new UnsupportedOperationException(); + return shard; } @Override diff --git a/src/main/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoder.java b/src/main/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoder.java index 36461353..9e2a4a0e 100644 --- a/src/main/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoder.java +++ b/src/main/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoder.java @@ -121,7 +121,7 @@ private void handleBeginMessage(Binlogdata.VEvent vEvent, ReplicationMessageProc } LOGGER.trace("Timestamp of begin transaction: {}", eventTimestamp); processor.process( - new TransactionalMessage(Operation.BEGIN, transactionId, eventTimestamp), newVgtid, false); + new TransactionalMessage(Operation.BEGIN, transactionId, eventTimestamp, vEvent.getShard()), newVgtid, false); } private void handleCommitMessage( @@ -135,7 +135,7 @@ private void handleCommitMessage( } LOGGER.trace("Timestamp of commit transaction: {}", commitTimestamp); processor.process( - new TransactionalMessage(Operation.COMMIT, transactionId, eventTimestamp), newVgtid, false); + new TransactionalMessage(Operation.COMMIT, transactionId, eventTimestamp, vEvent.getShard()), newVgtid, false); } private void decodeRows(Binlogdata.VEvent vEvent, ReplicationMessageProcessor processor, Vgtid newVgtid, boolean isLastRowEventOfTransaction) diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java new file mode 100644 index 00000000..bf8b10f1 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java @@ -0,0 +1,82 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +class Gtid { + + public String getVersion() { + return version; + } + + private String version = ""; + + public Set getHosts() { + return hosts; + } + + private Set hosts = new HashSet(); + + public List getSequenceValues() { + return sequenceValues; + } + + private List sequenceValues = new ArrayList(); + + private static final String PREFIX_LAST_CHAR = "/"; + + private static int getVersionEndIndex(String transactionId) { + return transactionId.indexOf(PREFIX_LAST_CHAR); + } + + private static String trimVersion(String transactionId) { + int index = getVersionEndIndex(transactionId); + if (index != -1) { + return transactionId.substring(index + 1); + } + return transactionId; + } + + private void initializeVersion(String transactionId) { + int index = getVersionEndIndex(transactionId); + if (index != -1) { + this.version = transactionId.substring(0, index); + } + } + + Gtid(String transactionId) { + initializeVersion(transactionId); + parseGtid(transactionId); + } + + private void parseGtid(String transactionId) { + transactionId = trimVersion(transactionId); + String[] transactions = transactionId.split(","); + for (String transaction : transactions) { + String[] parts = transaction.split(":"); + String hostname = parts[0]; + hosts.add(hostname); + String maxSequenceValue = parts[1].split("-")[1]; + sequenceValues.add(maxSequenceValue); + } + } + + public boolean isHostSetEqual(Gtid hosts) { + return this.hosts.equals(hosts.hosts); + } + + public boolean isHostSetSupersetOf(Gtid previousHosts) { + return this.hosts.containsAll(previousHosts.hosts); + } + + public boolean isHostSetSubsetOf(Gtid previousHosts) { + return previousHosts.hosts.containsAll(this.hosts); + } +} diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java new file mode 100644 index 00000000..7b722d27 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -0,0 +1,85 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import java.util.HashMap; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +import io.debezium.connector.vitess.Vgtid; + +public class VitessEpochProvider { + + private static final Logger LOGGER = LoggerFactory.getLogger(VitessEpochProvider.class); + private Map shardToEpoch = new HashMap<>(); + private static final ObjectMapper MAPPER = new ObjectMapper(); + + public Long getEpoch(Long previousEpoch, String previousTransactionId, String transactionId) { + Gtid previousGtid = new Gtid(previousTransactionId); + Gtid gtid = new Gtid(transactionId); + if (previousGtid.isHostSetEqual(gtid) || gtid.isHostSetSupersetOf(previousGtid)) { + return previousEpoch; + } + else if (gtid.isHostSetSubsetOf(previousGtid)) { + return previousEpoch + 1; + } + else { + LOGGER.error( + "Error determining epoch, previous host set: {}, host set: {}", + previousGtid, gtid); + throw new RuntimeException("Can't determine epoch"); + } + } + + public Map store(Map offset) { + try { + offset.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); + return offset; + } + catch (JsonProcessingException e) { + throw new RuntimeException("Cannot store epoch: " + shardToEpoch.toString()); + } + } + + public void load(Map offsets) { + try { + String shardToEpochString = (String) offsets.get(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH); + if (shardToEpochString != null) { + shardToEpoch = MAPPER.readValue(shardToEpochString, new TypeReference>() { + }); + } + } + catch (JsonProcessingException e) { + throw new RuntimeException("Cannot read epoch: " + shardToEpoch.toString()); + } + } + + public Long getEpoch(String shard, String previousVgtidString, String vgtidString) { + if (previousVgtidString == null) { + long epoch = 0L; + storeEpoch(shard, epoch); + return epoch; + } + Vgtid vgtid = Vgtid.of(vgtidString); + Vgtid previousVgtid = Vgtid.of(previousVgtidString); + String previousGtid = previousVgtid.getShardGtid(shard).getGtid(); + String gtid = vgtid.getShardGtid(shard).getGtid(); + long previousEpoch = shardToEpoch.get(shard); + long currentEpoch = getEpoch(previousEpoch, previousGtid, gtid); + storeEpoch(shard, currentEpoch); + return currentEpoch; + } + + private void storeEpoch(String shard, long epoch) { + shardToEpoch.put(shard, epoch); + } +} diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java new file mode 100644 index 00000000..67b58e13 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java @@ -0,0 +1,42 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import io.debezium.connector.vitess.VitessSchemaFactory; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.txmetadata.AbstractTransactionStructMaker; +import io.debezium.pipeline.txmetadata.OrderedTransactionContext; +import io.debezium.pipeline.txmetadata.TransactionStructMaker; +import io.debezium.spi.schema.DataCollectionId; + +public class VitessOrderedTransactionStructMaker extends AbstractTransactionStructMaker implements TransactionStructMaker { + + @Override + public Struct prepareTxStruct(OffsetContext offsetContext, DataCollectionId source) { + Struct struct = super.prepareTxStruct(offsetContext, source); + return addOrderMetadata(struct, offsetContext); + } + + private Struct addOrderMetadata(Struct struct, OffsetContext offsetContext) { + VitessTransactionOrderMetadata metadata = getVitessTransactionOrderMetadata(offsetContext); + struct.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_RANK, metadata.transactionRank.toString()); + struct.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, metadata.transactionEpoch); + return struct; + } + + private VitessTransactionOrderMetadata getVitessTransactionOrderMetadata(OffsetContext offsetContext) { + OrderedTransactionContext context = (OrderedTransactionContext) offsetContext.getTransactionContext(); + return (VitessTransactionOrderMetadata) context.getTransactionOrderMetadata(); + } + + @Override + public Schema getTransactionBlockSchema() { + return VitessSchemaFactory.get().getOrderedTransactionBlockSchema(); + } +} diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java new file mode 100644 index 00000000..ebda013f --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java @@ -0,0 +1,20 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import java.math.BigInteger; + +public class VitessRankProvider { + + public BigInteger getRank(String transactionId) { + Gtid gtid = new Gtid(transactionId); + BigInteger rank = new BigInteger("0"); + for (String sequenceValue : gtid.getSequenceValues()) { + rank = rank.add(new BigInteger(sequenceValue)); + } + return rank; + } +} diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfo.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfo.java new file mode 100644 index 00000000..aa02a910 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfo.java @@ -0,0 +1,28 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import io.debezium.pipeline.txmetadata.TransactionInfo; + +public class VitessTransactionInfo implements TransactionInfo { + + private final String vgtid; + private final String shard; + + public VitessTransactionInfo(String vgtid, String shard) { + this.vgtid = vgtid; + this.shard = shard; + } + + @Override + public String getTransactionId() { + return this.vgtid; + } + + public String getShard() { + return this.shard; + } +} diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java new file mode 100644 index 00000000..b3d84aa8 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java @@ -0,0 +1,54 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import java.math.BigInteger; +import java.util.Map; + +import io.debezium.connector.vitess.Vgtid; +import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.pipeline.txmetadata.TransactionInfo; +import io.debezium.pipeline.txmetadata.TransactionOrderMetadata; + +public class VitessTransactionOrderMetadata implements TransactionOrderMetadata { + + public static final String OFFSET_TRANSACTION_EPOCH = "transaction_epoch"; + public static final String OFFSET_TRANSACTION_RANK = "transaction_rank"; + protected String previousTransactionId = null; + protected Long transactionEpoch; + protected BigInteger transactionRank; + private VitessEpochProvider epochProvider = new VitessEpochProvider(); + private VitessRankProvider rankProvider = new VitessRankProvider(); + + @Override + public Map store(Map offset) { + return epochProvider.store(offset); + } + + @Override + public void load(Map offsets) { + this.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); + epochProvider.load(offsets); + } + + @Override + public void beginTransaction(TransactionInfo transactionInfo) { + VitessTransactionInfo vitessTransactionInfo = (VitessTransactionInfo) transactionInfo; + beginTransaction(vitessTransactionInfo.getShard(), vitessTransactionInfo.getTransactionId()); + } + + @Override + public void endTransaction() { + this.transactionEpoch = null; + this.transactionRank = null; + } + + private void beginTransaction(String shard, String vgtid) { + this.transactionEpoch = this.epochProvider.getEpoch(shard, this.previousTransactionId, vgtid); + this.transactionRank = this.rankProvider.getRank(Vgtid.of(vgtid).getShardGtid(shard).getGtid()); + this.previousTransactionId = vgtid; + } +} diff --git a/src/test/java/io/debezium/connector/vitess/VitessChangeRecordEmitterTest.java b/src/test/java/io/debezium/connector/vitess/VitessChangeRecordEmitterTest.java index aa3f30cf..668d2e6a 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessChangeRecordEmitterTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessChangeRecordEmitterTest.java @@ -125,7 +125,8 @@ public void shouldGetOldAndNewColumnValuesFromUpdate() { @Test(expected = UnsupportedOperationException.class) public void shouldNotSupportBeginMessage() { // setup fixture - ReplicationMessage message = new TransactionalMessage(ReplicationMessage.Operation.BEGIN, AnonymousValue.getString(), AnonymousValue.getInstant()); + ReplicationMessage message = new TransactionalMessage(ReplicationMessage.Operation.BEGIN, AnonymousValue.getString(), AnonymousValue.getInstant(), + AnonymousValue.getString()); // exercise SUT new VitessChangeRecordEmitter( @@ -140,7 +141,8 @@ public void shouldNotSupportBeginMessage() { @Test(expected = UnsupportedOperationException.class) public void shouldNotSupportCommitMessage() { // setup fixture - ReplicationMessage message = new TransactionalMessage(ReplicationMessage.Operation.COMMIT, AnonymousValue.getString(), AnonymousValue.getInstant()); + ReplicationMessage message = new TransactionalMessage(ReplicationMessage.Operation.COMMIT, AnonymousValue.getString(), AnonymousValue.getInstant(), + AnonymousValue.getString()); // exercise SUT new VitessChangeRecordEmitter( diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 02890042..4de99a8b 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -469,6 +469,69 @@ public void shouldUsePrevVgtidAsOffsetWhenNoVgtidInGrpcResponse() throws Excepti Testing.Print.enable(); } + @Test + @FixFor("") + public void shouldProvideOrderedTransactionMetadata() throws Exception { + TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); + TestHelper.applyVSchema("vitess_vschema.json"); + startConnector(config -> config + .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) + .with(VitessConnectorConfig.SHARD, "-80,80-"), + true, + "80-"); + assertConnectorIsRunning(); + + Vgtid baseVgtid = TestHelper.getCurrentVgtid(); + int expectedRecordsCount = 1; + consumer = testConsumer(expectedRecordsCount + 2); + + String rowValue = "(1, 1, 12, 12, 123, 123, 1234, 1234, 12345, 12345, 18446744073709551615, 1.5, 2.5, 12.34, true)"; + String insertQuery = "INSERT INTO numeric_table (" + + "tinyint_col," + + "tinyint_unsigned_col," + + "smallint_col," + + "smallint_unsigned_col," + + "mediumint_col," + + "mediumint_unsigned_col," + + "int_col," + + "int_unsigned_col," + + "bigint_col," + + "bigint_unsigned_col," + + "bigint_unsigned_overflow_col," + + "float_col," + + "double_col," + + "decimal_col," + + "boolean_col)" + + " VALUES " + rowValue; + StringBuilder insertRows = new StringBuilder().append(insertQuery); + for (int i = 1; i < expectedRecordsCount; i++) { + insertRows.append(", ").append(rowValue); + } + + String insertRowsStatement = insertRows.toString(); + + // exercise SUT + executeAndWait(insertRowsStatement, TEST_SHARDED_KEYSPACE); + // First transaction. + SourceRecord beginRecord = assertRecordBeginSourceRecord(); + assertThat(beginRecord.sourceOffset()).containsKey("transaction_epoch"); + String expectedTxId1 = ((Struct) beginRecord.value()).getString("id"); + for (int i = 1; i <= expectedRecordsCount; i++) { + SourceRecord record = assertRecordInserted(TEST_SHARDED_KEYSPACE + ".numeric_table", TestHelper.PK_FIELD); + final Struct txn = ((Struct) record.value()).getStruct("transaction"); + String txId = txn.getString("id"); + assertThat(txId).isNotNull(); + assertThat(txId).isEqualTo(expectedTxId1); + assertThat(txn.get("transaction_epoch")).isEqualTo(0L); + assertThat(txn.get("transaction_rank")).isNotNull(); + Vgtid actualVgtid = Vgtid.of(txId); + // The current vgtid is not the previous vgtid. + assertThat(actualVgtid).isNotEqualTo(baseVgtid); + } + assertRecordEnd(expectedTxId1, expectedRecordsCount); + } + @Test @FixFor("DBZ-5063") public void shouldUseSameTransactionIdWhenMultiGrpcResponses() throws Exception { @@ -1326,14 +1389,19 @@ private SourceRecord assertRecordUpdated(SourceRecord updatedRecord) { return updatedRecord; } + private SourceRecord assertRecordBeginSourceRecord() { + assertFalse("records not generated", consumer.isEmpty()); + SourceRecord record = consumer.remove(); + return record; + } + /** * Assert that the connector receives a valid BEGIN event. * * @return The transaction id */ private String assertRecordBegin() { - assertFalse("records not generated", consumer.isEmpty()); - SourceRecord record = consumer.remove(); + SourceRecord record = assertRecordBeginSourceRecord(); final Struct end = (Struct) record.value(); assertThat(end.getString("status")).isEqualTo("BEGIN"); return end.getString("id"); diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index d831159e..27629a47 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -13,6 +13,9 @@ import org.junit.Before; import org.junit.Test; +import io.debezium.pipeline.txmetadata.OrderedTransactionContext; +import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.util.Clock; import io.debezium.util.Collect; public class VitessOffsetContextTest { @@ -95,4 +98,28 @@ public void shouldResetToNewVGgtid() { new Vgtid.ShardGtid(TEST_KEYSPACE, TEST_SHARD, "new_gtid"), new Vgtid.ShardGtid(TEST_KEYSPACE, TEST_SHARD2, "new_gtid2")))); } + + @Test + public void shouldGetOrderedTransactionContext() { + VitessConnectorConfig config = new VitessConnectorConfig( + TestHelper.defaultConfig() + .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .build()); + VitessOffsetContext.Loader loader = new VitessOffsetContext.Loader(config); + Map offsets = Map.of(SourceInfo.VGTID_KEY, VGTID_JSON); + VitessOffsetContext context = loader.load(offsets); + TransactionContext transactionContext = context.getTransactionContext(); + assertThat(transactionContext).isInstanceOf(OrderedTransactionContext.class); + } + + @Test + public void shouldGetInitialOrderedTransactionContext() { + VitessConnectorConfig config = new VitessConnectorConfig( + TestHelper.defaultConfig() + .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .build()); + VitessOffsetContext context = VitessOffsetContext.initialContext(config, Clock.system()); + TransactionContext transactionContext = context.getTransactionContext(); + assertThat(transactionContext).isInstanceOf(OrderedTransactionContext.class); + } } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/GtidTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/GtidTest.java new file mode 100644 index 00000000..082b96ac --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/GtidTest.java @@ -0,0 +1,26 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Set; + +import org.junit.Test; + +public class GtidTest { + + @Test + public void shouldInit() { + String expectedVersion = "MySQL56"; + Gtid gtid = new Gtid(expectedVersion + "/host1:1-4,host2:2-10"); + assertThat(gtid.getVersion()).isEqualTo(expectedVersion); + assertThat(gtid.getSequenceValues()).isEqualTo(List.of("4", "10")); + assertThat(gtid.getHosts()).isEqualTo(Set.of("host1", "host2")); + } + +} diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java new file mode 100644 index 00000000..5a1dc34d --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java @@ -0,0 +1,61 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.assertj.core.api.Assertions; +import org.junit.Test; + +public class VitessEpochProviderTest { + + private String prefix = "MySQL56/"; + private String host1Tx1 = "027c67a2-c0b0-11ec-8a34-0ed0087913a5:1-11418261"; + private String host1Tx2 = "027c67a2-c0b0-11ec-8a34-0ed0087913a5:1-11418262"; + private String host2Tx1 = "08fb1cf3-0ce5-11ed-b921-0a8939501751:1-1443715"; + + private String previousTxId = prefix + String.join(",", host1Tx1, host2Tx1); + private String txId = prefix + String.join(",", host1Tx2, host2Tx1); + private String txIdShrunk = prefix + String.join(",", host1Tx2); + + private String txIdUpgrade = "MySQL82/" + String.join(",", host1Tx2); + + @Test + public void testGetEpochSameHostSet() { + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch(0L, previousTxId, txId); + assertThat(epoch).isEqualTo(0); + } + + @Test + public void testGetEpochShrunkHostSet() { + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch(0L, previousTxId, txIdShrunk); + assertThat(epoch).isEqualTo(1); + } + + @Test + public void testGetEpochExpandHostSet() { + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch(0L, previousTxId, txId); + assertThat(epoch).isEqualTo(0); + } + + @Test + public void testGetEpochDisjointThrowsException() { + VitessEpochProvider provider = new VitessEpochProvider(); + Assertions.assertThatThrownBy(() -> { + provider.getEpoch(0L, previousTxId, "foo:1-2,bar:2-4"); + }).isInstanceOf(RuntimeException.class); + } + + @Test + public void testGetEpochVersionUpgrade() { + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch(0L, previousTxId, txIdUpgrade); + assertThat(epoch).isEqualTo(1); + } +} diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java new file mode 100644 index 00000000..35071965 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java @@ -0,0 +1,24 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigInteger; + +import org.junit.Test; + +public class VitessRankProviderTest { + + @Test + public void shouldGetRankOneHost() { + String txId = "host1:1-4"; + VitessRankProvider provider = new VitessRankProvider(); + BigInteger rank = provider.getRank(txId); + assertThat(rank).isEqualTo(4); + } + +} diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java new file mode 100644 index 00000000..c4e75292 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java @@ -0,0 +1,109 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import io.debezium.connector.vitess.SourceInfo; +import io.debezium.pipeline.txmetadata.OrderedTransactionContext; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.junit.Test; + +import java.math.BigInteger; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.assertj.core.api.Assertions.assertThat; + +public class VitessTransactionOrderMetadataTest { + + private static final Schema sourceStructSchema = SchemaBuilder.struct().field(SourceInfo.VGTID_KEY, Schema.STRING_SCHEMA); + + @Test + public void shouldInit() { + new VitessTransactionOrderMetadata(); + } + + @Test + public void shouldLoad() { + String expectedId = "foo"; + String expectedEpoch = "{\"-80\": 0}"; + Map offsets = Map.of( + OrderedTransactionContext.OFFSET_TRANSACTION_ID, expectedId, + VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, expectedEpoch); + VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + metadata.load(offsets); + assertThat(metadata.previousTransactionId).isEqualTo(expectedId); + } + + @Test + public void shouldLoadWithNull() { + String expectedId = null; + Long expectedEpoch = null; + Map offsets = Collections.emptyMap(); + VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + metadata.load(offsets); + assertThat(metadata.previousTransactionId).isEqualTo(expectedId); + assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); + } + + @Test + public void shouldUpdateEpoch() { + VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + + String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; + BigInteger expectedRank = new BigInteger("7"); + long expectedEpoch = 0; + String expectedShard = "-80"; + + VitessTransactionInfo transactionInfo = new VitessTransactionInfo(expectedTxId, expectedShard); + metadata.beginTransaction(transactionInfo); + assertThat(metadata.transactionRank).isEqualTo(expectedRank); + assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); + + String expectedTxId2 = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3\", \"shard\": \"-80\"}]"; + BigInteger expectedRank2 = new BigInteger("3"); + long expectedEpoch2 = 1; + + VitessTransactionInfo transactionInfo2 = new VitessTransactionInfo(expectedTxId2, expectedShard); + metadata.beginTransaction(transactionInfo2); + assertThat(metadata.transactionRank).isEqualTo(expectedRank2); + assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch2); + } + + @Test + public void shouldUpdateRank() { + VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + + String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; + String expectedShard = "-80"; + + VitessTransactionInfo transactionInfo = new VitessTransactionInfo(expectedTxId, expectedShard); + metadata.beginTransaction(transactionInfo); + assertThat(metadata.transactionRank).isEqualTo(7); + + String expectedTxId2 = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3\", \"shard\": \"-80\"}]"; + VitessTransactionInfo transactionInfo2 = new VitessTransactionInfo(expectedTxId2, expectedShard); + metadata.beginTransaction(transactionInfo2); + assertThat(metadata.transactionRank).isEqualTo(3); + } + + @Test + public void shouldStoreOffsets() { + VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + + String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; + String expectedShard = "-80"; + + VitessTransactionInfo transactionInfo = new VitessTransactionInfo(expectedTxId, expectedShard); + metadata.beginTransaction(transactionInfo); + + Map offsets = new HashMap(); + String expectedEpoch = "{\"-80\":0}"; + Map actualOffsets = metadata.store(offsets); + assertThat(actualOffsets.get(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); + } +} From 7f6d04e98a77cbcecae47732d7ca6399efb743c3 Mon Sep 17 00:00:00 2001 From: twthorn Date: Fri, 29 Mar 2024 13:57:27 -0500 Subject: [PATCH 02/15] DBZ-7698 Refactor to generic transaction block customization --- .../vitess/VitessConnectorConfig.java | 10 ------ .../vitess/VitessEventMetadataProvider.java | 5 +++ .../connector/vitess/VitessOffsetContext.java | 7 ++-- .../connector/vitess/VitessSchemaFactory.java | 6 ++-- .../txmetadata/VitessEpochProvider.java | 4 +-- .../VitessOrderedTransactionStructMaker.java | 17 ++++----- ...ata.java => VitessTransactionContext.java} | 32 ++++++++++++----- .../connector/vitess/VitessConnectorIT.java | 5 ++- .../vitess/VitessOffsetContextTest.java | 10 +++--- ...java => VitessTransactionContextTest.java} | 35 +++++++++---------- 10 files changed, 68 insertions(+), 63 deletions(-) rename src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/{VitessTransactionOrderMetadata.java => VitessTransactionContext.java} (59%) rename src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/{VitessTransactionOrderMetadataTest.java => VitessTransactionContextTest.java} (79%) diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index 786f9de8..32bdd26a 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -32,10 +32,8 @@ import io.debezium.connector.SourceInfoStructMaker; import io.debezium.connector.vitess.connection.VitessTabletType; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionOrderMetadata; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.TemporalPrecisionMode; -import io.debezium.pipeline.txmetadata.TransactionOrderMetadata; import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.relational.ColumnFilterMode; import io.debezium.relational.RelationalDatabaseConnectorConfig; @@ -421,9 +419,6 @@ private static int validateTimePrecisionMode(Configuration config, Field field, public static final Field SOURCE_INFO_STRUCT_MAKER = CommonConnectorConfig.SOURCE_INFO_STRUCT_MAKER .withDefault(VitessSourceInfoStructMaker.class.getName()); - public static final Field TRANSACTION_ORDER_METADATA_FIELD = CommonConnectorConfig.TRANSACTION_ORDER_METADATA_FIELD - .withDefault(VitessTransactionOrderMetadata.class.getName()); - public static final Field TRANSACTION_STRUCT_MAKER = CommonConnectorConfig.TRANSACTION_STRUCT_MAKER .withDefault(VitessOrderedTransactionStructMaker.class.getName()); @@ -524,11 +519,6 @@ protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { return getSourceInfoStructMaker(SOURCE_INFO_STRUCT_MAKER, Module.name(), Module.version(), this); } - @Override - public TransactionOrderMetadata getTransactionOrderMetadata() { - return getTransactionOrderMetadata(TRANSACTION_ORDER_METADATA_FIELD); - } - @Override public TransactionStructMaker getTransactionStructMaker() { return getTransactionStructMaker(TRANSACTION_STRUCT_MAKER); diff --git a/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java b/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java index 91757e40..6f996551 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java +++ b/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java @@ -61,4 +61,9 @@ public String getTransactionId( return sourceInfo.getString(SourceInfo.VGTID_KEY); } + // @Override + // public TransactionInfo getTransactionInfo(DataCollectionId source, OffsetContext offset, Object key, Struct value) { + // + // } + } diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 4797699e..057e4481 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -19,7 +19,6 @@ import io.debezium.pipeline.CommonOffsetContext; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.txmetadata.TransactionContext; -import io.debezium.pipeline.txmetadata.TransactionContextSupplier; import io.debezium.relational.TableId; import io.debezium.spi.schema.DataCollectionId; import io.debezium.util.Clock; @@ -52,8 +51,7 @@ public static VitessOffsetContext initialContext( LOGGER.info("No previous offset exists. Use default VGTID."); final Vgtid defaultVgtid = VitessReplicationConnection.defaultVgtid(connectorConfig); // use the other transaction context - TransactionContextSupplier supplier = new TransactionContextSupplier(connectorConfig); - TransactionContext transactionContext = supplier.newTransactionContext(); + TransactionContext transactionContext = connectorConfig.getTransactionContext(); VitessOffsetContext context = new VitessOffsetContext( connectorConfig, defaultVgtid, clock.currentTimeAsInstant(), transactionContext); return context; @@ -149,8 +147,7 @@ public Loader(VitessConnectorConfig connectorConfig) { @Override public VitessOffsetContext load(Map offset) { final String vgtid = (String) offset.get(SourceInfo.VGTID_KEY); - TransactionContextSupplier supplier = new TransactionContextSupplier(connectorConfig); - TransactionContext transactionContext = supplier.loadTransactionContext(offset); + TransactionContext transactionContext = connectorConfig.getTransactionContext(); return new VitessOffsetContext( connectorConfig, Vgtid.of(vgtid), diff --git a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java index 1e80de2c..c2bceb18 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java +++ b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java @@ -11,7 +11,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionOrderMetadata; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; import io.debezium.data.Envelope; import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.schema.SchemaFactory; @@ -35,8 +35,8 @@ public Schema getOrderedTransactionBlockSchema() { .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_ID_KEY, Schema.STRING_SCHEMA) .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, Schema.INT64_SCHEMA) .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, Schema.INT64_SCHEMA) - .field(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) - .field(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) + .field(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) + .field(VitessTransactionContext.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) .build(); } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index 7b722d27..b7d27103 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -42,7 +42,7 @@ else if (gtid.isHostSetSubsetOf(previousGtid)) { public Map store(Map offset) { try { - offset.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); + offset.put(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); return offset; } catch (JsonProcessingException e) { @@ -52,7 +52,7 @@ public Map store(Map offset) { public void load(Map offsets) { try { - String shardToEpochString = (String) offsets.get(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH); + String shardToEpochString = (String) offsets.get(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH); if (shardToEpochString != null) { shardToEpoch = MAPPER.readValue(shardToEpochString, new TypeReference>() { }); diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java index 67b58e13..c10e7d33 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java @@ -11,28 +11,25 @@ import io.debezium.connector.vitess.VitessSchemaFactory; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.txmetadata.AbstractTransactionStructMaker; -import io.debezium.pipeline.txmetadata.OrderedTransactionContext; import io.debezium.pipeline.txmetadata.TransactionStructMaker; -import io.debezium.spi.schema.DataCollectionId; public class VitessOrderedTransactionStructMaker extends AbstractTransactionStructMaker implements TransactionStructMaker { @Override - public Struct prepareTxStruct(OffsetContext offsetContext, DataCollectionId source) { - Struct struct = super.prepareTxStruct(offsetContext, source); + public Struct prepareTxStruct(OffsetContext offsetContext, long dataCollectionEventOrder, Struct value) { + Struct struct = super.prepareTxStruct(offsetContext, dataCollectionEventOrder, value); return addOrderMetadata(struct, offsetContext); } private Struct addOrderMetadata(Struct struct, OffsetContext offsetContext) { - VitessTransactionOrderMetadata metadata = getVitessTransactionOrderMetadata(offsetContext); - struct.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_RANK, metadata.transactionRank.toString()); - struct.put(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, metadata.transactionEpoch); + VitessTransactionContext context = getVitessTransactionOrderMetadata(offsetContext); + struct.put(VitessTransactionContext.OFFSET_TRANSACTION_RANK, context.transactionRank.toString()); + struct.put(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, context.transactionEpoch); return struct; } - private VitessTransactionOrderMetadata getVitessTransactionOrderMetadata(OffsetContext offsetContext) { - OrderedTransactionContext context = (OrderedTransactionContext) offsetContext.getTransactionContext(); - return (VitessTransactionOrderMetadata) context.getTransactionOrderMetadata(); + private VitessTransactionContext getVitessTransactionOrderMetadata(OffsetContext offsetContext) { + return (VitessTransactionContext) offsetContext.getTransactionContext(); } @Override diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java similarity index 59% rename from src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java rename to src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java index b3d84aa8..2cced919 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadata.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java @@ -11,37 +11,51 @@ import io.debezium.connector.vitess.Vgtid; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.pipeline.txmetadata.TransactionInfo; -import io.debezium.pipeline.txmetadata.TransactionOrderMetadata; - -public class VitessTransactionOrderMetadata implements TransactionOrderMetadata { +public class VitessTransactionContext extends TransactionContext { public static final String OFFSET_TRANSACTION_EPOCH = "transaction_epoch"; public static final String OFFSET_TRANSACTION_RANK = "transaction_rank"; protected String previousTransactionId = null; - protected Long transactionEpoch; - protected BigInteger transactionRank; + protected Long transactionEpoch = 0L; + protected BigInteger transactionRank = null; private VitessEpochProvider epochProvider = new VitessEpochProvider(); private VitessRankProvider rankProvider = new VitessRankProvider(); + public VitessTransactionContext() { + } + + public VitessTransactionContext(TransactionContext transactionContext) { + super(); + // Copy fields + this.transactionId = transactionContext.transactionId; + this.perTableEventCount.putAll(transactionContext.perTableEventCount); + this.totalEventCount = transactionContext.totalEventCount; + } + @Override public Map store(Map offset) { + offset = super.store(offset); return epochProvider.store(offset); } - @Override - public void load(Map offsets) { - this.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); - epochProvider.load(offsets); + public static VitessTransactionContext load(Map offsets) { + TransactionContext transactionContext = TransactionContext.load(offsets); + VitessTransactionContext vitessTransactionContext = new VitessTransactionContext(transactionContext); + vitessTransactionContext.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); + vitessTransactionContext.epochProvider.load(offsets); + return vitessTransactionContext; } @Override public void beginTransaction(TransactionInfo transactionInfo) { + super.beginTransaction(transactionInfo); VitessTransactionInfo vitessTransactionInfo = (VitessTransactionInfo) transactionInfo; beginTransaction(vitessTransactionInfo.getShard(), vitessTransactionInfo.getTransactionId()); } @Override public void endTransaction() { + super.endTransaction(); this.transactionEpoch = null; this.transactionRank = null; } diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 4de99a8b..be619fde 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -46,6 +46,8 @@ import io.debezium.config.Configuration; import io.debezium.config.Field; import io.debezium.connector.vitess.connection.VitessReplicationConnection; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; import io.debezium.converters.CloudEventsConverterTest; import io.debezium.converters.spi.CloudEventsMaker; import io.debezium.data.Envelope; @@ -475,7 +477,8 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); TestHelper.applyVSchema("vitess_vschema.json"); startConnector(config -> config - .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) + .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) .with(VitessConnectorConfig.SHARD, "-80,80-"), true, diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 27629a47..6adeb531 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -13,7 +13,7 @@ import org.junit.Before; import org.junit.Test; -import io.debezium.pipeline.txmetadata.OrderedTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.util.Clock; import io.debezium.util.Collect; @@ -103,23 +103,23 @@ public void shouldResetToNewVGgtid() { public void shouldGetOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) .build()); VitessOffsetContext.Loader loader = new VitessOffsetContext.Loader(config); Map offsets = Map.of(SourceInfo.VGTID_KEY, VGTID_JSON); VitessOffsetContext context = loader.load(offsets); TransactionContext transactionContext = context.getTransactionContext(); - assertThat(transactionContext).isInstanceOf(OrderedTransactionContext.class); + assertThat(transactionContext).isInstanceOf(VitessTransactionContext.class); } @Test public void shouldGetInitialOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.PROVIDE_ORDERED_TRANSACTION_METADATA, true) + .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) .build()); VitessOffsetContext context = VitessOffsetContext.initialContext(config, Clock.system()); TransactionContext transactionContext = context.getTransactionContext(); - assertThat(transactionContext).isInstanceOf(OrderedTransactionContext.class); + assertThat(transactionContext).isInstanceOf(VitessTransactionContext.class); } } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java similarity index 79% rename from src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java rename to src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java index c4e75292..69977ad5 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionOrderMetadataTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java @@ -5,36 +5,35 @@ */ package io.debezium.connector.vitess.pipeline.txmetadata; -import io.debezium.connector.vitess.SourceInfo; -import io.debezium.pipeline.txmetadata.OrderedTransactionContext; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigInteger; import java.util.Collections; import java.util.HashMap; import java.util.Map; -import static org.assertj.core.api.Assertions.assertThat; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.junit.Test; -public class VitessTransactionOrderMetadataTest { +import io.debezium.connector.vitess.SourceInfo; + +public class VitessTransactionContextTest { private static final Schema sourceStructSchema = SchemaBuilder.struct().field(SourceInfo.VGTID_KEY, Schema.STRING_SCHEMA); @Test public void shouldInit() { - new VitessTransactionOrderMetadata(); + new VitessTransactionContext(); } @Test public void shouldLoad() { - String expectedId = "foo"; + String expectedId = null; String expectedEpoch = "{\"-80\": 0}"; Map offsets = Map.of( - OrderedTransactionContext.OFFSET_TRANSACTION_ID, expectedId, - VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH, expectedEpoch); - VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch); + VitessTransactionContext metadata = new VitessTransactionContext(); metadata.load(offsets); assertThat(metadata.previousTransactionId).isEqualTo(expectedId); } @@ -42,9 +41,9 @@ public void shouldLoad() { @Test public void shouldLoadWithNull() { String expectedId = null; - Long expectedEpoch = null; + Long expectedEpoch = 0L; Map offsets = Collections.emptyMap(); - VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + VitessTransactionContext metadata = new VitessTransactionContext(); metadata.load(offsets); assertThat(metadata.previousTransactionId).isEqualTo(expectedId); assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); @@ -52,7 +51,7 @@ public void shouldLoadWithNull() { @Test public void shouldUpdateEpoch() { - VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + VitessTransactionContext metadata = new VitessTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; BigInteger expectedRank = new BigInteger("7"); @@ -76,7 +75,7 @@ public void shouldUpdateEpoch() { @Test public void shouldUpdateRank() { - VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + VitessTransactionContext metadata = new VitessTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; String expectedShard = "-80"; @@ -93,7 +92,7 @@ public void shouldUpdateRank() { @Test public void shouldStoreOffsets() { - VitessTransactionOrderMetadata metadata = new VitessTransactionOrderMetadata(); + VitessTransactionContext metadata = new VitessTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; String expectedShard = "-80"; @@ -104,6 +103,6 @@ public void shouldStoreOffsets() { Map offsets = new HashMap(); String expectedEpoch = "{\"-80\":0}"; Map actualOffsets = metadata.store(offsets); - assertThat(actualOffsets.get(VitessTransactionOrderMetadata.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); + assertThat(actualOffsets.get(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); } } From 68eaf6c7a3f78ebf00496c8665fbf82c5b5bae40 Mon Sep 17 00:00:00 2001 From: twthorn Date: Fri, 29 Mar 2024 15:59:37 -0500 Subject: [PATCH 03/15] DBZ-7698 Improve test naming --- .../connector/vitess/VitessOffsetContextTest.java | 4 ++-- .../pipeline/txmetadata/VitessEpochProviderTest.java | 9 +++++---- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 6adeb531..8b390da1 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -100,7 +100,7 @@ public void shouldResetToNewVGgtid() { } @Test - public void shouldGetOrderedTransactionContext() { + public void shouldGetVitessTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) @@ -113,7 +113,7 @@ public void shouldGetOrderedTransactionContext() { } @Test - public void shouldGetInitialOrderedTransactionContext() { + public void shouldGetInitialVitessTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java index 5a1dc34d..785698c4 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java @@ -21,7 +21,8 @@ public class VitessEpochProviderTest { private String txId = prefix + String.join(",", host1Tx2, host2Tx1); private String txIdShrunk = prefix + String.join(",", host1Tx2); - private String txIdUpgrade = "MySQL82/" + String.join(",", host1Tx2); + private String txIdVersion5 = "MySQL57/" + String.join(",", host1Tx2); + private String txIdVersion8 = "MySQL82/" + String.join(",", host1Tx2); @Test public void testGetEpochSameHostSet() { @@ -53,9 +54,9 @@ public void testGetEpochDisjointThrowsException() { } @Test - public void testGetEpochVersionUpgrade() { + public void testVersionUpgradeDoesNotAffectEpoch() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch(0L, previousTxId, txIdUpgrade); - assertThat(epoch).isEqualTo(1); + Long epoch = provider.getEpoch(0L, txIdVersion5, txIdVersion8); + assertThat(epoch).isEqualTo(0L); } } From 9d5ebf33e9b6667098a5b1631315848b623f4764 Mon Sep 17 00:00:00 2001 From: twthorn Date: Fri, 29 Mar 2024 19:07:48 -0500 Subject: [PATCH 04/15] DBZ-7698 Fix naming, implement getTransactionInfo --- .../vitess/VitessEventMetadataProvider.java | 16 ++++++++++++---- .../connector/vitess/VitessSchemaFactory.java | 6 +++--- .../txmetadata/VitessEpochProvider.java | 4 ++-- ...va => VitessOrderedTransactionContext.java} | 16 ++++++++-------- .../VitessOrderedTransactionStructMaker.java | 10 +++++----- .../connector/vitess/VitessConnectorIT.java | 4 ++-- .../vitess/VitessOffsetContextTest.java | 14 +++++++------- ...> VitessOrderedTransactionContextTest.java} | 18 +++++++++--------- 8 files changed, 48 insertions(+), 40 deletions(-) rename src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/{VitessTransactionContext.java => VitessOrderedTransactionContext.java} (76%) rename src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/{VitessTransactionContextTest.java => VitessOrderedTransactionContextTest.java} (82%) diff --git a/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java b/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java index 6f996551..ee7b96b4 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java +++ b/src/main/java/io/debezium/connector/vitess/VitessEventMetadataProvider.java @@ -10,9 +10,11 @@ import org.apache.kafka.connect.data.Struct; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionInfo; import io.debezium.data.Envelope; import io.debezium.pipeline.source.spi.EventMetadataProvider; import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.txmetadata.TransactionInfo; import io.debezium.spi.schema.DataCollectionId; import io.debezium.util.Collect; @@ -61,9 +63,15 @@ public String getTransactionId( return sourceInfo.getString(SourceInfo.VGTID_KEY); } - // @Override - // public TransactionInfo getTransactionInfo(DataCollectionId source, OffsetContext offset, Object key, Struct value) { - // - // } + @Override + public TransactionInfo getTransactionInfo(DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null || source == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + String vgtid = sourceInfo.getString(SourceInfo.VGTID_KEY); + String shard = sourceInfo.getString(SourceInfo.SHARD_KEY); + return new VitessTransactionInfo(vgtid, shard); + } } diff --git a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java index c2bceb18..92b9f6c6 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java +++ b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java @@ -11,7 +11,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; import io.debezium.data.Envelope; import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.schema.SchemaFactory; @@ -35,8 +35,8 @@ public Schema getOrderedTransactionBlockSchema() { .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_ID_KEY, Schema.STRING_SCHEMA) .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, Schema.INT64_SCHEMA) .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, Schema.INT64_SCHEMA) - .field(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) - .field(VitessTransactionContext.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) + .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) + .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) .build(); } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index b7d27103..ac2e315f 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -42,7 +42,7 @@ else if (gtid.isHostSetSubsetOf(previousGtid)) { public Map store(Map offset) { try { - offset.put(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); + offset.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); return offset; } catch (JsonProcessingException e) { @@ -52,7 +52,7 @@ public Map store(Map offset) { public void load(Map offsets) { try { - String shardToEpochString = (String) offsets.get(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH); + String shardToEpochString = (String) offsets.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); if (shardToEpochString != null) { shardToEpoch = MAPPER.readValue(shardToEpochString, new TypeReference>() { }); diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java similarity index 76% rename from src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java rename to src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index 2cced919..f7a7abcc 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -12,7 +12,7 @@ import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.pipeline.txmetadata.TransactionInfo; -public class VitessTransactionContext extends TransactionContext { +public class VitessOrderedTransactionContext extends TransactionContext { public static final String OFFSET_TRANSACTION_EPOCH = "transaction_epoch"; public static final String OFFSET_TRANSACTION_RANK = "transaction_rank"; protected String previousTransactionId = null; @@ -21,10 +21,10 @@ public class VitessTransactionContext extends TransactionContext { private VitessEpochProvider epochProvider = new VitessEpochProvider(); private VitessRankProvider rankProvider = new VitessRankProvider(); - public VitessTransactionContext() { + public VitessOrderedTransactionContext() { } - public VitessTransactionContext(TransactionContext transactionContext) { + public VitessOrderedTransactionContext(TransactionContext transactionContext) { super(); // Copy fields this.transactionId = transactionContext.transactionId; @@ -38,12 +38,12 @@ public Map store(Map offset) { return epochProvider.store(offset); } - public static VitessTransactionContext load(Map offsets) { + public static VitessOrderedTransactionContext load(Map offsets) { TransactionContext transactionContext = TransactionContext.load(offsets); - VitessTransactionContext vitessTransactionContext = new VitessTransactionContext(transactionContext); - vitessTransactionContext.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); - vitessTransactionContext.epochProvider.load(offsets); - return vitessTransactionContext; + VitessOrderedTransactionContext vitessOrderedTransactionContext = new VitessOrderedTransactionContext(transactionContext); + vitessOrderedTransactionContext.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); + vitessOrderedTransactionContext.epochProvider.load(offsets); + return vitessOrderedTransactionContext; } @Override diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java index c10e7d33..eea58fa1 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java @@ -22,14 +22,14 @@ public Struct prepareTxStruct(OffsetContext offsetContext, long dataCollectionEv } private Struct addOrderMetadata(Struct struct, OffsetContext offsetContext) { - VitessTransactionContext context = getVitessTransactionOrderMetadata(offsetContext); - struct.put(VitessTransactionContext.OFFSET_TRANSACTION_RANK, context.transactionRank.toString()); - struct.put(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, context.transactionEpoch); + VitessOrderedTransactionContext context = getVitessTransactionOrderMetadata(offsetContext); + struct.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, context.transactionRank.toString()); + struct.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, context.transactionEpoch); return struct; } - private VitessTransactionContext getVitessTransactionOrderMetadata(OffsetContext offsetContext) { - return (VitessTransactionContext) offsetContext.getTransactionContext(); + private VitessOrderedTransactionContext getVitessTransactionOrderMetadata(OffsetContext offsetContext) { + return (VitessOrderedTransactionContext) offsetContext.getTransactionContext(); } @Override diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index be619fde..139cb87b 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -46,8 +46,8 @@ import io.debezium.config.Configuration; import io.debezium.config.Field; import io.debezium.connector.vitess.connection.VitessReplicationConnection; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; import io.debezium.converters.CloudEventsConverterTest; import io.debezium.converters.spi.CloudEventsMaker; import io.debezium.data.Envelope; @@ -477,7 +477,7 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); TestHelper.applyVSchema("vitess_vschema.json"); startConnector(config -> config - .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) + .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) .with(VitessConnectorConfig.SHARD, "-80,80-"), diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 8b390da1..ca4c9033 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -13,7 +13,7 @@ import org.junit.Before; import org.junit.Test; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.util.Clock; import io.debezium.util.Collect; @@ -100,26 +100,26 @@ public void shouldResetToNewVGgtid() { } @Test - public void shouldGetVitessTransactionContext() { + public void shouldGetVitessOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) + .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) .build()); VitessOffsetContext.Loader loader = new VitessOffsetContext.Loader(config); Map offsets = Map.of(SourceInfo.VGTID_KEY, VGTID_JSON); VitessOffsetContext context = loader.load(offsets); TransactionContext transactionContext = context.getTransactionContext(); - assertThat(transactionContext).isInstanceOf(VitessTransactionContext.class); + assertThat(transactionContext).isInstanceOf(VitessOrderedTransactionContext.class); } @Test - public void shouldGetInitialVitessTransactionContext() { + public void shouldGetInitialVitessOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessTransactionContext.class) + .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) .build()); VitessOffsetContext context = VitessOffsetContext.initialContext(config, Clock.system()); TransactionContext transactionContext = context.getTransactionContext(); - assertThat(transactionContext).isInstanceOf(VitessTransactionContext.class); + assertThat(transactionContext).isInstanceOf(VitessOrderedTransactionContext.class); } } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java similarity index 82% rename from src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java rename to src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java index 69977ad5..9685fa69 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java @@ -18,13 +18,13 @@ import io.debezium.connector.vitess.SourceInfo; -public class VitessTransactionContextTest { +public class VitessOrderedTransactionContextTest { private static final Schema sourceStructSchema = SchemaBuilder.struct().field(SourceInfo.VGTID_KEY, Schema.STRING_SCHEMA); @Test public void shouldInit() { - new VitessTransactionContext(); + new VitessOrderedTransactionContext(); } @Test @@ -32,8 +32,8 @@ public void shouldLoad() { String expectedId = null; String expectedEpoch = "{\"-80\": 0}"; Map offsets = Map.of( - VitessTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch); - VitessTransactionContext metadata = new VitessTransactionContext(); + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch); + VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); metadata.load(offsets); assertThat(metadata.previousTransactionId).isEqualTo(expectedId); } @@ -43,7 +43,7 @@ public void shouldLoadWithNull() { String expectedId = null; Long expectedEpoch = 0L; Map offsets = Collections.emptyMap(); - VitessTransactionContext metadata = new VitessTransactionContext(); + VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); metadata.load(offsets); assertThat(metadata.previousTransactionId).isEqualTo(expectedId); assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); @@ -51,7 +51,7 @@ public void shouldLoadWithNull() { @Test public void shouldUpdateEpoch() { - VitessTransactionContext metadata = new VitessTransactionContext(); + VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; BigInteger expectedRank = new BigInteger("7"); @@ -75,7 +75,7 @@ public void shouldUpdateEpoch() { @Test public void shouldUpdateRank() { - VitessTransactionContext metadata = new VitessTransactionContext(); + VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; String expectedShard = "-80"; @@ -92,7 +92,7 @@ public void shouldUpdateRank() { @Test public void shouldStoreOffsets() { - VitessTransactionContext metadata = new VitessTransactionContext(); + VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; String expectedShard = "-80"; @@ -103,6 +103,6 @@ public void shouldStoreOffsets() { Map offsets = new HashMap(); String expectedEpoch = "{\"-80\":0}"; Map actualOffsets = metadata.store(offsets); - assertThat(actualOffsets.get(VitessTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); + assertThat(actualOffsets.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); } } From d9f7ab28751774bf8e5d669e78a810d081659ece Mon Sep 17 00:00:00 2001 From: twthorn Date: Mon, 1 Apr 2024 14:31:09 -0500 Subject: [PATCH 05/15] DBZ-7698 fix itests by Removing default for TRANSACTION_STRUCT_MAKER --- .../io/debezium/connector/vitess/VitessConnectorConfig.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index 32bdd26a..61fd8c86 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -419,9 +419,6 @@ private static int validateTimePrecisionMode(Configuration config, Field field, public static final Field SOURCE_INFO_STRUCT_MAKER = CommonConnectorConfig.SOURCE_INFO_STRUCT_MAKER .withDefault(VitessSourceInfoStructMaker.class.getName()); - public static final Field TRANSACTION_STRUCT_MAKER = CommonConnectorConfig.TRANSACTION_STRUCT_MAKER - .withDefault(VitessOrderedTransactionStructMaker.class.getName()); - protected static final ConfigDefinition CONFIG_DEFINITION = RelationalDatabaseConnectorConfig.CONFIG_DEFINITION .edit() .name("Vitess") From 5f6e9e2d2efbb6250d93a36815776a233a1781d6 Mon Sep 17 00:00:00 2001 From: twthorn Date: Mon, 1 Apr 2024 16:56:45 -0500 Subject: [PATCH 06/15] DBZ-7698 Remove import --- .../java/io/debezium/connector/vitess/VitessConnectorConfig.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index 61fd8c86..c99703a3 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -31,7 +31,6 @@ import io.debezium.config.Field.ValidationOutput; import io.debezium.connector.SourceInfoStructMaker; import io.debezium.connector.vitess.connection.VitessTabletType; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.TemporalPrecisionMode; import io.debezium.pipeline.txmetadata.TransactionStructMaker; From 2416e72562106a716ac8e4b4c2f6f949d3c85a57 Mon Sep 17 00:00:00 2001 From: twthorn Date: Tue, 2 Apr 2024 12:52:00 -0500 Subject: [PATCH 07/15] DBZ-7698 Add more unit tests --- .../vitess/VitessConnectorConfig.java | 6 -- .../connector/vitess/VitessSchemaFactory.java | 58 -------------- .../debezium/connector/vitess/VgtidTest.java | 7 ++ .../connector/vitess/VitessConnectorIT.java | 1 - .../VitessEventMetadataProviderTest.java | 78 +++++++++++++++++++ .../vitess/VitessOffsetContextTest.java | 2 +- .../vitess/VitessSchemaFactoryTest.java | 40 ++++++++++ .../VStreamOutputMessageDecoderTest.java | 6 ++ ...tessOrderedTransactionStructMakerTest.java | 43 ++++++++++ .../txmetadata/VitessTransactionInfoTest.java | 25 ++++++ 10 files changed, 200 insertions(+), 66 deletions(-) create mode 100644 src/test/java/io/debezium/connector/vitess/VitessEventMetadataProviderTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java create mode 100644 src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfoTest.java diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index c99703a3..b5628eb4 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -33,7 +33,6 @@ import io.debezium.connector.vitess.connection.VitessTabletType; import io.debezium.jdbc.JdbcConfiguration; import io.debezium.jdbc.TemporalPrecisionMode; -import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.relational.ColumnFilterMode; import io.debezium.relational.RelationalDatabaseConnectorConfig; @@ -515,11 +514,6 @@ protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { return getSourceInfoStructMaker(SOURCE_INFO_STRUCT_MAKER, Module.name(), Module.version(), this); } - @Override - public TransactionStructMaker getTransactionStructMaker() { - return getTransactionStructMaker(TRANSACTION_STRUCT_MAKER); - } - public String getKeyspace() { return getConfig().getString(KEYSPACE); } diff --git a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java index 92b9f6c6..cf88ba2e 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java +++ b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java @@ -5,14 +5,10 @@ */ package io.debezium.connector.vitess; -import java.util.HashSet; -import java.util.Set; - import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; -import io.debezium.data.Envelope; import io.debezium.pipeline.txmetadata.TransactionStructMaker; import io.debezium.schema.SchemaFactory; @@ -39,58 +35,4 @@ public Schema getOrderedTransactionBlockSchema() { .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) .build(); } - - @Override - public Envelope.Builder datatypeEnvelopeSchema() { - return new Envelope.Builder() { - private final SchemaBuilder builder = SchemaBuilder.struct() - .version(Envelope.SCHEMA_VERSION); - - private final Set missingFields = new HashSet<>(); - - @Override - public Envelope.Builder withSchema(Schema fieldSchema, String... fieldNames) { - for (String fieldName : fieldNames) { - builder.field(fieldName, fieldSchema); - } - return this; - } - - @Override - public Envelope.Builder withName(String name) { - builder.name(name); - return this; - } - - @Override - public Envelope.Builder withDoc(String doc) { - builder.doc(doc); - return this; - } - - @Override - public Envelope build() { - builder.field(Envelope.FieldName.OPERATION, Envelope.OPERATION_REQUIRED ? Schema.STRING_SCHEMA : Schema.OPTIONAL_STRING_SCHEMA); - builder.field(Envelope.FieldName.TIMESTAMP, Schema.OPTIONAL_INT64_SCHEMA); - builder.field(Envelope.FieldName.TIMESTAMP_US, Schema.OPTIONAL_INT64_SCHEMA); - builder.field(Envelope.FieldName.TIMESTAMP_NS, Schema.OPTIONAL_INT64_SCHEMA); - builder.field(Envelope.FieldName.TRANSACTION, getOrderedTransactionBlockSchema()); - checkFieldIsDefined(Envelope.FieldName.OPERATION); - checkFieldIsDefined(Envelope.FieldName.BEFORE); - checkFieldIsDefined(Envelope.FieldName.AFTER); - checkFieldIsDefined(Envelope.FieldName.SOURCE); - checkFieldIsDefined(Envelope.FieldName.TRANSACTION); - if (!missingFields.isEmpty()) { - throw new IllegalStateException("The envelope schema is missing field(s) " + String.join(", ", missingFields)); - } - return new Envelope(builder.build()); - } - - private void checkFieldIsDefined(String fieldName) { - if (builder.field(fieldName) == null) { - missingFields.add(fieldName); - } - } - }; - } } diff --git a/src/test/java/io/debezium/connector/vitess/VgtidTest.java b/src/test/java/io/debezium/connector/vitess/VgtidTest.java index 702784e9..13eb3a0f 100644 --- a/src/test/java/io/debezium/connector/vitess/VgtidTest.java +++ b/src/test/java/io/debezium/connector/vitess/VgtidTest.java @@ -303,4 +303,11 @@ public void shouldEqualsIfEqualityHolds() { assertThat(vgtids.stream().allMatch(vgtid::equals)).isTrue(); } } + + @Test + public void shouldGetShardGtid() { + Vgtid vgtid1 = Vgtid.of(VGTID_JSON); + Vgtid.ShardGtid shardGtid = vgtid1.getShardGtid(TEST_SHARD); + assertThat(shardGtid).isEqualTo(new Vgtid.ShardGtid(TEST_KEYSPACE, TEST_SHARD, TEST_GTID)); + } } diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 139cb87b..85c5b514 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -472,7 +472,6 @@ public void shouldUsePrevVgtidAsOffsetWhenNoVgtidInGrpcResponse() throws Excepti } @Test - @FixFor("") public void shouldProvideOrderedTransactionMetadata() throws Exception { TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); TestHelper.applyVSchema("vitess_vschema.json"); diff --git a/src/test/java/io/debezium/connector/vitess/VitessEventMetadataProviderTest.java b/src/test/java/io/debezium/connector/vitess/VitessEventMetadataProviderTest.java new file mode 100644 index 00000000..3b136b25 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/VitessEventMetadataProviderTest.java @@ -0,0 +1,78 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess; + +import static io.debezium.connector.vitess.TestHelper.TEST_GTID; +import static io.debezium.connector.vitess.VgtidTest.TEST_GTID2; +import static io.debezium.connector.vitess.VgtidTest.TEST_KEYSPACE; +import static io.debezium.connector.vitess.VgtidTest.TEST_SHARD2; +import static io.debezium.connector.vitess.VgtidTest.VGTID_JSON; +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Instant; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.junit.Before; +import org.junit.Test; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.config.Configuration; +import io.debezium.connector.SnapshotRecord; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionInfo; +import io.debezium.relational.TableId; +import io.debezium.util.Clock; +import io.debezium.util.Collect; + +public class VitessEventMetadataProviderTest { + + private SourceInfo source; + + @Before + public void beforeEach() { + final VitessConnectorConfig connectorConfig = new VitessConnectorConfig( + Configuration.create() + .with(CommonConnectorConfig.TOPIC_PREFIX, "server_foo") + .with(VitessConnectorConfig.KEYSPACE, TEST_KEYSPACE) + .with(VitessConnectorConfig.SHARD, AnonymousValue.getString()) + .with(VitessConnectorConfig.VTGATE_HOST, AnonymousValue.getString()) + .with(VitessConnectorConfig.VTGATE_PORT, AnonymousValue.getInt()) + .build()); + source = new SourceInfo(connectorConfig); + source.resetVgtid( + Vgtid.of( + Collect.arrayListOf( + new Vgtid.ShardGtid(TEST_KEYSPACE, VgtidTest.TEST_SHARD, TEST_GTID), + new Vgtid.ShardGtid(TEST_KEYSPACE, TEST_SHARD2, TEST_GTID2))), + Instant.ofEpochMilli(1000)); + source.setTableId(new TableId("c", "s", "t")); + source.setShard(VgtidTest.TEST_SHARD); + source.setSnapshot(SnapshotRecord.FALSE); + } + + @Test + public void getTransactionInfo() { + VitessSourceInfoStructMaker maker = new VitessSourceInfoStructMaker(); + maker.init("foo", "bar", new VitessConnectorConfig(TestHelper.defaultConfig().build())); + Schema sourceSchema = maker.schema(); + Schema schema = VitessSchemaFactory.get().datatypeEnvelopeSchema() + .withSchema(Schema.STRING_SCHEMA, "before") + .withSchema(Schema.STRING_SCHEMA, "after") + .withSchema(sourceSchema, "source") + .withTransaction(VitessSchemaFactory.get().getOrderedTransactionBlockSchema()) + .build().schema(); + VitessEventMetadataProvider provider = new VitessEventMetadataProvider(); + VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig().build()); + Struct valueStruct = new Struct(schema).put("source", source.struct()); + VitessTransactionInfo info = (VitessTransactionInfo) provider.getTransactionInfo( + new TableId("foo", "bar", "foo"), + VitessOffsetContext.initialContext(config, Clock.system()), + null, + valueStruct); + assertThat(info.getShard()).isEqualTo(VgtidTest.TEST_SHARD); + assertThat(info.getTransactionId()).isEqualTo(VGTID_JSON); + } +} diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index ca4c9033..2dbdcf3b 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -100,7 +100,7 @@ public void shouldResetToNewVGgtid() { } @Test - public void shouldGetVitessOrderedTransactionContext() { + public void shouldLoadVitessOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) diff --git a/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java b/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java new file mode 100644 index 00000000..416f08d7 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java @@ -0,0 +1,40 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.junit.Test; + +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; +import io.debezium.pipeline.txmetadata.TransactionStructMaker; + +public class VitessSchemaFactoryTest { + + @Test + public void get() { + assertThat(VitessSchemaFactory.get()).isNotNull(); + } + + @Test + public void getOrderedTransactionBlockSchema() { + Schema orderedTransactionBlockSchema = VitessSchemaFactory.get().getOrderedTransactionBlockSchema(); + List fields = orderedTransactionBlockSchema.fields(); + assertThat(fields).contains(new Field(TransactionStructMaker.DEBEZIUM_TRANSACTION_ID_KEY, 0, Schema.STRING_SCHEMA)); + assertThat(fields).contains(new Field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, 1, Schema.INT64_SCHEMA)); + assertThat(fields).contains(new Field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, 2, Schema.INT64_SCHEMA)); + assertThat(fields).contains(new Field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, 3, Schema.INT64_SCHEMA)); + assertThat(fields).contains(new Field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, 4, Schema.STRING_SCHEMA)); + } + + @Test + public void datatypeEnvelopeSchema() { + } +} diff --git a/src/test/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoderTest.java b/src/test/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoderTest.java index da905db5..fee609c5 100644 --- a/src/test/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoderTest.java +++ b/src/test/java/io/debezium/connector/vitess/connection/VStreamOutputMessageDecoderTest.java @@ -51,8 +51,10 @@ public void before() { @Test public void shouldProcessBeginEvent() throws Exception { // setup fixture + String expectedShard = "shard"; Binlogdata.VEvent event = Binlogdata.VEvent.newBuilder() .setType(Binlogdata.VEventType.BEGIN) + .setShard(expectedShard) .setTimestamp(AnonymousValue.getLong()) .build(); Vgtid newVgtid = Vgtid.of(VgtidTest.VGTID_JSON); @@ -66,6 +68,7 @@ public void shouldProcessBeginEvent() throws Exception { assertThat(message).isNotNull(); assertThat(message).isInstanceOf(TransactionalMessage.class); assertThat(message.getOperation()).isEqualTo(ReplicationMessage.Operation.BEGIN); + assertThat(message.getShard()).isEqualTo(expectedShard); assertThat(message.getTransactionId()).isEqualTo(newVgtid.toString()); assertThat(vgtid).isEqualTo(newVgtid); processed[0] = true; @@ -102,10 +105,12 @@ public void shouldNotProcessBeginEventIfNoVgtid() throws Exception { @Test public void shouldProcessCommitEvent() throws Exception { + String expectedShard = "shard"; // setup fixture Binlogdata.VEvent event = Binlogdata.VEvent.newBuilder() .setType(Binlogdata.VEventType.COMMIT) .setTimestamp(AnonymousValue.getLong()) + .setShard(expectedShard) .build(); Vgtid newVgtid = Vgtid.of(VgtidTest.VGTID_JSON); decoder.setTransactionId(newVgtid.toString()); @@ -119,6 +124,7 @@ public void shouldProcessCommitEvent() throws Exception { assertThat(message).isNotNull(); assertThat(message).isInstanceOf(TransactionalMessage.class); assertThat(message.getOperation()).isEqualTo(ReplicationMessage.Operation.COMMIT); + assertThat(message.getShard()).isEqualTo(expectedShard); assertThat(message.getTransactionId()).isEqualTo(newVgtid.toString()); assertThat(vgtid).isEqualTo(newVgtid); processed[0] = true; diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java new file mode 100644 index 00000000..89a32d90 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java @@ -0,0 +1,43 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.time.Instant; + +import org.apache.kafka.connect.data.Struct; +import org.junit.Test; + +import io.debezium.connector.vitess.TestHelper; +import io.debezium.connector.vitess.Vgtid; +import io.debezium.connector.vitess.VgtidTest; +import io.debezium.connector.vitess.VitessConnectorConfig; +import io.debezium.connector.vitess.VitessOffsetContext; +import io.debezium.connector.vitess.VitessSchemaFactory; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.pipeline.txmetadata.TransactionContext; + +public class VitessOrderedTransactionStructMakerTest { + + @Test + public void prepareTxStruct() { + VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig().build()); + VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(); + TransactionContext transactionContext = new VitessOrderedTransactionContext(); + transactionContext.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, VgtidTest.TEST_SHARD)); + OffsetContext context = new VitessOffsetContext(config, Vgtid.of(VgtidTest.VGTID_JSON), Instant.now(), transactionContext); + Struct struct = maker.prepareTxStruct(context, 0, null); + assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(0L); + assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK)).isEqualTo("1513"); + } + + @Test + public void getTransactionBlockSchema() { + VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(); + assertThat(maker.getTransactionBlockSchema()).isEqualTo(VitessSchemaFactory.get().getOrderedTransactionBlockSchema()); + } +} diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfoTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfoTest.java new file mode 100644 index 00000000..2574cca5 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessTransactionInfoTest.java @@ -0,0 +1,25 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.vitess.pipeline.txmetadata; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Test; + +public class VitessTransactionInfoTest { + + @Test + public void getTransactionId() { + String expectedId = "vgtid"; + assertThat(new VitessTransactionInfo(expectedId, "shard").getTransactionId()).isEqualTo(expectedId); + } + + @Test + public void getShard() { + String expectedShard = "shard"; + assertThat(new VitessTransactionInfo("vgtid", expectedShard).getShard()).isEqualTo(expectedShard); + } +} From 967be822317b2265a761368d85daf87d342dad6f Mon Sep 17 00:00:00 2001 From: twthorn Date: Tue, 2 Apr 2024 16:45:13 -0500 Subject: [PATCH 08/15] DBZ-7698 Switch to Decimal type, add examples --- .../connector/vitess/VitessSchemaFactory.java | 4 ++- .../VitessOrderedTransactionContext.java | 32 +++++++++++++++++-- .../VitessOrderedTransactionStructMaker.java | 17 +++++++++- .../txmetadata/VitessRankProvider.java | 8 ++--- .../connector/vitess/VitessConnectorIT.java | 15 ++++++--- .../vitess/VitessSchemaFactoryTest.java | 3 +- .../VitessOrderedTransactionContextTest.java | 26 ++++++++------- ...tessOrderedTransactionStructMakerTest.java | 3 +- .../txmetadata/VitessRankProviderTest.java | 6 ++-- 9 files changed, 85 insertions(+), 29 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java index cf88ba2e..5e6bb634 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java +++ b/src/main/java/io/debezium/connector/vitess/VitessSchemaFactory.java @@ -5,6 +5,7 @@ */ package io.debezium.connector.vitess; +import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; @@ -25,6 +26,7 @@ public static VitessSchemaFactory get() { } public Schema getOrderedTransactionBlockSchema() { + Schema rankSchema = Decimal.schema(0).schema(); return SchemaBuilder.struct().optional() .name(TRANSACTION_BLOCK_SCHEMA_NAME) .version(TRANSACTION_BLOCK_SCHEMA_VERSION) @@ -32,7 +34,7 @@ public Schema getOrderedTransactionBlockSchema() { .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, Schema.INT64_SCHEMA) .field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, Schema.INT64_SCHEMA) .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, Schema.INT64_SCHEMA) - .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, Schema.STRING_SCHEMA) + .field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, rankSchema) .build(); } } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index f7a7abcc..7f9ad69f 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -5,7 +5,7 @@ */ package io.debezium.connector.vitess.pipeline.txmetadata; -import java.math.BigInteger; +import java.math.BigDecimal; import java.util.Map; import io.debezium.connector.vitess.Vgtid; @@ -17,7 +17,7 @@ public class VitessOrderedTransactionContext extends TransactionContext { public static final String OFFSET_TRANSACTION_RANK = "transaction_rank"; protected String previousTransactionId = null; protected Long transactionEpoch = 0L; - protected BigInteger transactionRank = null; + protected BigDecimal transactionRank = null; private VitessEpochProvider epochProvider = new VitessEpochProvider(); private VitessRankProvider rankProvider = new VitessRankProvider(); @@ -32,6 +32,34 @@ public VitessOrderedTransactionContext(TransactionContext transactionContext) { this.totalEventCount = transactionContext.totalEventCount; } + /** + * Stores the needed information for determining Vitess rank & Epoch. Example (excluding standard fields added by super class): + * Input Offset map: + * { + * "transaction_id": "[{\"keyspace\":\ks1\",\"shard\":\"-80\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]} \ + * {\"keyspace\":\ks1\",\"shard\":\"80-\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]}" + * } + * Current shard to epoch map, in epoch provider: + * { + * "-80": 0, + * "80-", 1 + * } + * Output offset map: + * { + * "transaction_id": "[{\"keyspace\":\ks1\",\"shard\":\"-80\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]} \ + * {\"keyspace\":\ks1\",\"shard\":\"80-\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]}" + * "transaction_epoch": { + * "-80": 0, + * "80-", 1 + * } + * } + * + * Note: there is no need to store the transaction rank. We get the previous transaction ID from the "transaction_id" field + * and use that to compute the epoch. Rank requires no state (sum of max offsets of all hosts). + * + * @param offset + * @return + */ @Override public Map store(Map offset) { offset = super.store(offset); diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java index eea58fa1..d167c676 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java @@ -15,6 +15,21 @@ public class VitessOrderedTransactionStructMaker extends AbstractTransactionStructMaker implements TransactionStructMaker { + /** + * Adds the transaction block to a change log message. Transaction block example: + * "transaction": { + * "id": "[{\"keyspace\":\ks1\",\"shard\":\"-80\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]}, + * {\"keyspace\":\ks1\",\"shard\":\"80-\",\"gtid\":\"MySQL56/host1:123,host2:234\",\"table_p_ks\":[]}", + * "total_order": 1, + * "data_collection_order": 1, + * "transaction_epoch": 0, + * "transaction_rank": 853 + * } + * @param offsetContext + * @param dataCollectionEventOrder + * @param value + * @return Struct with ordered transaction metadata + */ @Override public Struct prepareTxStruct(OffsetContext offsetContext, long dataCollectionEventOrder, Struct value) { Struct struct = super.prepareTxStruct(offsetContext, dataCollectionEventOrder, value); @@ -23,7 +38,7 @@ public Struct prepareTxStruct(OffsetContext offsetContext, long dataCollectionEv private Struct addOrderMetadata(Struct struct, OffsetContext offsetContext) { VitessOrderedTransactionContext context = getVitessTransactionOrderMetadata(offsetContext); - struct.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, context.transactionRank.toString()); + struct.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, context.transactionRank); struct.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, context.transactionEpoch); return struct; } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java index ebda013f..829630e7 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java @@ -5,15 +5,15 @@ */ package io.debezium.connector.vitess.pipeline.txmetadata; -import java.math.BigInteger; +import java.math.BigDecimal; public class VitessRankProvider { - public BigInteger getRank(String transactionId) { + public BigDecimal getRank(String transactionId) { Gtid gtid = new Gtid(transactionId); - BigInteger rank = new BigInteger("0"); + BigDecimal rank = new BigDecimal("0"); for (String sequenceValue : gtid.getSequenceValues()) { - rank = rank.add(new BigInteger(sequenceValue)); + rank = rank.add(new BigDecimal(sequenceValue)); } return rank; } diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 85c5b514..1fda66ef 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -13,6 +13,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.math.BigDecimal; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; @@ -48,6 +49,7 @@ import io.debezium.connector.vitess.connection.VitessReplicationConnection; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessRankProvider; import io.debezium.converters.CloudEventsConverterTest; import io.debezium.converters.spi.CloudEventsMaker; import io.debezium.data.Envelope; @@ -478,10 +480,9 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { startConnector(config -> config .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) - .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) - .with(VitessConnectorConfig.SHARD, "-80,80-"), + .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true), true, - "80-"); + "-80,80-"); assertConnectorIsRunning(); Vgtid baseVgtid = TestHelper.getCurrentVgtid(); @@ -519,14 +520,18 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { SourceRecord beginRecord = assertRecordBeginSourceRecord(); assertThat(beginRecord.sourceOffset()).containsKey("transaction_epoch"); String expectedTxId1 = ((Struct) beginRecord.value()).getString("id"); + Long expectedEpoch = 0L; for (int i = 1; i <= expectedRecordsCount; i++) { SourceRecord record = assertRecordInserted(TEST_SHARDED_KEYSPACE + ".numeric_table", TestHelper.PK_FIELD); + Struct source = (Struct) ((Struct) record.value()).get("source"); + String shard = source.getString("shard"); final Struct txn = ((Struct) record.value()).getStruct("transaction"); String txId = txn.getString("id"); assertThat(txId).isNotNull(); assertThat(txId).isEqualTo(expectedTxId1); - assertThat(txn.get("transaction_epoch")).isEqualTo(0L); - assertThat(txn.get("transaction_rank")).isNotNull(); + assertThat(txn.get("transaction_epoch")).isEqualTo(expectedEpoch); + BigDecimal expectedRank = new VitessRankProvider().getRank(Vgtid.of(expectedTxId1).getShardGtid(shard).getGtid()); + assertThat(txn.get("transaction_rank")).isEqualTo(expectedRank); Vgtid actualVgtid = Vgtid.of(txId); // The current vgtid is not the previous vgtid. assertThat(actualVgtid).isNotEqualTo(baseVgtid); diff --git a/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java b/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java index 416f08d7..c102fbb5 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessSchemaFactoryTest.java @@ -9,6 +9,7 @@ import java.util.List; +import org.apache.kafka.connect.data.Decimal; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.junit.Test; @@ -31,7 +32,7 @@ public void getOrderedTransactionBlockSchema() { assertThat(fields).contains(new Field(TransactionStructMaker.DEBEZIUM_TRANSACTION_TOTAL_ORDER_KEY, 1, Schema.INT64_SCHEMA)); assertThat(fields).contains(new Field(TransactionStructMaker.DEBEZIUM_TRANSACTION_DATA_COLLECTION_ORDER_KEY, 2, Schema.INT64_SCHEMA)); assertThat(fields).contains(new Field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, 3, Schema.INT64_SCHEMA)); - assertThat(fields).contains(new Field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, 4, Schema.STRING_SCHEMA)); + assertThat(fields).contains(new Field(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK, 4, Decimal.schema(0))); } @Test diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java index 9685fa69..f1c7ebe5 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java @@ -7,7 +7,7 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.math.BigInteger; +import java.math.BigDecimal; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -17,6 +17,8 @@ import org.junit.Test; import io.debezium.connector.vitess.SourceInfo; +import io.debezium.connector.vitess.VgtidTest; +import io.debezium.pipeline.txmetadata.TransactionContext; public class VitessOrderedTransactionContextTest { @@ -29,13 +31,15 @@ public void shouldInit() { @Test public void shouldLoad() { - String expectedId = null; - String expectedEpoch = "{\"-80\": 0}"; + String expectedId = VgtidTest.VGTID_JSON; + String expectedEpoch = "{\"-80\": 5}"; Map offsets = Map.of( - VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch); - VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); - metadata.load(offsets); - assertThat(metadata.previousTransactionId).isEqualTo(expectedId); + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch, + TransactionContext.OFFSET_TRANSACTION_ID, expectedId); + VitessOrderedTransactionContext context = VitessOrderedTransactionContext.load(offsets); + assertThat(context.previousTransactionId).isEqualTo(expectedId); + context.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, "-80")); + assertThat(context.transactionEpoch).isEqualTo(5); } @Test @@ -54,7 +58,7 @@ public void shouldUpdateEpoch() { VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; - BigInteger expectedRank = new BigInteger("7"); + BigDecimal expectedRank = new BigDecimal("7"); long expectedEpoch = 0; String expectedShard = "-80"; @@ -64,7 +68,7 @@ public void shouldUpdateEpoch() { assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); String expectedTxId2 = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3\", \"shard\": \"-80\"}]"; - BigInteger expectedRank2 = new BigInteger("3"); + BigDecimal expectedRank2 = new BigDecimal("3"); long expectedEpoch2 = 1; VitessTransactionInfo transactionInfo2 = new VitessTransactionInfo(expectedTxId2, expectedShard); @@ -82,12 +86,12 @@ public void shouldUpdateRank() { VitessTransactionInfo transactionInfo = new VitessTransactionInfo(expectedTxId, expectedShard); metadata.beginTransaction(transactionInfo); - assertThat(metadata.transactionRank).isEqualTo(7); + assertThat(metadata.transactionRank).isEqualTo(new BigDecimal(7)); String expectedTxId2 = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3\", \"shard\": \"-80\"}]"; VitessTransactionInfo transactionInfo2 = new VitessTransactionInfo(expectedTxId2, expectedShard); metadata.beginTransaction(transactionInfo2); - assertThat(metadata.transactionRank).isEqualTo(3); + assertThat(metadata.transactionRank).isEqualTo(new BigDecimal(3)); } @Test diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java index 89a32d90..3524b777 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java @@ -7,6 +7,7 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.math.BigDecimal; import java.time.Instant; import org.apache.kafka.connect.data.Struct; @@ -32,7 +33,7 @@ public void prepareTxStruct() { OffsetContext context = new VitessOffsetContext(config, Vgtid.of(VgtidTest.VGTID_JSON), Instant.now(), transactionContext); Struct struct = maker.prepareTxStruct(context, 0, null); assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(0L); - assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK)).isEqualTo("1513"); + assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK)).isEqualTo(new BigDecimal(1513)); } @Test diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java index 35071965..2d1bbb77 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java @@ -7,7 +7,7 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.math.BigInteger; +import java.math.BigDecimal; import org.junit.Test; @@ -17,8 +17,8 @@ public class VitessRankProviderTest { public void shouldGetRankOneHost() { String txId = "host1:1-4"; VitessRankProvider provider = new VitessRankProvider(); - BigInteger rank = provider.getRank(txId); - assertThat(rank).isEqualTo(4); + BigDecimal rank = provider.getRank(txId); + assertThat(rank).isEqualTo(new BigDecimal(4)); } } From 084e805ccccfc715ebaeb3dc7741cd695d81927f Mon Sep 17 00:00:00 2001 From: twthorn Date: Tue, 2 Apr 2024 19:12:24 -0500 Subject: [PATCH 09/15] DBZ-7698 Improve naming, minor refactors --- .../pipeline/txmetadata/VitessEpochProvider.java | 8 ++++---- .../txmetadata/VitessOrderedTransactionContext.java | 11 +++++------ .../pipeline/txmetadata/VitessRankProvider.java | 2 +- .../debezium/connector/vitess/VitessConnectorIT.java | 2 +- .../pipeline/txmetadata/VitessEpochProviderTest.java | 10 +++++----- .../VitessOrderedTransactionContextTest.java | 4 ++-- .../pipeline/txmetadata/VitessRankProviderTest.java | 3 +-- 7 files changed, 19 insertions(+), 21 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index ac2e315f..e2889167 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -23,9 +23,9 @@ public class VitessEpochProvider { private Map shardToEpoch = new HashMap<>(); private static final ObjectMapper MAPPER = new ObjectMapper(); - public Long getEpoch(Long previousEpoch, String previousTransactionId, String transactionId) { - Gtid previousGtid = new Gtid(previousTransactionId); - Gtid gtid = new Gtid(transactionId); + public Long getEpochForGtid(Long previousEpoch, String previousGtidString, String gtidString) { + Gtid previousGtid = new Gtid(previousGtidString); + Gtid gtid = new Gtid(gtidString); if (previousGtid.isHostSetEqual(gtid) || gtid.isHostSetSupersetOf(previousGtid)) { return previousEpoch; } @@ -74,7 +74,7 @@ public Long getEpoch(String shard, String previousVgtidString, String vgtidStrin String previousGtid = previousVgtid.getShardGtid(shard).getGtid(); String gtid = vgtid.getShardGtid(shard).getGtid(); long previousEpoch = shardToEpoch.get(shard); - long currentEpoch = getEpoch(previousEpoch, previousGtid, gtid); + long currentEpoch = getEpochForGtid(previousEpoch, previousGtid, gtid); storeEpoch(shard, currentEpoch); return currentEpoch; } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index 7f9ad69f..14df5ba6 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -15,11 +15,10 @@ public class VitessOrderedTransactionContext extends TransactionContext { public static final String OFFSET_TRANSACTION_EPOCH = "transaction_epoch"; public static final String OFFSET_TRANSACTION_RANK = "transaction_rank"; - protected String previousTransactionId = null; + protected String previousVgtid = null; protected Long transactionEpoch = 0L; protected BigDecimal transactionRank = null; private VitessEpochProvider epochProvider = new VitessEpochProvider(); - private VitessRankProvider rankProvider = new VitessRankProvider(); public VitessOrderedTransactionContext() { } @@ -69,7 +68,7 @@ public Map store(Map offset) { public static VitessOrderedTransactionContext load(Map offsets) { TransactionContext transactionContext = TransactionContext.load(offsets); VitessOrderedTransactionContext vitessOrderedTransactionContext = new VitessOrderedTransactionContext(transactionContext); - vitessOrderedTransactionContext.previousTransactionId = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); + vitessOrderedTransactionContext.previousVgtid = (String) offsets.get(TransactionContext.OFFSET_TRANSACTION_ID); vitessOrderedTransactionContext.epochProvider.load(offsets); return vitessOrderedTransactionContext; } @@ -89,8 +88,8 @@ public void endTransaction() { } private void beginTransaction(String shard, String vgtid) { - this.transactionEpoch = this.epochProvider.getEpoch(shard, this.previousTransactionId, vgtid); - this.transactionRank = this.rankProvider.getRank(Vgtid.of(vgtid).getShardGtid(shard).getGtid()); - this.previousTransactionId = vgtid; + this.transactionEpoch = this.epochProvider.getEpoch(shard, this.previousVgtid, vgtid); + this.transactionRank = VitessRankProvider.getRank(Vgtid.of(vgtid).getShardGtid(shard).getGtid()); + this.previousVgtid = vgtid; } } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java index 829630e7..a06497fd 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProvider.java @@ -9,7 +9,7 @@ public class VitessRankProvider { - public BigDecimal getRank(String transactionId) { + public static BigDecimal getRank(String transactionId) { Gtid gtid = new Gtid(transactionId); BigDecimal rank = new BigDecimal("0"); for (String sequenceValue : gtid.getSequenceValues()) { diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 1fda66ef..cc5a3c85 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -530,7 +530,7 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { assertThat(txId).isNotNull(); assertThat(txId).isEqualTo(expectedTxId1); assertThat(txn.get("transaction_epoch")).isEqualTo(expectedEpoch); - BigDecimal expectedRank = new VitessRankProvider().getRank(Vgtid.of(expectedTxId1).getShardGtid(shard).getGtid()); + BigDecimal expectedRank = VitessRankProvider.getRank(Vgtid.of(expectedTxId1).getShardGtid(shard).getGtid()); assertThat(txn.get("transaction_rank")).isEqualTo(expectedRank); Vgtid actualVgtid = Vgtid.of(txId); // The current vgtid is not the previous vgtid. diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java index 785698c4..7372dbbf 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java @@ -27,21 +27,21 @@ public class VitessEpochProviderTest { @Test public void testGetEpochSameHostSet() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch(0L, previousTxId, txId); + Long epoch = provider.getEpochForGtid(0L, previousTxId, txId); assertThat(epoch).isEqualTo(0); } @Test public void testGetEpochShrunkHostSet() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch(0L, previousTxId, txIdShrunk); + Long epoch = provider.getEpochForGtid(0L, previousTxId, txIdShrunk); assertThat(epoch).isEqualTo(1); } @Test public void testGetEpochExpandHostSet() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch(0L, previousTxId, txId); + Long epoch = provider.getEpochForGtid(0L, previousTxId, txId); assertThat(epoch).isEqualTo(0); } @@ -49,14 +49,14 @@ public void testGetEpochExpandHostSet() { public void testGetEpochDisjointThrowsException() { VitessEpochProvider provider = new VitessEpochProvider(); Assertions.assertThatThrownBy(() -> { - provider.getEpoch(0L, previousTxId, "foo:1-2,bar:2-4"); + provider.getEpochForGtid(0L, previousTxId, "foo:1-2,bar:2-4"); }).isInstanceOf(RuntimeException.class); } @Test public void testVersionUpgradeDoesNotAffectEpoch() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch(0L, txIdVersion5, txIdVersion8); + Long epoch = provider.getEpochForGtid(0L, txIdVersion5, txIdVersion8); assertThat(epoch).isEqualTo(0L); } } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java index f1c7ebe5..d4d5aed3 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContextTest.java @@ -37,7 +37,7 @@ public void shouldLoad() { VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch, TransactionContext.OFFSET_TRANSACTION_ID, expectedId); VitessOrderedTransactionContext context = VitessOrderedTransactionContext.load(offsets); - assertThat(context.previousTransactionId).isEqualTo(expectedId); + assertThat(context.previousVgtid).isEqualTo(expectedId); context.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, "-80")); assertThat(context.transactionEpoch).isEqualTo(5); } @@ -49,7 +49,7 @@ public void shouldLoadWithNull() { Map offsets = Collections.emptyMap(); VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); metadata.load(offsets); - assertThat(metadata.previousTransactionId).isEqualTo(expectedId); + assertThat(metadata.previousVgtid).isEqualTo(expectedId); assertThat(metadata.transactionEpoch).isEqualTo(expectedEpoch); } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java index 2d1bbb77..4ff17202 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessRankProviderTest.java @@ -16,8 +16,7 @@ public class VitessRankProviderTest { @Test public void shouldGetRankOneHost() { String txId = "host1:1-4"; - VitessRankProvider provider = new VitessRankProvider(); - BigDecimal rank = provider.getRank(txId); + BigDecimal rank = VitessRankProvider.getRank(txId); assertThat(rank).isEqualTo(new BigDecimal(4)); } From c4e0eefa578d4e8cc17212d94ae32cfef666dfa1 Mon Sep 17 00:00:00 2001 From: twthorn Date: Wed, 3 Apr 2024 10:00:38 -0500 Subject: [PATCH 10/15] DBZ-7698 Add another test, refactor method to static --- .../txmetadata/VitessEpochProvider.java | 2 +- .../txmetadata/VitessEpochProviderTest.java | 28 ++++++++++++------- 2 files changed, 19 insertions(+), 11 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index e2889167..d2020424 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -23,7 +23,7 @@ public class VitessEpochProvider { private Map shardToEpoch = new HashMap<>(); private static final ObjectMapper MAPPER = new ObjectMapper(); - public Long getEpochForGtid(Long previousEpoch, String previousGtidString, String gtidString) { + public static Long getEpochForGtid(Long previousEpoch, String previousGtidString, String gtidString) { Gtid previousGtid = new Gtid(previousGtidString); Gtid gtid = new Gtid(gtidString); if (previousGtid.isHostSetEqual(gtid) || gtid.isHostSetSupersetOf(previousGtid)) { diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java index 7372dbbf..e59dd451 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java @@ -7,9 +7,13 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Map; + import org.assertj.core.api.Assertions; import org.junit.Test; +import io.debezium.connector.vitess.VgtidTest; + public class VitessEpochProviderTest { private String prefix = "MySQL56/"; @@ -26,37 +30,41 @@ public class VitessEpochProviderTest { @Test public void testGetEpochSameHostSet() { - VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpochForGtid(0L, previousTxId, txId); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId); assertThat(epoch).isEqualTo(0); } @Test - public void testGetEpochShrunkHostSet() { + public void testGetEpochVgtid() { VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpochForGtid(0L, previousTxId, txIdShrunk); + String expectedEpoch = "{\"-80\": 5}"; + provider.load(Map.of(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch)); + Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); + assertThat(epoch).isEqualTo(5); + } + + @Test + public void testGetEpochShrunkHostSet() { + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txIdShrunk); assertThat(epoch).isEqualTo(1); } @Test public void testGetEpochExpandHostSet() { - VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpochForGtid(0L, previousTxId, txId); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId); assertThat(epoch).isEqualTo(0); } @Test public void testGetEpochDisjointThrowsException() { - VitessEpochProvider provider = new VitessEpochProvider(); Assertions.assertThatThrownBy(() -> { - provider.getEpochForGtid(0L, previousTxId, "foo:1-2,bar:2-4"); + VitessEpochProvider.getEpochForGtid(0L, previousTxId, "foo:1-2,bar:2-4"); }).isInstanceOf(RuntimeException.class); } @Test public void testVersionUpgradeDoesNotAffectEpoch() { - VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpochForGtid(0L, txIdVersion5, txIdVersion8); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, txIdVersion5, txIdVersion8); assertThat(epoch).isEqualTo(0L); } } From e973a4949757ba117c7d570b548993e8ff499729 Mon Sep 17 00:00:00 2001 From: twthorn Date: Wed, 3 Apr 2024 12:37:17 -0500 Subject: [PATCH 11/15] DBZ-7698 Override and use load instance method --- .../connector/vitess/VitessOffsetContext.java | 2 +- .../VitessOrderedTransactionContext.java | 25 ++++++++++++++++++ .../vitess/VitessOffsetContextTest.java | 26 +++++++++++++++++-- 3 files changed, 50 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 057e4481..762f5d6e 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -147,7 +147,7 @@ public Loader(VitessConnectorConfig connectorConfig) { @Override public VitessOffsetContext load(Map offset) { final String vgtid = (String) offset.get(SourceInfo.VGTID_KEY); - TransactionContext transactionContext = connectorConfig.getTransactionContext(); + TransactionContext transactionContext = connectorConfig.getTransactionContext().newTransactionContextFromOffsets(offset); return new VitessOffsetContext( connectorConfig, Vgtid.of(vgtid), diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index 14df5ba6..75fa13f1 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -65,6 +65,11 @@ public Map store(Map offset) { return epochProvider.store(offset); } + @Override + public TransactionContext newTransactionContextFromOffsets(Map offsets) { + return load(offsets); + } + public static VitessOrderedTransactionContext load(Map offsets) { TransactionContext transactionContext = TransactionContext.load(offsets); VitessOrderedTransactionContext vitessOrderedTransactionContext = new VitessOrderedTransactionContext(transactionContext); @@ -92,4 +97,24 @@ private void beginTransaction(String shard, String vgtid) { this.transactionRank = VitessRankProvider.getRank(Vgtid.of(vgtid).getShardGtid(shard).getGtid()); this.previousVgtid = vgtid; } + + @Override + public String toString() { + return "VitessOrderedTransactionContext [currentTransactionId=" + transactionId + ", perTableEventCount=" + + perTableEventCount + ", totalEventCount=" + totalEventCount + "]" + ", previousVgtid=" + previousVgtid + + ", transactionEpoch=" + transactionEpoch + ", transactionRank=" + transactionRank; + } + + public String getPreviousVgtid() { + return previousVgtid; + } + + public Long getTransactionEpoch() { + return transactionEpoch; + } + + public BigDecimal getTransactionRank() { + return transactionRank; + } + } diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 2dbdcf3b..1ecab07a 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -13,7 +13,11 @@ import org.junit.Before; import org.junit.Test; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionInfo; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.util.Clock; import io.debezium.util.Collect; @@ -100,16 +104,32 @@ public void shouldResetToNewVGgtid() { } @Test - public void shouldLoadVitessOrderedTransactionContext() { + public void shouldLoadVitessOrderedTransactionContext() throws JsonProcessingException { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) .build()); VitessOffsetContext.Loader loader = new VitessOffsetContext.Loader(config); - Map offsets = Map.of(SourceInfo.VGTID_KEY, VGTID_JSON); + ObjectMapper objectMapper = new ObjectMapper(); + Long expectedEpoch1 = 2L; + Long expectedEpoch2 = 3L; + String shard1 = "-80"; + String shard2 = "80-"; + Map offsets = Map.of( + SourceInfo.VGTID_KEY, VGTID_JSON, + TransactionContext.OFFSET_TRANSACTION_ID, VGTID_JSON, + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, objectMapper.writeValueAsString(Map.of( + shard1, expectedEpoch1, + shard2, expectedEpoch2))); VitessOffsetContext context = loader.load(offsets); TransactionContext transactionContext = context.getTransactionContext(); assertThat(transactionContext).isInstanceOf(VitessOrderedTransactionContext.class); + VitessOrderedTransactionContext orderedTransactionContext = (VitessOrderedTransactionContext) transactionContext; + assertThat(orderedTransactionContext.getPreviousVgtid()).isEqualTo(VGTID_JSON); + orderedTransactionContext.beginTransaction(new VitessTransactionInfo(VGTID_JSON, shard1)); + assertThat(orderedTransactionContext.getTransactionEpoch()).isEqualTo(expectedEpoch1); + orderedTransactionContext.beginTransaction(new VitessTransactionInfo(VGTID_JSON, shard2)); + assertThat(orderedTransactionContext.getTransactionEpoch()).isEqualTo(expectedEpoch2); } @Test @@ -121,5 +141,7 @@ public void shouldGetInitialVitessOrderedTransactionContext() { VitessOffsetContext context = VitessOffsetContext.initialContext(config, Clock.system()); TransactionContext transactionContext = context.getTransactionContext(); assertThat(transactionContext).isInstanceOf(VitessOrderedTransactionContext.class); + VitessOrderedTransactionContext orderedTransactionContext = (VitessOrderedTransactionContext) transactionContext; + assertThat(orderedTransactionContext.getPreviousVgtid()).isEqualTo(null); } } From 34541c87ed948ff23b5c0987ebeefee1dd8a3235 Mon Sep 17 00:00:00 2001 From: twthorn Date: Wed, 3 Apr 2024 14:51:43 -0500 Subject: [PATCH 12/15] DBZ-7698 Handle current/empty gtid for epoch provider, and missing shard in shard to epoch map --- .../connector/vitess/VitessOffsetContext.java | 1 + .../vitess/pipeline/txmetadata/Gtid.java | 5 +++ .../txmetadata/VitessEpochProvider.java | 2 +- .../VitessOrderedTransactionContext.java | 2 +- .../debezium/connector/vitess/TestHelper.java | 2 +- .../txmetadata/VitessEpochProviderTest.java | 38 +++++++++++++++++++ 6 files changed, 47 insertions(+), 3 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 762f5d6e..1654475e 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -146,6 +146,7 @@ public Loader(VitessConnectorConfig connectorConfig) { @Override public VitessOffsetContext load(Map offset) { + LOGGER.info("Previous offset exists, load from {}", offset); final String vgtid = (String) offset.get(SourceInfo.VGTID_KEY); TransactionContext transactionContext = connectorConfig.getTransactionContext().newTransactionContextFromOffsets(offset); return new VitessOffsetContext( diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java index bf8b10f1..834b0d70 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java @@ -10,6 +10,8 @@ import java.util.List; import java.util.Set; +import io.debezium.connector.vitess.Vgtid; + class Gtid { public String getVersion() { @@ -58,6 +60,9 @@ private void initializeVersion(String transactionId) { private void parseGtid(String transactionId) { transactionId = trimVersion(transactionId); + if (transactionId.equals(Vgtid.CURRENT_GTID) || transactionId.equals(Vgtid.EMPTY_GTID)) { + return; + } String[] transactions = transactionId.split(","); for (String transaction : transactions) { String[] parts = transaction.split(":"); diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index d2020424..2bf298bc 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -73,7 +73,7 @@ public Long getEpoch(String shard, String previousVgtidString, String vgtidStrin Vgtid previousVgtid = Vgtid.of(previousVgtidString); String previousGtid = previousVgtid.getShardGtid(shard).getGtid(); String gtid = vgtid.getShardGtid(shard).getGtid(); - long previousEpoch = shardToEpoch.get(shard); + long previousEpoch = shardToEpoch.getOrDefault(shard, 0L); long currentEpoch = getEpochForGtid(previousEpoch, previousGtid, gtid); storeEpoch(shard, currentEpoch); return currentEpoch; diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index 75fa13f1..8bbbb580 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -67,7 +67,7 @@ public Map store(Map offset) { @Override public TransactionContext newTransactionContextFromOffsets(Map offsets) { - return load(offsets); + return VitessOrderedTransactionContext.load(offsets); } public static VitessOrderedTransactionContext load(Map offsets) { diff --git a/src/test/java/io/debezium/connector/vitess/TestHelper.java b/src/test/java/io/debezium/connector/vitess/TestHelper.java index 6d229d25..c1c1c65a 100644 --- a/src/test/java/io/debezium/connector/vitess/TestHelper.java +++ b/src/test/java/io/debezium/connector/vitess/TestHelper.java @@ -59,7 +59,7 @@ public class TestHelper { "{\"keyspace\":\"%s\",\"shard\":\"%s\",\"gtid\":\"%s\"}" + "]"; - protected static final String VGTID_JSON_TEMPLATE = "[" + + public static final String VGTID_JSON_TEMPLATE = "[" + "{\"keyspace\":\"%s\",\"shard\":\"%s\",\"gtid\":\"%s\",\"table_p_ks\":[]}," + "{\"keyspace\":\"%s\",\"shard\":\"%s\",\"gtid\":\"%s\",\"table_p_ks\":[]}" + "]"; diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java index e59dd451..def44e4e 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProviderTest.java @@ -5,6 +5,7 @@ */ package io.debezium.connector.vitess.pipeline.txmetadata; +import static io.debezium.connector.vitess.TestHelper.VGTID_JSON_TEMPLATE; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -12,6 +13,7 @@ import org.assertj.core.api.Assertions; import org.junit.Test; +import io.debezium.connector.vitess.Vgtid; import io.debezium.connector.vitess.VgtidTest; public class VitessEpochProviderTest { @@ -43,6 +45,42 @@ public void testGetEpochVgtid() { assertThat(epoch).isEqualTo(5); } + @Test + public void testIncrementEpochVgtidCurrent() { + Long expectedEpoch = 0L; + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); + assertThat(epoch).isEqualTo(expectedEpoch); + String vgtidJsonCurrent = String.format( + VGTID_JSON_TEMPLATE, + VgtidTest.TEST_KEYSPACE, + VgtidTest.TEST_SHARD, + Vgtid.CURRENT_GTID, + VgtidTest.TEST_KEYSPACE, + VgtidTest.TEST_SHARD2, + Vgtid.CURRENT_GTID); + Long epoch2 = provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonCurrent); + assertThat(epoch2).isEqualTo(1L); + } + + @Test + public void testIncrementEpochVgtidEmpty() { + Long expectedEpoch = 0L; + VitessEpochProvider provider = new VitessEpochProvider(); + Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); + assertThat(epoch).isEqualTo(expectedEpoch); + String vgtidJsonCurrent = String.format( + VGTID_JSON_TEMPLATE, + VgtidTest.TEST_KEYSPACE, + VgtidTest.TEST_SHARD, + Vgtid.EMPTY_GTID, + VgtidTest.TEST_KEYSPACE, + VgtidTest.TEST_SHARD2, + Vgtid.EMPTY_GTID); + Long epoch2 = provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonCurrent); + assertThat(epoch2).isEqualTo(1L); + } + @Test public void testGetEpochShrunkHostSet() { Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txIdShrunk); From 80b45a76c3f3c2a8f9baa3c4535a6e1e6becd28a Mon Sep 17 00:00:00 2001 From: twthorn Date: Thu, 4 Apr 2024 15:41:38 -0500 Subject: [PATCH 13/15] DBZ-7698 Add integration test for epoch change --- .../vitess/pipeline/txmetadata/Gtid.java | 5 - .../txmetadata/VitessEpochProvider.java | 11 ++ .../connector/vitess/VitessConnectorIT.java | 102 +++++++++++++++++- .../txmetadata/VitessEpochProviderTest.java | 52 +++++++-- 4 files changed, 154 insertions(+), 16 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java index 834b0d70..bf8b10f1 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/Gtid.java @@ -10,8 +10,6 @@ import java.util.List; import java.util.Set; -import io.debezium.connector.vitess.Vgtid; - class Gtid { public String getVersion() { @@ -60,9 +58,6 @@ private void initializeVersion(String transactionId) { private void parseGtid(String transactionId) { transactionId = trimVersion(transactionId); - if (transactionId.equals(Vgtid.CURRENT_GTID) || transactionId.equals(Vgtid.EMPTY_GTID)) { - return; - } String[] transactions = transactionId.split(","); for (String transaction : transactions) { String[] parts = transaction.split(":"); diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java index 2bf298bc..c703c943 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessEpochProvider.java @@ -15,6 +15,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.DebeziumException; import io.debezium.connector.vitess.Vgtid; public class VitessEpochProvider { @@ -23,7 +24,17 @@ public class VitessEpochProvider { private Map shardToEpoch = new HashMap<>(); private static final ObjectMapper MAPPER = new ObjectMapper(); + private static boolean isInvalidGtid(String gtid) { + return gtid.equals(Vgtid.CURRENT_GTID) || gtid.equals(Vgtid.EMPTY_GTID); + } + public static Long getEpochForGtid(Long previousEpoch, String previousGtidString, String gtidString) { + if (isInvalidGtid(previousGtidString)) { + return previousEpoch + 1; + } + if (isInvalidGtid(gtidString)) { + throw new DebeziumException("Invalid GTID: The current GTID cannot be one of current or empty " + gtidString); + } Gtid previousGtid = new Gtid(previousGtidString); Gtid gtid = new Gtid(gtidString); if (previousGtid.isHostSetEqual(gtid) || gtid.isHostSetSupersetOf(previousGtid)) { diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index cc5a3c85..a98975fc 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -8,6 +8,7 @@ import static io.debezium.connector.vitess.TestHelper.TEST_SERVER; import static io.debezium.connector.vitess.TestHelper.TEST_SHARDED_KEYSPACE; import static io.debezium.connector.vitess.TestHelper.TEST_UNSHARDED_KEYSPACE; +import static io.debezium.connector.vitess.TestHelper.VGTID_JSON_TEMPLATE; import static junit.framework.TestCase.assertEquals; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.Assert.assertFalse; @@ -43,6 +44,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.fasterxml.jackson.databind.ObjectMapper; + import io.debezium.config.CommonConnectorConfig; import io.debezium.config.Configuration; import io.debezium.config.Field; @@ -539,6 +542,95 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { assertRecordEnd(expectedTxId1, expectedRecordsCount); } + @Test + public void shouldIncrementEpochWhenFastForwardVgtidWithOrderedTransactionMetadata() throws Exception { + TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); + TestHelper.applyVSchema("vitess_vschema.json"); + + ObjectMapper mapper = new ObjectMapper(); + Map srcPartition = Collect.hashMapOf(VitessPartition.SERVER_PARTITION_KEY, TEST_SERVER); + String currentVgtid = String.format( + VGTID_JSON_TEMPLATE, + TEST_SHARDED_KEYSPACE, + VgtidTest.TEST_SHARD, + Vgtid.CURRENT_GTID, + TEST_SHARDED_KEYSPACE, + VgtidTest.TEST_SHARD2, + Vgtid.CURRENT_GTID); + Map shardToEpoch = Map.of(VgtidTest.TEST_SHARD, 2L, VgtidTest.TEST_SHARD2, 3L); + Map offsetId = Map.of( + VitessOrderedTransactionContext.OFFSET_TRANSACTION_ID, currentVgtid, + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, mapper.writeValueAsString(shardToEpoch), + SourceInfo.VGTID_KEY, currentVgtid); + Map, Map> offsets = Map.of(srcPartition, offsetId); + Configuration config = TestHelper.defaultConfig() + .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) + .with(CommonConnectorConfig.TOPIC_PREFIX, TEST_SERVER) + .with(VitessConnectorConfig.KEYSPACE, TEST_SHARDED_KEYSPACE) + .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) + .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) + .with(VitessConnectorConfig.SHARD, "-80,80-") + .build(); + + storeOffsets(config, offsets); + + startConnector(config); + assertConnectorIsRunning(); + + Vgtid baseVgtid = TestHelper.getCurrentVgtid(); + int expectedRecordsCount = 1; + consumer = testConsumer(expectedRecordsCount + 2); + + String rowValue = "(1, 1, 12, 12, 123, 123, 1234, 1234, 12345, 12345, 18446744073709551615, 1.5, 2.5, 12.34, true)"; + String insertQuery = "INSERT INTO numeric_table (" + + "tinyint_col," + + "tinyint_unsigned_col," + + "smallint_col," + + "smallint_unsigned_col," + + "mediumint_col," + + "mediumint_unsigned_col," + + "int_col," + + "int_unsigned_col," + + "bigint_col," + + "bigint_unsigned_col," + + "bigint_unsigned_overflow_col," + + "float_col," + + "double_col," + + "decimal_col," + + "boolean_col)" + + " VALUES " + rowValue; + StringBuilder insertRows = new StringBuilder().append(insertQuery); + for (int i = 1; i < expectedRecordsCount; i++) { + insertRows.append(", ").append(rowValue); + } + + String insertRowsStatement = insertRows.toString(); + + // exercise SUT + executeAndWait(insertRowsStatement, TEST_SHARDED_KEYSPACE); + // First transaction. + SourceRecord beginRecord = assertRecordBeginSourceRecord(); + assertThat(beginRecord.sourceOffset()).containsKey("transaction_epoch"); + String expectedTxId1 = ((Struct) beginRecord.value()).getString("id"); + for (int i = 1; i <= expectedRecordsCount; i++) { + SourceRecord record = assertRecordInserted(TEST_SHARDED_KEYSPACE + ".numeric_table", TestHelper.PK_FIELD); + Struct source = (Struct) ((Struct) record.value()).get("source"); + String shard = source.getString("shard"); + Long expectedEpoch = shardToEpoch.get(shard) + 1; + final Struct txn = ((Struct) record.value()).getStruct("transaction"); + String txId = txn.getString("id"); + assertThat(txId).isNotNull(); + assertThat(txId).isEqualTo(expectedTxId1); + assertThat(txn.get("transaction_epoch")).isEqualTo(expectedEpoch); + BigDecimal expectedRank = VitessRankProvider.getRank(Vgtid.of(expectedTxId1).getShardGtid(shard).getGtid()); + assertThat(txn.get("transaction_rank")).isEqualTo(expectedRank); + Vgtid actualVgtid = Vgtid.of(txId); + // The current vgtid is not the previous vgtid. + assertThat(actualVgtid).isNotEqualTo(baseVgtid); + } + assertRecordEnd(expectedTxId1, expectedRecordsCount); + } + @Test @FixFor("DBZ-5063") public void shouldUseSameTransactionIdWhenMultiGrpcResponses() throws Exception { @@ -1293,10 +1385,16 @@ private void startConnector(Function { + provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonCurrent); + }).isInstanceOf(DebeziumException.class).hasMessageContaining("Invalid"); } @Test - public void testIncrementEpochVgtidEmpty() { + public void testInvalidEmptyGtid() { Long expectedEpoch = 0L; VitessEpochProvider provider = new VitessEpochProvider(); Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); assertThat(epoch).isEqualTo(expectedEpoch); - String vgtidJsonCurrent = String.format( + String vgtidJsonEmpty = String.format( VGTID_JSON_TEMPLATE, VgtidTest.TEST_KEYSPACE, VgtidTest.TEST_SHARD, @@ -77,8 +110,9 @@ public void testIncrementEpochVgtidEmpty() { VgtidTest.TEST_KEYSPACE, VgtidTest.TEST_SHARD2, Vgtid.EMPTY_GTID); - Long epoch2 = provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonCurrent); - assertThat(epoch2).isEqualTo(1L); + assertThatThrownBy(() -> { + provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonEmpty); + }).isInstanceOf(DebeziumException.class).hasMessageContaining("Invalid"); } @Test From ccfc7b930aa03a574b1241cac0da96c341e3c172 Mon Sep 17 00:00:00 2001 From: twthorn Date: Thu, 11 Apr 2024 13:56:00 -0500 Subject: [PATCH 14/15] DBZ-7698 Use DebeziumException, refactor things to match debezium-core --- src/main/java/io/debezium/connector/vitess/Vgtid.java | 4 +++- .../txmetadata/VitessOrderedTransactionContext.java | 10 +++++----- .../VitessOrderedTransactionStructMaker.java | 9 +++++++-- .../java/io/debezium/connector/vitess/VgtidTest.java | 10 ++++++++++ .../VitessOrderedTransactionStructMakerTest.java | 7 ++++--- 5 files changed, 29 insertions(+), 11 deletions(-) diff --git a/src/main/java/io/debezium/connector/vitess/Vgtid.java b/src/main/java/io/debezium/connector/vitess/Vgtid.java index 8cce24f8..c9f8bab1 100644 --- a/src/main/java/io/debezium/connector/vitess/Vgtid.java +++ b/src/main/java/io/debezium/connector/vitess/Vgtid.java @@ -17,6 +17,8 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; +import io.debezium.DebeziumException; + import binlogdata.Binlogdata; /** Vitess source position coordinates. */ @@ -93,7 +95,7 @@ public ShardGtid getShardGtid(String shard) { return shardGtid; } } - throw new RuntimeException("Gtid for shard missing, shard: " + shard + "vgtid: " + this.rawVgtid.toString()); + throw new DebeziumException("Gtid for shard missing, shard: " + shard + "vgtid: " + this.rawVgtid.toString()); } public boolean isSingleShard() { diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java index 8bbbb580..27fd528c 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionContext.java @@ -26,9 +26,9 @@ public VitessOrderedTransactionContext() { public VitessOrderedTransactionContext(TransactionContext transactionContext) { super(); // Copy fields - this.transactionId = transactionContext.transactionId; - this.perTableEventCount.putAll(transactionContext.perTableEventCount); - this.totalEventCount = transactionContext.totalEventCount; + super.setTransactionId(transactionContext.getTransactionId()); + super.putPerTableEventCount(transactionContext.getPerTableEventCount()); + super.setTotalEventCount(transactionContext.getTotalEventCount()); } /** @@ -100,8 +100,8 @@ private void beginTransaction(String shard, String vgtid) { @Override public String toString() { - return "VitessOrderedTransactionContext [currentTransactionId=" + transactionId + ", perTableEventCount=" - + perTableEventCount + ", totalEventCount=" + totalEventCount + "]" + ", previousVgtid=" + previousVgtid + return "VitessOrderedTransactionContext [currentTransactionId=" + getTransactionId() + ", perTableEventCount=" + + getPerTableEventCount() + ", totalEventCount=" + getTotalEventCount() + "]" + ", previousVgtid=" + previousVgtid + ", transactionEpoch=" + transactionEpoch + ", transactionRank=" + transactionRank; } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java index d167c676..65b15d21 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMaker.java @@ -8,6 +8,7 @@ import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.Struct; +import io.debezium.config.Configuration; import io.debezium.connector.vitess.VitessSchemaFactory; import io.debezium.pipeline.spi.OffsetContext; import io.debezium.pipeline.txmetadata.AbstractTransactionStructMaker; @@ -15,6 +16,10 @@ public class VitessOrderedTransactionStructMaker extends AbstractTransactionStructMaker implements TransactionStructMaker { + public VitessOrderedTransactionStructMaker(Configuration config) { + super(config); + } + /** * Adds the transaction block to a change log message. Transaction block example: * "transaction": { @@ -31,8 +36,8 @@ public class VitessOrderedTransactionStructMaker extends AbstractTransactionStru * @return Struct with ordered transaction metadata */ @Override - public Struct prepareTxStruct(OffsetContext offsetContext, long dataCollectionEventOrder, Struct value) { - Struct struct = super.prepareTxStruct(offsetContext, dataCollectionEventOrder, value); + public Struct addTransactionBlock(OffsetContext offsetContext, long dataCollectionEventOrder, Struct value) { + Struct struct = super.addTransactionBlock(offsetContext, dataCollectionEventOrder, value); return addOrderMetadata(struct, offsetContext); } diff --git a/src/test/java/io/debezium/connector/vitess/VgtidTest.java b/src/test/java/io/debezium/connector/vitess/VgtidTest.java index 13eb3a0f..f8b9ac7d 100644 --- a/src/test/java/io/debezium/connector/vitess/VgtidTest.java +++ b/src/test/java/io/debezium/connector/vitess/VgtidTest.java @@ -14,12 +14,14 @@ import static io.debezium.connector.vitess.TestHelper.VGTID_JSON_NO_PKS_TEMPLATE; import static io.debezium.connector.vitess.TestHelper.VGTID_JSON_TEMPLATE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.junit.Test; import org.skyscreamer.jsonassert.JSONAssert; +import io.debezium.DebeziumException; import io.debezium.util.Collect; import binlogdata.Binlogdata; @@ -310,4 +312,12 @@ public void shouldGetShardGtid() { Vgtid.ShardGtid shardGtid = vgtid1.getShardGtid(TEST_SHARD); assertThat(shardGtid).isEqualTo(new Vgtid.ShardGtid(TEST_KEYSPACE, TEST_SHARD, TEST_GTID)); } + + @Test + public void shouldGetMissingShardGtidThrowsDebeziumException() { + Vgtid vgtid1 = Vgtid.of(VGTID_JSON); + assertThatThrownBy(() -> { + Vgtid.ShardGtid shardGtid = vgtid1.getShardGtid("missing_shard"); + }).isInstanceOf(DebeziumException.class); + } } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java index 3524b777..c5e7dc1b 100644 --- a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionStructMakerTest.java @@ -13,6 +13,7 @@ import org.apache.kafka.connect.data.Struct; import org.junit.Test; +import io.debezium.config.Configuration; import io.debezium.connector.vitess.TestHelper; import io.debezium.connector.vitess.Vgtid; import io.debezium.connector.vitess.VgtidTest; @@ -27,18 +28,18 @@ public class VitessOrderedTransactionStructMakerTest { @Test public void prepareTxStruct() { VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig().build()); - VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(); + VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(Configuration.empty()); TransactionContext transactionContext = new VitessOrderedTransactionContext(); transactionContext.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, VgtidTest.TEST_SHARD)); OffsetContext context = new VitessOffsetContext(config, Vgtid.of(VgtidTest.VGTID_JSON), Instant.now(), transactionContext); - Struct struct = maker.prepareTxStruct(context, 0, null); + Struct struct = maker.addTransactionBlock(context, 0, null); assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(0L); assertThat(struct.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_RANK)).isEqualTo(new BigDecimal(1513)); } @Test public void getTransactionBlockSchema() { - VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(); + VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(Configuration.empty()); assertThat(maker.getTransactionBlockSchema()).isEqualTo(VitessSchemaFactory.get().getOrderedTransactionBlockSchema()); } } From 947f303a3257ad443b4289544241929553a0a3b8 Mon Sep 17 00:00:00 2001 From: twthorn Date: Tue, 16 Apr 2024 12:21:54 -0500 Subject: [PATCH 15/15] DBZ-7698 Refactor to single factory interface/class/config --- .../vitess/VitessDatabaseSchema.java | 2 +- .../connector/vitess/VitessOffsetContext.java | 5 +-- ...tessOrderedTransactionMetadataFactory.java | 31 +++++++++++++++++++ .../connector/vitess/VitessConnectorIT.java | 8 ++--- .../vitess/VitessOffsetContextTest.java | 5 +-- 5 files changed, 41 insertions(+), 10 deletions(-) create mode 100644 src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java diff --git a/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java b/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java index 0ec23f44..36c26130 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java +++ b/src/main/java/io/debezium/connector/vitess/VitessDatabaseSchema.java @@ -45,7 +45,7 @@ public VitessDatabaseSchema( schemaNameAdjuster, config.customConverterRegistry(), config.getSourceInfoStructMaker().schema(), - config.getTransactionStructMaker().getTransactionBlockSchema(), + config.getTransactionMetadataFactory().getTransactionStructMaker().getTransactionBlockSchema(), config.getFieldNamer(), false), false, diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 1654475e..0ebce59d 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -51,7 +51,7 @@ public static VitessOffsetContext initialContext( LOGGER.info("No previous offset exists. Use default VGTID."); final Vgtid defaultVgtid = VitessReplicationConnection.defaultVgtid(connectorConfig); // use the other transaction context - TransactionContext transactionContext = connectorConfig.getTransactionContext(); + TransactionContext transactionContext = connectorConfig.getTransactionMetadataFactory().getTransactionContext(); VitessOffsetContext context = new VitessOffsetContext( connectorConfig, defaultVgtid, clock.currentTimeAsInstant(), transactionContext); return context; @@ -148,7 +148,8 @@ public Loader(VitessConnectorConfig connectorConfig) { public VitessOffsetContext load(Map offset) { LOGGER.info("Previous offset exists, load from {}", offset); final String vgtid = (String) offset.get(SourceInfo.VGTID_KEY); - TransactionContext transactionContext = connectorConfig.getTransactionContext().newTransactionContextFromOffsets(offset); + TransactionContext transactionContext = connectorConfig.getTransactionMetadataFactory() + .getTransactionContext().newTransactionContextFromOffsets(offset); return new VitessOffsetContext( connectorConfig, Vgtid.of(vgtid), diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java new file mode 100644 index 00000000..391a787e --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java @@ -0,0 +1,31 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.vitess.pipeline.txmetadata; + +import io.debezium.config.Configuration; +import io.debezium.pipeline.txmetadata.TransactionContext; +import io.debezium.pipeline.txmetadata.TransactionStructMaker; +import io.debezium.pipeline.txmetadata.spi.TransactionMetadataFactory; + +public class VitessOrderedTransactionMetadataFactory implements TransactionMetadataFactory { + + private final Configuration configuraiton; + + public VitessOrderedTransactionMetadataFactory(Configuration configuration) { + this.configuraiton = configuration; + } + + @Override + public TransactionContext getTransactionContext() { + return new VitessOrderedTransactionContext(); + } + + @Override + public TransactionStructMaker getTransactionStructMaker() { + return new VitessOrderedTransactionStructMaker(configuraiton); + } +} diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index a98975fc..c4b83b05 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -51,7 +51,7 @@ import io.debezium.config.Field; import io.debezium.connector.vitess.connection.VitessReplicationConnection; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; -import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionStructMaker; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; import io.debezium.connector.vitess.pipeline.txmetadata.VitessRankProvider; import io.debezium.converters.CloudEventsConverterTest; import io.debezium.converters.spi.CloudEventsMaker; @@ -481,8 +481,7 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); TestHelper.applyVSchema("vitess_vschema.json"); startConnector(config -> config - .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) - .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) + .with(CommonConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true), true, "-80,80-"); @@ -564,10 +563,9 @@ public void shouldIncrementEpochWhenFastForwardVgtidWithOrderedTransactionMetada SourceInfo.VGTID_KEY, currentVgtid); Map, Map> offsets = Map.of(srcPartition, offsetId); Configuration config = TestHelper.defaultConfig() - .with(CommonConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) + .with(CommonConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) .with(CommonConnectorConfig.TOPIC_PREFIX, TEST_SERVER) .with(VitessConnectorConfig.KEYSPACE, TEST_SHARDED_KEYSPACE) - .with(CommonConnectorConfig.TRANSACTION_STRUCT_MAKER, VitessOrderedTransactionStructMaker.class) .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) .with(VitessConnectorConfig.SHARD, "-80,80-") .build(); diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 1ecab07a..3b53ff3b 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -17,6 +17,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; import io.debezium.connector.vitess.pipeline.txmetadata.VitessTransactionInfo; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.util.Clock; @@ -107,7 +108,7 @@ public void shouldResetToNewVGgtid() { public void shouldLoadVitessOrderedTransactionContext() throws JsonProcessingException { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) + .with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) .build()); VitessOffsetContext.Loader loader = new VitessOffsetContext.Loader(config); ObjectMapper objectMapper = new ObjectMapper(); @@ -136,7 +137,7 @@ public void shouldLoadVitessOrderedTransactionContext() throws JsonProcessingExc public void shouldGetInitialVitessOrderedTransactionContext() { VitessConnectorConfig config = new VitessConnectorConfig( TestHelper.defaultConfig() - .with(VitessConnectorConfig.TRANSACTION_CONTEXT, VitessOrderedTransactionContext.class) + .with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) .build()); VitessOffsetContext context = VitessOffsetContext.initialContext(config, Clock.system()); TransactionContext transactionContext = context.getTransactionContext();