diff --git a/src/main/java/io/debezium/connector/vitess/Vgtid.java b/src/main/java/io/debezium/connector/vitess/Vgtid.java index 8a0de71a..b66f46f8 100644 --- a/src/main/java/io/debezium/connector/vitess/Vgtid.java +++ b/src/main/java/io/debezium/connector/vitess/Vgtid.java @@ -18,8 +18,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; -import io.debezium.DebeziumException; - import binlogdata.Binlogdata; /** Vitess source position coordinates. */ @@ -112,9 +110,6 @@ public Vgtid getLocalVgtid(String shard) { public ShardGtid getShardGtid(String shard) { ShardGtid shardGtid = shardNameToShardGtid.get(shard); - if (shardGtid == null) { - throw new DebeziumException("Gtid for shard missing, shard: " + shard + "vgtid: " + this.rawVgtid.toString()); - } return shardGtid; } diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnector.java b/src/main/java/io/debezium/connector/vitess/VitessConnector.java index c92e912c..37ca08cd 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnector.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnector.java @@ -13,7 +13,6 @@ import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -21,8 +20,6 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigValue; import org.apache.kafka.connect.connector.Task; -import org.apache.kafka.connect.source.SourceConnectorContext; -import org.apache.kafka.connect.storage.OffsetStorageReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,57 +54,6 @@ public Class taskClass() { return VitessConnectorTask.class; } - protected Map getGtidPerShardFromStorage(int numTasks, int gen, boolean expectsOffset) { - // Note that in integration test, EmbeddedEngine didn't initialize SourceConnector with SourceConnectorContext - if (context == null - || !(context instanceof SourceConnectorContext) || context().offsetStorageReader() == null) { - LOGGER.warn("Context {} is not setup for the connector, this can happen in unit tests.", context); - return null; - } - return getGtidPerShardFromStorage( - context().offsetStorageReader(), - connectorConfig, - numTasks, - gen, - expectsOffset); - } - - public static Map getGtidPerShardFromStorage( - OffsetStorageReader offsetStorageReader, - VitessConnectorConfig connectorConfig, - int numTasks, int gen, boolean expectsOffset) { - if (gen < 0) { - return Collections.emptyMap(); - } - final Map gtidsPerShard = new HashMap<>(); - for (int i = 0; i < numTasks; i++) { - String taskKey = VitessConnector.getTaskKeyName(i, numTasks, gen); - VitessPartition par = new VitessPartition(connectorConfig.getLogicalName(), taskKey); - Map offset = offsetStorageReader.offset(par.getSourcePartition()); - if (offset == null && gen == 0) { - LOGGER.info("No previous offset for partition: {}, fall back to only server key", par); - par = new VitessPartition(connectorConfig.getLogicalName(), null); - offset = offsetStorageReader.offset(par.getSourcePartition()); - } - if (offset == null) { - if (expectsOffset) { - throw new IllegalArgumentException(String.format("No offset found for %s", par)); - } - else { - LOGGER.warn("No offset found for task key: {}", taskKey); - continue; - } - } - final String vgtidStr = (String) offset.get(SourceInfo.VGTID_KEY); - Objects.requireNonNull(vgtidStr, String.format("No vgtid from %s", offset)); - List shardGtids = Vgtid.of(vgtidStr).getShardGtids(); - for (Vgtid.ShardGtid shardGtid : shardGtids) { - gtidsPerShard.put(shardGtid.getShard(), shardGtid.getGtid()); - } - } - return gtidsPerShard; - } - @Override public List> taskConfigs(int maxTasks) { LOGGER.info("Calculating taskConfigs for {} tasks", maxTasks); @@ -148,68 +94,35 @@ public List> taskConfigs(int maxTasks, List currentS LOGGER.info("There are {} vitess shards for maxTasks: {}, we will use {} tasks", currentShards == null ? null : currentShards.size(), maxTasks, tasks); // Check the task offsets persisted from previous gen, we expect the offsets are saved - Map prevGtidsPerShard = gen > 0 ? getGtidPerShardFromStorage(prevNumTasks, gen - 1, true) : null; - LOGGER.info("Previous gtids Per shard: {}", prevGtidsPerShard); - Set previousShards = prevGtidsPerShard != null ? prevGtidsPerShard.keySet() : null; - if (gen > 0 && tasks == prevNumTasks && hasSameShards(previousShards, currentShards)) { - throw new IllegalArgumentException(String.format( - "Previous num.tasks: %s and current num.tasks: %s are the same. " - + "And previous shards: %s and current shards: %s are the same. " - + "Please choose different tasks.max or have different number of vitess shards " - + "if you want to change the task parallelism. " - + "Otherwise please reset the offset.storage.task.key.gen config to its original value", - prevNumTasks, tasks, previousShards, currentShards)); - } - if (prevGtidsPerShard != null && !hasSameShards(prevGtidsPerShard.keySet(), currentShards)) { - LOGGER.warn("Some shards for the previous generation {} are not persisted. Expected shards: {}", - prevGtidsPerShard.keySet(), currentShards); - if (prevGtidsPerShard.keySet().containsAll(currentShards)) { - throw new IllegalArgumentException(String.format("Previous shards: %s is the superset of current shards: %s. " - + "We will lose gtid positions for some shards if we continue", - prevGtidsPerShard.keySet(), currentShards)); - } - } - final String keyspace = connectorConfig.getKeyspace(); - // Check the task offsets for the current gen, the offset might not be persisted if this gen just turned on - Map gtidsPerShard = getGtidPerShardFromStorage(tasks, gen, false); - if (gtidsPerShard != null && !hasSameShards(gtidsPerShard.keySet(), currentShards)) { - LOGGER.warn("Some shards for the current generation {} are not persisted. Expected shards: {}", - gtidsPerShard.keySet(), currentShards); - if (!currentShards.containsAll(gtidsPerShard.keySet())) { - LOGGER.warn("Shards from persisted offset: {} not contained within current db shards: {}", - gtidsPerShard.keySet(), currentShards); - // gtidsPerShard has shards not present in the current db shards, we have to rely on the shards - // from gtidsPerShard and we have to require all task offsets are persisted. - gtidsPerShard = getGtidPerShardFromStorage(tasks, gen, true); - } - } - // Use the shards from task offsets persisted in the offset storage if it's not empty. - // The shards from offset storage might be different than the current db shards, this can happen when - // debezium was offline and there was a shard split happened during that time. - // In this case we want to use the old shards from the saved offset storage. Those old shards will - // eventually be replaced with new shards as binlog stream processing handles the shard split event - // and new shards will be persisted in offset storage after the shard split event. - List shards = null; - if (gtidsPerShard != null && gtidsPerShard.size() == 0) { - // if there is no offset persisted for current gen, look for previous gen - if (prevGtidsPerShard != null && prevGtidsPerShard.size() != 0 && !currentShards.containsAll(prevGtidsPerShard.keySet())) { - LOGGER.info("Using shards from persisted offset from prev gen: {}", prevGtidsPerShard.keySet()); - shards = new ArrayList<>(prevGtidsPerShard.keySet()); - } - else { - LOGGER.warn("No persisted offset for current or previous gen, using current shards from db: {}", currentShards); - shards = currentShards; + + VitessOffsetRetriever previousGen = new VitessOffsetRetriever( + connectorConfig, prevNumTasks, gen - 1, true, context().offsetStorageReader()); + + Map prevGtidsPerShard = previousGen.getGtidPerShard(); + validateNoLostShardData(prevGtidsPerShard, currentShards, "gtid positions"); + validateGeneration(prevGtidsPerShard, gen, tasks, prevNumTasks, currentShards); + + VitessOffsetRetriever currentGen = new VitessOffsetRetriever( + connectorConfig, tasks, gen, false, context().offsetStorageReader()); + + Map gtidsPerShard = currentGen.getGtidPerShard(); + validateCurrentGen(currentGen, gtidsPerShard, currentShards, VitessOffsetRetriever.ValueType.GTID); + List shards = determineShards(prevGtidsPerShard, gtidsPerShard, currentShards); + + if (VitessOffsetRetriever.isShardEpochMapEnabled(connectorConfig)) { + Map prevEpochsPerShard = previousGen.getEpochPerShard(); + validateNoLostShardData(prevEpochsPerShard, currentShards, "epochs"); + Map epochsPerShard = currentGen.getEpochPerShard(); + validateCurrentGen(currentGen, epochsPerShard, currentShards, VitessOffsetRetriever.ValueType.EPOCH); + List shardsFromEpoch = determineShards(prevEpochsPerShard, epochsPerShard, currentShards); + if (!shardsFromEpoch.equals(shards)) { + throw new IllegalArgumentException(String.format( + "Shards from gtids %s & shards from epoch must be the same %s", + shards, shardsFromEpoch)); + } } - else if (gtidsPerShard != null && !currentShards.containsAll(gtidsPerShard.keySet())) { - LOGGER.info("Persisted offset has different shards, Using shards from persisted offset: {}", gtidsPerShard.keySet()); - shards = new ArrayList<>(gtidsPerShard.keySet()); - } - else { - // In this case, we prefer the current db shards since gtidsPerShard might only be partially persisted - LOGGER.warn("Current db shards is the superset of persisted offset, using current shards from db: {}", currentShards); - shards = currentShards; - } + // Read GTIDs from config for initial run, only fallback to using this if no stored previous GTIDs, no current GTIDs shards.sort(Comparator.naturalOrder()); Map> shardsPerTask = new HashMap<>(); @@ -240,6 +153,87 @@ else if (gtidsPerShard != null && !currentShards.containsAll(gtidsPerShard.keySe } } + private void validateNoLostShardData(Map prevShardToValues, List currentShards, String valueName) { + if (prevShardToValues != null && !hasSameShards(prevShardToValues.keySet(), currentShards)) { + LOGGER.warn("Some shards for the previous generation {} are not persisted. Expected shards: {}", + prevShardToValues.keySet(), currentShards); + if (prevShardToValues.keySet().containsAll(currentShards)) { + throw new IllegalArgumentException(String.format("Previous shards: %s is the superset of current shards: %s. " + + "We will lose %s for some shards if we continue", + prevShardToValues.keySet(), currentShards, valueName)); + } + } + } + + private static void validateGeneration(Map prevGtidsPerShard, int gen, int tasks, int prevNumTasks, List currentShards) { + LOGGER.info("Previous gtids Per shard: {}", prevGtidsPerShard); + Set previousShards = prevGtidsPerShard != null ? prevGtidsPerShard.keySet() : null; + if (gen > 0 && tasks == prevNumTasks && hasSameShards(previousShards, currentShards)) { + throw new IllegalArgumentException(String.format( + "Previous num.tasks: %s and current num.tasks: %s are the same. " + + "And previous shards: %s and current shards: %s are the same. " + + "Please choose different tasks.max or have different number of vitess shards " + + "if you want to change the task parallelism. " + + "Otherwise please reset the offset.storage.task.key.gen config to its original value", + prevNumTasks, tasks, previousShards, currentShards)); + } + } + + private Map validateCurrentGen(VitessOffsetRetriever retriever, Map valuePerShard, List currentShards, + VitessOffsetRetriever.ValueType valueType) { + if (valuePerShard != null && !hasSameShards(valuePerShard.keySet(), currentShards)) { + LOGGER.warn("Some shards {} for the current generation {} are not persisted. Expected shards: {}", + valueType.name(), valuePerShard.keySet(), currentShards); + if (!currentShards.containsAll(valuePerShard.keySet())) { + LOGGER.warn("Shards {} from persisted offset: {} not contained within current db shards: {}", + valueType.name(), valuePerShard.keySet(), currentShards); + // valuePerShard has shards not present in the current db shards, we have to rely on the shards + // from valuePerShard and we have to require all task offsets are persisted. + retriever.setExpectsOffset(true); + valuePerShard = retriever.getValuePerShardFromStorage(valueType); + } + } + return valuePerShard; + } + + /** + * Use the shards from task offsets persisted in the offset storage if it's not empty. + * The shards from offset storage might be different than the current db shards. This can happen when + * Debezium was offline, and a shard split occurred during that time. + * In this case, we want to use the old shards from the saved offset storage. Those old shards will + * eventually be replaced with new shards as binlog stream processing handles the shard split event, + * and new shards will be persisted in offset storage after the shard split event. + * + * @param gtidsPerShard GTIDs per shard. + * @param prevGtidsPerShard Previous GTIDs per shard. + * @param currentShards Current shards. + * @return List of shards to use. + */ + private List determineShards(Map prevGtidsPerShard, Map gtidsPerShard, List currentShards) { + List shards; + if (gtidsPerShard != null && gtidsPerShard.size() == 0) { + // if there is no offset persisted for current gen, look for previous gen + if (prevGtidsPerShard != null && prevGtidsPerShard.size() != 0 && !currentShards.containsAll(prevGtidsPerShard.keySet())) { + LOGGER.info("Using shards from persisted offset from prev gen: {}", prevGtidsPerShard.keySet()); + shards = new ArrayList<>(prevGtidsPerShard.keySet()); + } + else { + LOGGER.warn("No persisted offset for current or previous gen, using current shards from db: {}", currentShards); + shards = currentShards; + } + } + else if (gtidsPerShard != null && !currentShards.containsAll(gtidsPerShard.keySet())) { + LOGGER.info("Persisted offset has different shards, Using shards from persisted offset: {}", gtidsPerShard.keySet()); + shards = new ArrayList<>(gtidsPerShard.keySet()); + } + else { + // In this case, we prefer the current db shards since gtidsPerShard might only be partially persisted + LOGGER.warn("Current db shards is the superset of persisted offset, using current shards from db: {}", currentShards); + shards = currentShards; + } + return shards; + } + protected static final String getTaskKeyName(int tid, int numTasks, int gen) { return String.format("task%d_%d_%d", tid, numTasks, gen); } diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java index d4fc8703..8da599cf 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorConfig.java @@ -31,6 +31,7 @@ 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.ShardEpochMap; import io.debezium.heartbeat.Heartbeat; import io.debezium.heartbeat.HeartbeatConnectionProvider; import io.debezium.heartbeat.HeartbeatErrorHandler; @@ -248,6 +249,17 @@ public static BigIntUnsignedHandlingMode parse(String value, String defaultValue + " If not configured, the connector streams changes from the latest position for the given shard(s)." + " If snapshot.mode is INITIAL (default), the connector starts copying the tables for the given shard(s) first regardless of gtid value."); + public static final Field SHARD_EPOCH_MAP = Field.create(VITESS_CONFIG_GROUP_PREFIX + "shard.epoch.map") + .withDisplayName("shard.epoch.map") + .withType(Type.STRING) + .withWidth(Width.LONG) + .withDefault("") + .withImportance(ConfigDef.Importance.LOW) + .withValidation(VitessConnectorConfig::validateShardEpochMap) + .withDescription( + "ShardEpochMap to use for the initial epoch values for the given shards. If not configured the connector streams changes" + + "from a default value of 0."); + public static final Field GTID = Field.create(VITESS_CONFIG_GROUP_PREFIX + "gtid") .withDisplayName("gtid") .withType(Type.STRING) @@ -492,11 +504,16 @@ private static int validateTimePrecisionMode(Configuration config, Field field, // VitessConnector will populate the value of this param and pass on to VitessConnectorTask protected static final String VITESS_TASK_SHARDS_CONFIG = "vitess.task.shards"; + // The vitess.task.shard.to.epoch config, the value is a JSON map with vitess shard names mapping + // to epoch values. The vitess connector will populate the value of this param and pass on to + // VitessConnectorTask when the TransactionContextFactory is set to VitessOrderedTransactionContext + public static final String VITESS_TASK_SHARD_EPOCH_MAP_CONFIG = "vitess.task.shard.epoch.map"; + // The vgtid assigned to the given task in the json format, this is the same format as we would see // in the Kafka offset storage. // e.g. [{\"keyspace\":\"ks\",\"shard\":\"-80\",\"gtid\":\"MySQL56/0001:1-114\"}, // {\"keyspace\":\"ks\",\"shard\":\"80-\",\"gtid\":\"MySQL56/0002:1-122\"}] - protected static final String VITESS_TASK_VGTID_CONFIG = "vitess.task.vgtid"; + public static final String VITESS_TASK_VGTID_CONFIG = "vitess.task.vgtid"; /** * The set of {@link Field}s defined as part of this configuration. @@ -553,6 +570,10 @@ public String getVgtid() { return (value != null && !VGTID.defaultValueAsString().equals(value)) ? value : Vgtid.CURRENT_GTID; } + public String getShardEpochMap() { + return getConfig().getString(SHARD_EPOCH_MAP); + } + public boolean excludeEmptyShards() { return getConfig().getBoolean(EXCLUDE_EMPTY_SHARDS); } @@ -585,6 +606,22 @@ private static int validateVgtids(Configuration config, Field field, ValidationO return 0; } + private static int validateShardEpochMap(Configuration config, Field field, ValidationOutput problems) { + // Get the GTID as a string so that the default value is used if GTID is not set + String shardEpochMapString = config.getString(field); + if (shardEpochMapString.isEmpty()) { + return 0; + } + try { + ShardEpochMap shardEpochMap = ShardEpochMap.of(shardEpochMapString); + } + catch (IllegalStateException e) { + problems.accept(field, shardEpochMapString, "Shard epoch map string improperly formatted"); + return 1; + } + return 0; + } + public String getVtgateHost() { return getConfig().getString(VTGATE_HOST); } @@ -667,6 +704,10 @@ public List getVitessTaskKeyShards() { return getConfig().getStrings(VITESS_TASK_SHARDS_CONFIG, CSV_DELIMITER); } + public ShardEpochMap getVitessTaskShardEpochMap() { + return ShardEpochMap.of(getConfig().getString(VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)); + } + public Vgtid getVitessTaskVgtid() { String vgtidStr = getConfig().getString(VITESS_TASK_VGTID_CONFIG); return vgtidStr == null ? null : Vgtid.of(vgtidStr); diff --git a/src/main/java/io/debezium/connector/vitess/VitessConnectorTask.java b/src/main/java/io/debezium/connector/vitess/VitessConnectorTask.java index 2b7004d6..59475cfc 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessConnectorTask.java +++ b/src/main/java/io/debezium/connector/vitess/VitessConnectorTask.java @@ -9,6 +9,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -27,6 +28,7 @@ import io.debezium.connector.vitess.connection.ReplicationConnection; import io.debezium.connector.vitess.connection.VitessReplicationConnection; import io.debezium.connector.vitess.metrics.VitessChangeEventSourceMetricsFactory; +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; import io.debezium.pipeline.ChangeEventSourceCoordinator; import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.ErrorHandler; @@ -148,52 +150,104 @@ protected ChangeEventSourceCoordinator sta } @VisibleForTesting - protected Configuration getConfigWithOffsets(Configuration config) { + public Configuration getConfigWithOffsets(Configuration config) { VitessConnectorConfig connectorConfig = new VitessConnectorConfig(config); if (connectorConfig.offsetStoragePerTask()) { - int gen = connectorConfig.getOffsetStorageTaskKeyGen(); - int prevGen = gen - 1; - Map prevGtidsPerShard = VitessConnector.getGtidPerShardFromStorage( - context.offsetStorageReader(), - connectorConfig, - connectorConfig.getPrevNumTasks(), - prevGen, - true); - LOGGER.info("prevGtidsPerShard {}", prevGtidsPerShard); - Map gtidsPerShard = VitessConnector.getGtidPerShardFromStorage( - context.offsetStorageReader(), - connectorConfig, - connectorConfig.getVitessTotalTasksConfig(), - gen, - false); - LOGGER.info("gtidsPerShard {}", gtidsPerShard); - List shards = connectorConfig.getVitessTaskKeyShards(); - Map configGtidsPerShard = getConfigGtidsPerShard(connectorConfig, shards); - LOGGER.info("configGtidsPerShard {}", configGtidsPerShard); - final String keyspace = connectorConfig.getKeyspace(); - - List shardGtids = new ArrayList<>(); - for (String shard : shards) { - String gtidStr; - if (gtidsPerShard.containsKey(shard)) { - gtidStr = gtidsPerShard.get(shard); - LOGGER.info("Using offsets from current gen: shard {}, gen {}, gtid {}", shard, gen, gtidStr); - } - else if (prevGtidsPerShard.containsKey(shard)) { - gtidStr = prevGtidsPerShard.get(shard); - LOGGER.warn("Using offsets from previous gen: shard {}, gen {}, gtid {}", shard, gen, gtidStr); - } - else { - gtidStr = configGtidsPerShard.getOrDefault(shard, null); - LOGGER.warn("Using offsets from config: shard {}, gtid {}", shard, gtidStr); - } - shardGtids.add(new Vgtid.ShardGtid(keyspace, shard, gtidStr)); + config = getVitessTaskValuePerShard(config, connectorConfig, VitessOffsetRetriever.ValueType.GTID); + if (VitessOffsetRetriever.isShardEpochMapEnabled(connectorConfig)) { + config = getVitessTaskValuePerShard(config, connectorConfig, VitessOffsetRetriever.ValueType.EPOCH); + } + } + return config; + } + + private Configuration getVitessTaskValuePerShard(Configuration config, VitessConnectorConfig connectorConfig, VitessOffsetRetriever.ValueType valueType) { + int gen = connectorConfig.getOffsetStorageTaskKeyGen(); + int prevGen = gen - 1; + VitessOffsetRetriever prevGenRetriever = new VitessOffsetRetriever( + connectorConfig, + connectorConfig.getPrevNumTasks(), + prevGen, + true, + context.offsetStorageReader()); + Map prevGenValuesPerShard = prevGenRetriever.getValuePerShardFromStorage(valueType); + LOGGER.info("{} per shard: {}", valueType.name(), prevGenValuesPerShard); + VitessOffsetRetriever retriever = new VitessOffsetRetriever( + connectorConfig, + connectorConfig.getVitessTotalTasksConfig(), + gen, + false, + context.offsetStorageReader()); + Map curGenValuesPerShard = retriever.getValuePerShardFromStorage(valueType); + LOGGER.info("{} per shard {}", valueType.name(), curGenValuesPerShard); + List shards = connectorConfig.getVitessTaskKeyShards(); + Map configValuesPerShard = null; + switch (valueType) { + case GTID: + configValuesPerShard = getConfigGtidsPerShard(connectorConfig, shards); + break; + case EPOCH: + configValuesPerShard = getConfigShardEpochMapPerShard(connectorConfig, shards); + break; + } + LOGGER.info("config {} per shard {}", valueType.name(), configValuesPerShard); + final String keyspace = connectorConfig.getKeyspace(); + + Map valuesPerShard = new TreeMap(); + for (String shard : shards) { + Object value; + if (curGenValuesPerShard != null && curGenValuesPerShard.containsKey(shard)) { + value = curGenValuesPerShard.get(shard); + LOGGER.info("Using offsets from current gen: shard {}, gen {}, {} {}", shard, gen, valueType.name(), value); + } + else if (prevGenValuesPerShard != null && prevGenValuesPerShard.containsKey(shard)) { + value = prevGenValuesPerShard.get(shard); + LOGGER.warn("Using offsets from previous gen: shard {}, gen {}, {} {}", shard, gen, valueType.name(), value); + } + else { + value = configValuesPerShard.getOrDefault(shard, null); + LOGGER.warn("Using offsets from config: shard {}, {} {}", shard, valueType.name(), valueType.name(), value); } - return config.edit().with(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG, Vgtid.of(shardGtids)).build(); + valuesPerShard.put(shard, value); + } + switch (valueType) { + case GTID: + Map gtidsPerShard = (Map) valuesPerShard; + Vgtid vgtid = getVgtid(gtidsPerShard, keyspace); + config = config.edit().with(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG, vgtid).build(); + break; + case EPOCH: + Map epochsPerShard = (Map) valuesPerShard; + ShardEpochMap shardEpochMap = getShardEpochMap(epochsPerShard); + config = config.edit().with(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG, shardEpochMap).build(); + break; + } + return config; + } + + private ShardEpochMap getShardEpochMap(Map epochMap) { + return new ShardEpochMap(epochMap); + } + + private Vgtid getVgtid(Map gtidsPerShard, String keyspace) { + List shardGtids = new ArrayList(); + for (Map.Entry entry : gtidsPerShard.entrySet()) { + shardGtids.add(new Vgtid.ShardGtid(keyspace, entry.getKey(), entry.getValue())); + } + return Vgtid.of(shardGtids); + } + + private static Map getConfigShardEpochMapPerShard(VitessConnectorConfig connectorConfig, List shards) { + String shardEpochMapString = connectorConfig.getShardEpochMap(); + Function initEpoch = x -> 0L; + Map shardEpochMap; + if (shardEpochMapString.isEmpty()) { + shardEpochMap = buildMap(shards, initEpoch); } else { - return config; + shardEpochMap = ShardEpochMap.of(shardEpochMapString).getMap(); } + return shardEpochMap; } private static Map getConfigGtidsPerShard(VitessConnectorConfig connectorConfig, List shards) { @@ -220,7 +274,7 @@ else if (shards != null) { return configGtidsPerShard; } - private static Map buildMap(List keys, Function function) { + private static Map buildMap(List keys, Function function) { return IntStream.range(0, keys.size()) .boxed() .collect(Collectors.toMap(keys::get, function)); diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java index 0ebce59d..ea134013 100644 --- a/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetContext.java @@ -45,7 +45,15 @@ public VitessOffsetContext( this.transactionContext = transactionContext; } - /** Initialize VitessOffsetContext if no previous offset exists */ + /** + * Initialize VitessOffsetContext if no previous offset exists. This happens if either + * 1. This is a new connector + * 2. The generation has changed + * + * @param connectorConfig + * @param clock + * @return + */ public static VitessOffsetContext initialContext( VitessConnectorConfig connectorConfig, Clock clock) { LOGGER.info("No previous offset exists. Use default VGTID."); @@ -144,6 +152,14 @@ public Loader(VitessConnectorConfig connectorConfig) { this.connectorConfig = connectorConfig; } + /** + * Loads the previously stored offsets for vgtid & transaction context (e.g., epoch). + * If offset storage per task mode is enabled, this is called only if there is no generation change. + * If offset storage per task mode is disabled, this is called only if previous offsets exist. + * + * @param offset + * @return + */ @Override public VitessOffsetContext load(Map offset) { LOGGER.info("Previous offset exists, load from {}", offset); diff --git a/src/main/java/io/debezium/connector/vitess/VitessOffsetRetriever.java b/src/main/java/io/debezium/connector/vitess/VitessOffsetRetriever.java new file mode 100644 index 00000000..c5afb34d --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/VitessOffsetRetriever.java @@ -0,0 +1,128 @@ +/* + * 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 java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.function.Function; + +import org.apache.kafka.connect.storage.OffsetStorageReader; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; + +/** + * Retrieves values from offsets, specifically for retrieving the previous VGTID and shard epoch + * values. + */ +public class VitessOffsetRetriever { + + private static final Logger LOGGER = LoggerFactory.getLogger(VitessConnector.class); + + private final int numTasks; + private final int gen; + private boolean expectsOffset; + private final VitessConnectorConfig config; + private final OffsetStorageReader reader; + + public VitessOffsetRetriever(VitessConnectorConfig config, int numTasks, int gen, boolean expectsOffset, OffsetStorageReader reader) { + this.config = config; + this.numTasks = numTasks; + this.gen = gen; + this.expectsOffset = expectsOffset; + this.reader = reader; + } + + public static boolean isShardEpochMapEnabled(VitessConnectorConfig config) { + return config.getTransactionMetadataFactory() instanceof VitessOrderedTransactionMetadataFactory; + } + + public void setExpectsOffset(boolean expectsOffset) { + this.expectsOffset = expectsOffset; + } + + public enum ValueType { + GTID(SourceInfo.VGTID_KEY, ValueType::parseGtid), + EPOCH(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, ValueType::parseEpoch); + + private final String typeName; + private final Function> parserFunction; + + ValueType(String typeName, Function> parserFunction) { + this.typeName = typeName; + this.parserFunction = parserFunction; + } + + private static Map parseGtid(String vgtidStr) { + Map shardToGtid = new HashMap<>(); + List shardGtids = Vgtid.of(vgtidStr).getShardGtids(); + for (Vgtid.ShardGtid shardGtid : shardGtids) { + shardToGtid.put(shardGtid.getShard(), shardGtid.getGtid()); + } + return shardToGtid; + } + + private static Map parseEpoch(String epochString) { + ShardEpochMap shardToEpoch = ShardEpochMap.of(epochString); + return (Map) shardToEpoch.getMap(); + } + } + + public Map getGtidPerShard() { + return (Map) getValuePerShardFromStorage(ValueType.GTID); + } + + public Map getEpochPerShard() { + return (Map) getValuePerShardFromStorage(ValueType.EPOCH); + } + + public Map getValuePerShardFromStorage(ValueType valueType) { + String key = valueType.typeName; + Function> valueReader = valueType.parserFunction; + return getValuePerShardFromStorage( + key, + valueReader); + } + + public Map getValuePerShardFromStorage(String key, Function> valueReader) { + if (gen < 0) { + return null; + } + final Map valuesPerShard = new HashMap<>(); + for (int i = 0; i < numTasks; i++) { + String taskKey = VitessConnector.getTaskKeyName(i, numTasks, gen); + VitessPartition par = new VitessPartition(config.getLogicalName(), taskKey); + Map offset = reader.offset(par.getSourcePartition()); + if (offset == null && gen == 0) { + LOGGER.info("No previous offset for partition: {}, fall back to only server key", par); + par = new VitessPartition(config.getLogicalName(), null); + offset = reader.offset(par.getSourcePartition()); + } + if (offset == null) { + if (expectsOffset) { + throw new IllegalArgumentException(String.format("No offset found for %s", par)); + } + else { + LOGGER.warn("No offset found for task key: {}", taskKey); + continue; + } + } + final String stringValue = (String) offset.get(key); + Objects.requireNonNull(stringValue, String.format("Missing %s from %s", key, offset)); + Map shardToValue = valueReader.apply(stringValue); + valuesPerShard.putAll(shardToValue); + + } + return valuesPerShard; + } + +} diff --git a/src/main/java/io/debezium/connector/vitess/connection/VitessReplicationConnection.java b/src/main/java/io/debezium/connector/vitess/connection/VitessReplicationConnection.java index ee6ba4d0..9bffb4d9 100644 --- a/src/main/java/io/debezium/connector/vitess/connection/VitessReplicationConnection.java +++ b/src/main/java/io/debezium/connector/vitess/connection/VitessReplicationConnection.java @@ -24,6 +24,7 @@ import io.debezium.connector.vitess.VitessConnectorConfig; import io.debezium.connector.vitess.VitessDatabaseSchema; import io.debezium.connector.vitess.VitessMetadata; +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; import io.debezium.util.Strings; import io.grpc.ManagedChannel; import io.grpc.ManagedChannelBuilder; @@ -401,7 +402,14 @@ public static Vgtid buildVgtid(String keyspace, List shards, List shards = config.getShard(); String vgtidString = config.getVgtid(); List gtids; @@ -434,6 +445,7 @@ public static Vgtid defaultVgtid(VitessConnectorConfig config) { vgtid = buildVgtid(config.getKeyspace(), shards, gtids); } else { + // If it's not current or empty, then it must be an actual vgtid vgtid = Vgtid.of(vgtidString); } LOGGER.info("VGTID '{}' is set to the GTID {} for keyspace: {} shard: {}", @@ -443,6 +455,38 @@ public static Vgtid defaultVgtid(VitessConnectorConfig config) { return vgtid; } + /** + * Get latest shard epoch map. If offset storage mode is enabled, then read the epoch that was set + * by {@link io.debezium.connector.vitess.VitessConnectorTask#getConfigWithOffsets} for the task shard epoch map + * property. If not then read it from the configs or get the shards from Vitess to initialize. + * + * @param config + * @return + */ + public static ShardEpochMap defaultShardEpochMap(VitessConnectorConfig config) { + ShardEpochMap shardEpochMap; + if (config.offsetStoragePerTask()) { + // The epoch values are read or initialized to zero in VitessConnectorTask + shardEpochMap = config.getVitessTaskShardEpochMap(); + LOGGER.info("ShardEpochMap '{}' is set for the keyspace: {}", + shardEpochMap, config.getKeyspace()); + } + else { + if (!config.getShardEpochMap().isEmpty()) { + shardEpochMap = ShardEpochMap.of(config.getShardEpochMap()); + } + else if (config.getShard() == null || config.getShard().isEmpty()) { + List shards = new VitessMetadata(config).getShards(); + shardEpochMap = ShardEpochMap.init(shards); + } + else { + List shards = config.getShard(); + shardEpochMap = ShardEpochMap.init(shards); + } + } + return shardEpochMap; + } + public String connectionString() { return String.format("vtgate gRPC connection %s:%s", config.getVtgateHost(), config.getVtgatePort()); } diff --git a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMap.java b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMap.java new file mode 100644 index 00000000..c6f14d37 --- /dev/null +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMap.java @@ -0,0 +1,89 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Objects; +import java.util.TreeMap; +import java.util.stream.Collectors; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +/** + * Tracks the mapping of shards to epoch values. Used for serializing/deserializing to JSON and to update + * epochs per shard. + */ +public class ShardEpochMap { + + private static final ObjectMapper MAPPER = new ObjectMapper(); + private final Map shardEpochMap; + + public ShardEpochMap() { + this.shardEpochMap = new TreeMap<>(); + } + + public ShardEpochMap(Map shardToEpoch) { + this.shardEpochMap = new TreeMap<>(shardToEpoch); + } + + public static ShardEpochMap init(List shards) { + Map map = shards.stream().collect(Collectors.toMap(s -> s, s -> 0L)); + return new ShardEpochMap(map); + } + + public static ShardEpochMap of(String shardToEpochJson) { + try { + return new ShardEpochMap(MAPPER.readValue(shardToEpochJson, new TypeReference<>() { + })); + } + catch (JsonProcessingException e) { + throw new IllegalStateException("Cannot read shard epoch map", e); + } + } + + @Override + public String toString() { + try { + return MAPPER.writeValueAsString(shardEpochMap); + } + catch (JsonProcessingException e) { + throw new IllegalStateException("Cannot convert shard epoch map to string", e); + } + } + + public Long get(String shard) { + return shardEpochMap.get(shard); + } + + public Long put(String shard, Long epoch) { + return shardEpochMap.put(shard, epoch); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShardEpochMap shardToEpoch = (ShardEpochMap) o; + return Objects.equals(this.shardEpochMap, shardToEpoch.shardEpochMap); + } + + @Override + public int hashCode() { + return Objects.hash(shardEpochMap); + } + + public Map getMap() { + return new TreeMap<>(shardEpochMap); + } +} 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 837b9e92..12c0de20 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 @@ -5,33 +5,43 @@ */ 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.DebeziumException; import io.debezium.connector.vitess.Vgtid; +import io.debezium.connector.vitess.VitessConnectorConfig; +import io.debezium.connector.vitess.VitessConnectorTask; +import io.debezium.connector.vitess.connection.VitessReplicationConnection; +import io.debezium.util.Strings; public class VitessEpochProvider { private static final Logger LOGGER = LoggerFactory.getLogger(VitessEpochProvider.class); - private Map shardToEpoch = new HashMap<>(); - private static final ObjectMapper MAPPER = new ObjectMapper(); + private ShardEpochMap shardEpochMap; + private boolean isFirstTransaction = true; + + public VitessEpochProvider() { + shardEpochMap = new ShardEpochMap(); + } + + public VitessEpochProvider(ShardEpochMap shardToEpoch) { + this.shardEpochMap = shardToEpoch; + } 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)) { + public static Long getEpochForGtid(Long previousEpoch, String previousGtidString, String gtidString, boolean isFirstTransaction) { + if (isFirstTransaction && isInvalidGtid(previousGtidString)) { return previousEpoch + 1; } + else if (isInvalidGtid(previousGtidString)) { + throw new DebeziumException("Invalid GTID: The previous GTID cannot be one of current or empty after the first transaction " + gtidString); + } if (isInvalidGtid(gtidString)) { throw new DebeziumException("Invalid GTID: The current GTID cannot be one of current or empty " + gtidString); } @@ -51,52 +61,81 @@ else if (gtid.isHostSetSubsetOf(previousGtid)) { } } + public ShardEpochMap getShardEpochMap() { + return shardEpochMap; + } + + /** + * Initialize the VitessEpochProvider. Called if either: + * 1. Change in offset storage generation (task number change or vitess shard set change): Read from the config that is set to be the + * shard epoch map derived from previous generation and other info in {@link VitessConnectorTask} + * 2. Newly created connector: Set all shards equal to 0 to initialize shardToEpoch map + * + * @param config VitessConnectorConfig to use for initialization + * @return VitessEpochProvider + */ + public static VitessEpochProvider initialize(VitessConnectorConfig config) { + ShardEpochMap shardEpochMap = VitessReplicationConnection.defaultShardEpochMap(config); + return new VitessEpochProvider(shardEpochMap); + } + public Map store(Map offset) { - try { - offset.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, MAPPER.writeValueAsString(shardToEpoch)); - return offset; - } - catch (JsonProcessingException e) { - throw new RuntimeException("Cannot store epoch: " + shardToEpoch.toString()); - } + offset.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, shardEpochMap.toString()); + return offset; } + /** + * Load the shard epoch map from offsets. If we enabled ordered transaction metadata for the first time, + * then there will be no offsets so use default empty map + * + * @param offsets Offsets to load + */ public void load(Map offsets) { - try { - String shardToEpochString = (String) offsets.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); - if (shardToEpochString != null) { - shardToEpoch = MAPPER.readValue(shardToEpochString, new TypeReference>() { - }); - } - } - catch (JsonProcessingException e) { - throw new RuntimeException("Cannot read shardToEpoch from offsets: " + offsets); + String shardToEpochString = (String) offsets.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); + if (!Strings.isNullOrEmpty(shardToEpochString)) { + shardEpochMap = ShardEpochMap.of(shardToEpochString); } } public Long getEpoch(String shard, String previousVgtidString, String vgtidString) { if (previousVgtidString == null) { - if (shardToEpoch.get(shard) != null) { - throw new DebeziumException("Previous VGTID is null but shardToEpoch map is not null: " + shardToEpoch.toString() + - ", update VGTID in offsets to resume"); - } - // When the connector is first created it has no previous VGTID in offsets (and there is no epoch stored) - long epoch = 0L; - storeEpoch(shard, epoch); - return epoch; + throw new DebeziumException(String.format("Previous vgtid string cannot be null shard %s current %s", shard, vgtidString)); } - 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.getOrDefault(shard, 0L); - long currentEpoch = getEpochForGtid(previousEpoch, previousGtid, gtid); - storeEpoch(shard, currentEpoch); - return currentEpoch; + processVgtid(previousVgtid, vgtid); + if (isFirstTransaction) { + isFirstTransaction = false; + } + return shardEpochMap.get(shard); } - private void storeEpoch(String shard, long epoch) { - shardToEpoch.put(shard, epoch); + private void processVgtid(Vgtid previousVgtid, Vgtid vgtid) { + for (Vgtid.ShardGtid shardGtid : vgtid.getShardGtids()) { + String shard = shardGtid.getShard(); + String gtid = shardGtid.getGtid(); + Vgtid.ShardGtid previousShardGtid = previousVgtid.getShardGtid(shard); + if (previousShardGtid != null) { + String previousGtid = previousShardGtid.getGtid(); + // If there is a previous GTID, then we should have initialized shardEpochMap with the shard + Long previousEpoch = shardEpochMap.get(shard); + if (previousEpoch == null) { + throw new DebeziumException(String.format( + "Previous epoch cannot be null for shard %s when shard present in previous vgtid %s", + shard, previousVgtid)); + } + Long epoch = getEpochForGtid(previousEpoch, previousGtid, gtid, isFirstTransaction); + shardEpochMap.put(shard, epoch); + } + else { + // A re-shard happened while we are streaming set the new value to zero + // TODO: Add support to inherit epoch from ancestor shard + shardEpochMap.put(shard, 0L); + } + } + // Note: we could purge all shards from the shard epoch map that are not present in the current vgtid. + // However, this poses some risk of losing epoch values, so we leave them as is. There may be dormant shards + // that we still have epoch values for, but that should be fine. Once we allow for epochs to be inherited from other shards + // we could reconsider purging them to ensure the epoch shard map does not grow too large. } } 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 149bd04e..17e833eb 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 @@ -10,6 +10,8 @@ import io.debezium.connector.vitess.SourceInfo; import io.debezium.connector.vitess.Vgtid; +import io.debezium.connector.vitess.VitessConnectorConfig; +import io.debezium.connector.vitess.connection.VitessReplicationConnection; import io.debezium.pipeline.txmetadata.TransactionContext; import io.debezium.pipeline.txmetadata.TransactionInfo; @@ -66,6 +68,12 @@ public Map store(Map offset) { return epochProvider.store(offset); } + /** + * Called when there are previous offsets, and we need to create a {@link VitessOrderedTransactionContext} + * from those offsets. + * @param offsets Offsets to load + * @return + */ @Override public TransactionContext newTransactionContextFromOffsets(Map offsets) { return VitessOrderedTransactionContext.load(offsets); @@ -79,6 +87,21 @@ public static VitessOrderedTransactionContext load(Map offsets) { return vitessOrderedTransactionContext; } + /** + * Always called when we need to create a TransactionContext. Returns a {@link VitessOrderedTransactionContext} initialized + * based on the {@link VitessConnectorConfig}. If offsets exist, then it will be followed by a call to + * {@link VitessOrderedTransactionContext#newTransactionContextFromOffsets(Map)} + * from those offsets. + * @param config + * @return {@link VitessOrderedTransactionContext} + */ + public static VitessOrderedTransactionContext initialize(VitessConnectorConfig config) { + VitessOrderedTransactionContext vitessOrderedTransactionContext = new VitessOrderedTransactionContext(); + vitessOrderedTransactionContext.epochProvider = VitessEpochProvider.initialize(config); + vitessOrderedTransactionContext.previousVgtid = VitessReplicationConnection.defaultVgtid(config).toString(); + return vitessOrderedTransactionContext; + } + @Override public void beginTransaction(TransactionInfo transactionInfo) { super.beginTransaction(transactionInfo); 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 index 391a787e..8feb7bcf 100644 --- a/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java +++ b/src/main/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactory.java @@ -7,25 +7,28 @@ package io.debezium.connector.vitess.pipeline.txmetadata; import io.debezium.config.Configuration; +import io.debezium.connector.vitess.VitessConnectorConfig; 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; + private final Configuration configuration; public VitessOrderedTransactionMetadataFactory(Configuration configuration) { - this.configuraiton = configuration; + this.configuration = configuration; } @Override public TransactionContext getTransactionContext() { - return new VitessOrderedTransactionContext(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + VitessOrderedTransactionContext context = VitessOrderedTransactionContext.initialize(connectorConfig); + return context; } @Override public TransactionStructMaker getTransactionStructMaker() { - return new VitessOrderedTransactionStructMaker(configuraiton); + return new VitessOrderedTransactionStructMaker(configuration); } } diff --git a/src/test/java/io/debezium/connector/vitess/TestHelper.java b/src/test/java/io/debezium/connector/vitess/TestHelper.java index 9c2424a0..c8487dce 100644 --- a/src/test/java/io/debezium/connector/vitess/TestHelper.java +++ b/src/test/java/io/debezium/connector/vitess/TestHelper.java @@ -17,6 +17,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.stream.Collectors; @@ -29,6 +30,7 @@ import io.debezium.connector.vitess.connection.ReplicationMessage; import io.debezium.connector.vitess.connection.ReplicationMessageColumn; import io.debezium.connector.vitess.connection.VitessTabletType; +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; import io.debezium.relational.RelationalDatabaseConnectorConfig; import io.debezium.relational.TableId; import io.vitess.proto.Query; @@ -45,6 +47,9 @@ public class TestHelper { public static final String TEST_SHARD = "0"; public static final String TEST_SHARD1 = "-80"; public static final String TEST_SHARD2 = "80-"; + public static final Long TEST_SHARD1_EPOCH = 2L; + public static final Long TEST_SHARD2_EPOCH = 3L; + public static final ShardEpochMap TEST_SHARD_TO_EPOCH = new ShardEpochMap(Map.of(TEST_SHARD1, TEST_SHARD1_EPOCH, TEST_SHARD2, TEST_SHARD2_EPOCH)); public static final String TEST_GTID = "MySQL56/a790d864-9ba1-11ea-99f6-0242ac11000a:1-1513"; public static final String TEST_TABLE = "test_table"; diff --git a/src/test/java/io/debezium/connector/vitess/VgtidTest.java b/src/test/java/io/debezium/connector/vitess/VgtidTest.java index 02a0ea1f..eada2eea 100644 --- a/src/test/java/io/debezium/connector/vitess/VgtidTest.java +++ b/src/test/java/io/debezium/connector/vitess/VgtidTest.java @@ -14,14 +14,12 @@ 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; @@ -415,8 +413,6 @@ public void shouldGetLocalVgtid() { @Test public void shouldGetMissingShardGtidThrowsDebeziumException() { Vgtid vgtid1 = Vgtid.of(VGTID_JSON); - assertThatThrownBy(() -> { - Vgtid.ShardGtid shardGtid = vgtid1.getShardGtid("missing_shard"); - }).isInstanceOf(DebeziumException.class); + assertThat(vgtid1.getShardGtid("missing_shard")).isNull(); } } diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorConfigTest.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorConfigTest.java index 9766506f..744c8053 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorConfigTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorConfigTest.java @@ -6,6 +6,7 @@ package io.debezium.connector.vitess; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD_TO_EPOCH; import static org.assertj.core.api.Assertions.assertThat; import java.util.ArrayList; @@ -80,4 +81,39 @@ public void shouldExcludeEmptyShards() { assertThat(connectorConfig.excludeEmptyShards()).isTrue(); } + @Test + public void shouldGetVitessTaskEpochShardMapConfig() { + Configuration configuration = TestHelper.defaultConfig().with( + VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG, TEST_SHARD_TO_EPOCH.toString()).build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + assertThat(connectorConfig.getVitessTaskShardEpochMap()).isEqualTo(TEST_SHARD_TO_EPOCH); + } + + @Test + public void shouldGetVitessEpochShardMapConfig() { + Configuration configuration = TestHelper.defaultConfig().with( + VitessConnectorConfig.SHARD_EPOCH_MAP, TEST_SHARD_TO_EPOCH.toString()).build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + assertThat(connectorConfig.getShardEpochMap()).isEqualTo(TEST_SHARD_TO_EPOCH.toString()); + } + + @Test + public void shouldGetVitessEpochShardMapConfigDefault() { + Configuration configuration = TestHelper.defaultConfig().build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + assertThat(connectorConfig.getShardEpochMap()).isEqualTo(""); + } + + @Test + public void shouldImproperShardEpochMapFailValidation() { + Configuration configuration = TestHelper.defaultConfig().with(VitessConnectorConfig.SHARD_EPOCH_MAP, "foo").build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + List inputs = new ArrayList<>(); + Consumer printConsumer = (input) -> { + inputs.add(input); + }; + connectorConfig.validateAndRecord(List.of(VitessConnectorConfig.SHARD_EPOCH_MAP), printConsumer); + assertThat(inputs.size()).isEqualTo(1); + } + } diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java index 6d0b379c..33c21b83 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorIT.java @@ -8,7 +8,12 @@ import static io.debezium.connector.vitess.TestHelper.TEST_EMPTY_SHARD_KEYSPACE; import static io.debezium.connector.vitess.TestHelper.TEST_NON_EMPTY_SHARD; import static io.debezium.connector.vitess.TestHelper.TEST_SERVER; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1_EPOCH; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD2; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD2_EPOCH; import static io.debezium.connector.vitess.TestHelper.TEST_SHARDED_KEYSPACE; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD_TO_EPOCH; 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; @@ -51,6 +56,7 @@ import io.debezium.config.Configuration; import io.debezium.config.Field; import io.debezium.connector.vitess.connection.VitessReplicationConnection; +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; import io.debezium.connector.vitess.pipeline.txmetadata.VitessRankProvider; @@ -645,7 +651,9 @@ public void shouldProvideOrderedTransactionMetadata() throws Exception { SourceRecord beginRecord = assertRecordBeginSourceRecord(); assertThat(beginRecord.sourceOffset()).containsKey("transaction_epoch"); String expectedTxId1 = ((Struct) beginRecord.value()).getString("id"); - Long expectedEpoch = 0L; + // A 0 epoch is only used by a connector that starts with a valid gtid in its config. + // For a connector that starts with current (default) or snapshot (empty), increment epoch (in this case from 0 -> 1 + Long expectedEpoch = 1L; 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"); @@ -771,7 +779,9 @@ public void shouldProvideTransactionMetadataWithoutIdOrTransactionTopic() throws // exercise SUT executeAndWait(insertRowsStatement, TEST_SHARDED_KEYSPACE); // First transaction. - Long expectedEpoch = 0L; + // A 0 epoch is only used by a connector that starts with a valid gtid in its config. + // For a connector that starts with current (default) or snapshot (empty), increment epoch (in this case from 0 -> 1 + Long expectedEpoch = 1L; 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"); @@ -836,7 +846,9 @@ public void shouldProvideTransactionMetadataWithoutIdOrTransactionTopicAndUseLoc // exercise SUT executeAndWait(insertRowsStatement, TEST_SHARDED_KEYSPACE); // First transaction. - Long expectedEpoch = 0L; + // A 0 epoch is only used by a connector that starts with a valid gtid in its config. + // For a connector that starts with current (default) or snapshot (empty), increment epoch (in this case from 0 -> 1 + Long expectedEpoch = 1L; 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"); @@ -1215,6 +1227,109 @@ public void shouldMultiShardMultiTaskConfigSubscriptionHaveMultiShardGtidsInVgti assertInsert(INSERT_NUMERIC_TYPES_STMT, schemasAndValuesForNumericTypes(), TEST_SHARDED_KEYSPACE, TestHelper.PK_FIELD, hasMultipleShards); } + @Test + public void shouldMaintainEpochMapWithChangeInOffsetStoragePerTask() throws Exception { + TestHelper.executeDDL("vitess_create_tables.ddl", TEST_SHARDED_KEYSPACE); + TestHelper.applyVSchema("vitess_vschema.json"); + + 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 offsetId = Map.of( + VitessOrderedTransactionContext.OFFSET_TRANSACTION_ID, currentVgtid, + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, TEST_SHARD_TO_EPOCH.toString(), + SourceInfo.VGTID_KEY, currentVgtid); + Map, Map> offsets = Map.of(srcPartition, offsetId); + Configuration config = TestHelper.defaultConfig() + .with(CommonConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) + .with(CommonConnectorConfig.TOPIC_PREFIX, TEST_SERVER) + .with(VitessConnectorConfig.KEYSPACE, TEST_SHARDED_KEYSPACE) + .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); + + // exercise SUT + executeAndWait(INSERT_NUMERIC_TYPES_STMT, TEST_SHARDED_KEYSPACE); + // First transaction. + SourceRecord beginRecord = assertRecordBeginSourceRecord(); + + ShardEpochMap beginShardToEpoch = ShardEpochMap.of((String) beginRecord.sourceOffset().get("transaction_epoch")); + assertThat(beginShardToEpoch.get(TEST_SHARD1)).isEqualTo(TEST_SHARD1_EPOCH + 1); + assertThat(beginShardToEpoch.get(TEST_SHARD2)).isEqualTo(TEST_SHARD2_EPOCH + 1); + String expectedTxId = ((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"); + + ShardEpochMap shardToEpoch = ShardEpochMap.of((String) record.sourceOffset().get("transaction_epoch")); + assertThat(shardToEpoch.get(TEST_SHARD1)).isEqualTo(TEST_SHARD1_EPOCH + 1); + assertThat(shardToEpoch.get(TEST_SHARD2)).isEqualTo(TEST_SHARD2_EPOCH + 1); + + final Struct txn = ((Struct) record.value()).getStruct("transaction"); + Long epoch = (Long) txn.get("transaction_epoch"); + assertThat(epoch).isEqualTo(TEST_SHARD_TO_EPOCH.get(shard) + 1); + + } + assertRecordEnd(expectedTxId, expectedRecordsCount); + + stopConnector(); + + Configuration config2 = TestHelper.defaultConfig() + .with(CommonConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class) + .with(CommonConnectorConfig.TOPIC_PREFIX, TEST_SERVER) + .with(VitessConnectorConfig.KEYSPACE, TEST_SHARDED_KEYSPACE) + .with(CommonConnectorConfig.PROVIDE_TRANSACTION_METADATA, true) + .with(VitessConnectorConfig.SHARD, "-80,80-") + .with(VitessConnectorConfig.TASKS_MAX_CONFIG, 2) + .with(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK, "true") + .with(VitessConnectorConfig.OFFSET_STORAGE_TASK_KEY_GEN, "0") + .with(VitessConnectorConfig.PREV_NUM_TASKS, "1") + .build(); + startConnector(config2); + assertConnectorIsRunning(); + + consumer = testConsumer(expectedRecordsCount + 2); + executeAndWait(INSERT_NUMERIC_TYPES_STMT, TEST_SHARDED_KEYSPACE); + + SourceRecord beginRecord2 = assertRecordBeginSourceRecord(); + assertThat(beginRecord2.sourceOffset()).containsKey("transaction_epoch"); + String expectedTxId2 = ((Struct) beginRecord2.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 = TEST_SHARD_TO_EPOCH.get(shard) + 1; + final Struct txn = ((Struct) record.value()).getStruct("transaction"); + String txId = txn.getString("id"); + assertThat(txId).isNotNull(); + assertThat(txId).isEqualTo(expectedTxId2); + assertThat(txn.get("transaction_epoch")).isEqualTo(expectedEpoch); + BigDecimal expectedRank = VitessRankProvider.getRank(Vgtid.of(expectedTxId2).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(expectedTxId2, expectedRecordsCount); + } + @Test @FixFor("DBZ-2578") public void shouldUseMultiColumnPkAsRecordKey() throws Exception { diff --git a/src/test/java/io/debezium/connector/vitess/VitessConnectorTest.java b/src/test/java/io/debezium/connector/vitess/VitessConnectorTest.java index 563f82d4..4d5b3b6b 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessConnectorTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessConnectorTest.java @@ -9,6 +9,7 @@ import static io.debezium.connector.vitess.TestHelper.TEST_SHARDED_KEYSPACE; import static io.debezium.connector.vitess.TestHelper.TEST_UNSHARDED_KEYSPACE; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -16,12 +17,14 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import java.util.function.Function; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigValue; @@ -40,6 +43,9 @@ import io.debezium.config.Configuration; import io.debezium.connector.common.OffsetReader; import io.debezium.connector.vitess.connection.VitessReplicationConnection; +import io.debezium.connector.vitess.pipeline.txmetadata.ShardEpochMap; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionContext; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; import io.debezium.embedded.KafkaConnectUtil; import io.debezium.junit.logging.LogInterceptor; import io.debezium.pipeline.spi.OffsetContext; @@ -127,6 +133,7 @@ public void testTaskConfigsOffsetStorageModeSingle() { put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); } }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); connector.start(props); List shards = Arrays.asList("-4000", "4000-8000", "8000-c000", "c000-"); List> taskConfigs = connector.taskConfigs(1, shards); @@ -146,6 +153,46 @@ public void testTaskConfigsOffsetStorageModeSingle() { assertEquals("value", firstConfig.get("key")); } + @Test + public void testTaskConfigsOffsetStorageModeSingleWithOrderMetadata() { + VitessConnector connector = new VitessConnector(); + Map props = new HashMap<>() { + { + put("key", "value"); + put(VitessConnectorConfig.KEYSPACE.name(), TEST_UNSHARDED_KEYSPACE); + put(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK.name(), "true"); + put(VitessConnectorConfig.OFFSET_STORAGE_TASK_KEY_GEN.name(), "0"); + put(VitessConnectorConfig.PREV_NUM_TASKS.name(), "1"); + put(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY.name(), VitessOrderedTransactionMetadataFactory.class.getName()); + put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); + } + }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); + connector.start(props); + List shards = Arrays.asList("-4000", "4000-8000", "8000-c000", "c000-"); + List> taskConfigs = connector.taskConfigs(1, shards); + taskConfigs = getConfigWithOffsetsHelper(taskConfigs); + assertThat(taskConfigs.size() == 1); + Map firstConfig = taskConfigs.get(0); + assertThat(firstConfig.size() == 4); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG)).isEqualTo( + VitessConnector.getTaskKeyName(0, 1, 0)); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_SHARDS_CONFIG)).isEqualTo( + String.join(",", shards)); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG)).isEqualTo("task0_1_0"); + + ShardEpochMap epochMap = new ShardEpochMap( + Map.of("-4000", 0L, "4000-8000", 0L, "8000-c000", 0L, "c000-", 0L)); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)).isEqualTo( + epochMap.toString()); + + List gtidStrs = Arrays.asList(Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID, + Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID); + Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); + assertEquals(vgtid.toString(), firstConfig.get(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG)); + assertEquals("value", firstConfig.get("key")); + } + @Test public void testTaskConfigsSingleTaskMultipleShards() { VitessConnector connector = new VitessConnector(); @@ -175,6 +222,35 @@ public void testTaskConfigsSingleTaskMultipleShards() { assertEquals("value", firstConfig.get("key")); } + @Test + public void testTaskConfigsSingleTaskMultipleShardsOrderMetadata() { + VitessConnector connector = new VitessConnector(); + List shards = Arrays.asList("-01", "01-"); + String shardCsv = String.join(",", shards); + Map props = new HashMap<>() { + { + put("key", "value"); + put(VitessConnectorConfig.KEYSPACE.name(), TEST_SHARDED_KEYSPACE); + put(VitessConnectorConfig.SHARD.name(), shardCsv); + put(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY.name(), VitessOrderedTransactionMetadataFactory.class.getName()); + put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); + } + }; + connector.start(props); + List> taskConfigs = connector.taskConfigs(1); + assertThat(taskConfigs.size() == 1); + Map firstConfig = taskConfigs.get(0); + assertThat(firstConfig.size() == 3); + assertEquals(firstConfig.get(VitessConnectorConfig.SHARD.name()), + String.join(",", shards)); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)).isNull(); + Vgtid vgtid = VitessReplicationConnection.defaultVgtid(new VitessConnectorConfig(Configuration.from(firstConfig))); + assertThat(vgtid.getShardGtids()).isEqualTo(Collect.arrayListOf( + new Vgtid.ShardGtid(TEST_SHARDED_KEYSPACE, shards.get(0), "current"), + new Vgtid.ShardGtid(TEST_SHARDED_KEYSPACE, shards.get(1), "current"))); + assertEquals("value", firstConfig.get("key")); + } + @Test public void testTaskConfigsSingleTaskMultipleShardsMultipleTasks() { VitessConnector connector = new VitessConnector(); @@ -191,6 +267,7 @@ public void testTaskConfigsSingleTaskMultipleShardsMultipleTasks() { put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); } }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); connector.start(props); List> taskConfigs = connector.taskConfigs(maxTasks); taskConfigs = getConfigWithOffsetsHelper(taskConfigs); @@ -214,6 +291,57 @@ public void testTaskConfigsSingleTaskMultipleShardsMultipleTasks() { assertEquals("value", firstConfig.get("key")); } + @Test + public void testTaskConfigsSingleTaskMultipleShardsMultipleTasksOrderMetadata() { + VitessConnector connector = new VitessConnector(); + List shards = Arrays.asList("-80", "80-90", "90-"); + String shardCsv = String.join(",", shards); + int maxTasks = 2; + Map props = new HashMap<>() { + { + put("key", "value"); + put(VitessConnectorConfig.KEYSPACE.name(), TEST_SHARDED_KEYSPACE); + put(VitessConnectorConfig.SHARD.name(), shardCsv); + put(VitessConnectorConfig.TASKS_MAX_CONFIG, String.valueOf(maxTasks)); + put(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK.name(), "true"); + put(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY.name(), VitessOrderedTransactionMetadataFactory.class.getName()); + put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); + } + }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); + connector.start(props); + List> taskConfigs = connector.taskConfigs(maxTasks); + taskConfigs = getConfigWithOffsetsHelper(taskConfigs); + int expectedConfigSize = 12; + assertEquals(taskConfigs.size(), maxTasks); + Map firstConfig = taskConfigs.get(0); + assertThat(firstConfig.size()).isEqualTo(expectedConfigSize); + assertEquals(firstConfig.get(VitessConnectorConfig.SHARD.name()), + String.join(",", shards)); + VitessConnectorConfig config = new VitessConnectorConfig(Configuration.from(firstConfig)); + Vgtid vgtid = VitessReplicationConnection.defaultVgtid(config); + assertThat(vgtid.getShardGtids()).isEqualTo(Collect.arrayListOf( + new Vgtid.ShardGtid(TEST_SHARDED_KEYSPACE, shards.get(0), "current"), + new Vgtid.ShardGtid(TEST_SHARDED_KEYSPACE, shards.get(2), "current"))); + + ShardEpochMap expectedShardEpochMap = new ShardEpochMap(Map.of("-80", 0L, "90-", 0L)); + ShardEpochMap actualShardEpochMap = VitessReplicationConnection.defaultShardEpochMap(config); + assertThat(actualShardEpochMap).isEqualTo(expectedShardEpochMap); + + Map secondConfig = taskConfigs.get(1); + assertEquals(secondConfig.size(), expectedConfigSize); + assertEquals(secondConfig.get(VitessConnectorConfig.SHARD.name()), + String.join(",", shards)); + Vgtid secondVgtid = VitessReplicationConnection.defaultVgtid(new VitessConnectorConfig(Configuration.from(secondConfig))); + assertThat(secondVgtid.getShardGtids()).isEqualTo(Collect.arrayListOf( + new Vgtid.ShardGtid(TEST_SHARDED_KEYSPACE, shards.get(1), "current"))); + assertEquals("value", firstConfig.get("key")); + + ShardEpochMap epochMap2 = new ShardEpochMap( + Map.of("80-90", 0L)); + assertThat(secondConfig.get(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)).isEqualTo(epochMap2.toString()); + } + @Test public void testTaskConfigsSingleTaskMultipleShardsMismatchedGtidsMultipleTasks() { VitessConnector connector = new VitessConnector(); @@ -269,7 +397,7 @@ public void testTaskConfigsValidatesDeprecatedConfig() { } @Test - public void testTaskConfigsSingleTaskNoShardsMultipleGtidsMultipleTasks() { + public void testTaskConfigsMultipleTasksNoShardsMultipleGtids() { VitessConnector connector = new VitessConnector(); String vgtid = VgtidTest.VGTID_JSON; int maxTasks = 2; @@ -312,7 +440,7 @@ public void testTaskConfigsSingleTaskNoShardsNoGtidsMultipleTasks() { } @Test - public void testTaskConfigsSingleTaskMultipleShardsMultipleGtidsMultipleTasks() { + public void testTaskConfigsMultipleTasksMultipleShardsMultipleGtids() { VitessConnector connector = new VitessConnector(); List shards = Arrays.asList("-70", "70-80", "80-90", "90-"); List gtids = Arrays.asList(TestHelper.TEST_GTID, Vgtid.CURRENT_GTID, TestHelper.TEST_GTID, Vgtid.CURRENT_GTID); @@ -334,6 +462,7 @@ public void testTaskConfigsSingleTaskMultipleShardsMultipleGtidsMultipleTasks() put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); } }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); connector.start(props); List> taskConfigs = connector.taskConfigs(maxTasks); taskConfigs = getConfigWithOffsetsHelper(taskConfigs); @@ -482,6 +611,46 @@ public void testTaskConfigsOffsetStorageModeDouble() { put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); } }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); + connector.start(props); + List shards = Arrays.asList("-4000", "4000-8000", "8000-c000", "c000-"); + List> taskConfigs = connector.taskConfigs(2, shards); + taskConfigs = getConfigWithOffsetsHelper(taskConfigs); + assertThat(taskConfigs.size() == 2); + Map firstConfig = taskConfigs.get(0); + assertThat(firstConfig.size() == 4); + assertEquals(firstConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG), "task0_2_0"); + assertEquals(firstConfig.get(VitessConnectorConfig.VITESS_TASK_SHARDS_CONFIG), "-4000,8000-c000"); + List gtidStrs = Arrays.asList(Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID); + List shards0 = Arrays.asList("-4000", "8000-c000"); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs); + assertEquals(firstConfig.get(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG), vgtid0.toString()); + assertEquals(firstConfig.get("key"), "value"); + Map secondConfig = taskConfigs.get(1); + assertThat(secondConfig.size() == 4); + assertEquals(secondConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG), "task1_2_0"); + assertEquals(secondConfig.get(VitessConnectorConfig.VITESS_TASK_SHARDS_CONFIG), "4000-8000,c000-"); + List shards1 = Arrays.asList("4000-8000", "c000-"); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs); + assertEquals(secondConfig.get(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG), vgtid1.toString()); + assertEquals(secondConfig.get("key"), "value"); + } + + @Test + public void testTaskConfigsOffsetStorageModeDoubleOrderMetadata() { + VitessConnector connector = new VitessConnector(); + Map props = new HashMap<>() { + { + put("key", "value"); + put(VitessConnectorConfig.KEYSPACE.name(), TEST_UNSHARDED_KEYSPACE); + put(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK.name(), "true"); + put(VitessConnectorConfig.OFFSET_STORAGE_TASK_KEY_GEN.name(), "0"); + put(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY.name(), VitessOrderedTransactionMetadataFactory.class.getName()); + put(VitessConnectorConfig.PREV_NUM_TASKS.name(), "1"); + put(VitessConnectorConfig.SNAPSHOT_MODE.name(), VitessConnectorConfig.SnapshotMode.NEVER.getValue()); + } + }; + connector.initialize(new ContextHelper().getSourceConnectorContext()); connector.start(props); List shards = Arrays.asList("-4000", "4000-8000", "8000-c000", "c000-"); List> taskConfigs = connector.taskConfigs(2, shards); @@ -496,6 +665,11 @@ public void testTaskConfigsOffsetStorageModeDouble() { Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs); assertEquals(firstConfig.get(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG), vgtid0.toString()); assertEquals(firstConfig.get("key"), "value"); + + ShardEpochMap epochMap = new ShardEpochMap( + Map.of("-4000", 0L, "8000-c000", 0L)); + assertThat(firstConfig.get(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)).isEqualTo(epochMap.toString()); + Map secondConfig = taskConfigs.get(1); assertThat(secondConfig.size() == 4); assertEquals(secondConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG), "task1_2_0"); @@ -504,6 +678,10 @@ public void testTaskConfigsOffsetStorageModeDouble() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs); assertEquals(secondConfig.get(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG), vgtid1.toString()); assertEquals(secondConfig.get("key"), "value"); + + ShardEpochMap epochMap2 = new ShardEpochMap( + Map.of("4000-8000", 0L, "c000-", 0L)); + assertThat(secondConfig.get(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG)).isEqualTo(epochMap2.toString()); } @Test @@ -568,6 +746,7 @@ public void testTaskConfigsNegativePrevNumTasks() { @Test public void testTaskConfigsSameNumTasks() { VitessConnector connector = new VitessConnector(); + connector.initialize(new ContextHelper().getSourceConnectorContext()); Map props = new HashMap<>() { { put("key", "value"); @@ -598,9 +777,9 @@ public void testTaskConfigsOffsetMigrationSingle() { final int gen = 1; final int numTasks = 1; try { - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, numTasks, vgtid0.toString(), null); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, numTasks, getVgtidOffset(vgtid0.toString()), null); fail("Should not reach here because prev.num.tasks and num.tasks are the same, vgtids:" - + vgtids); + + offsets); } catch (IllegalArgumentException ex) { // This is expected(); @@ -617,12 +796,48 @@ public void testTaskConfigsOffsetMigrationDouble() { Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); final int gen = 1; final int numTasks = 2; - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, 1, vgtid0.toString(), null); - assertThat(vgtids.size() == numTasks); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, 1, getVgtidOffset(vgtid0.toString()), null); + assertThat(offsets.size() == numTasks); + Map gtidPerShard = new HashMap<>(); + for (int tid = 0; tid < numTasks; tid++) { + String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); + assertThat(gtidStr != null); + Vgtid vgtid = Vgtid.of(gtidStr); + assertThat(vgtid.getShardGtids().size() == 1); + for (int i = 0; i < vgtid.getShardGtids().size(); i++) { + Vgtid.ShardGtid shardGtid = vgtid.getShardGtids().get(i); + gtidPerShard.put(shardGtid.getShard(), shardGtid.getGtid()); + } + } + assertEquals(expectedGtidPerShard, gtidPerShard); + } + + @Test + public void testTaskConfigsOffsetMigrationDoubleFromServerToPerTaskOrderMetadata() { + List shards = Arrays.asList("s0", "s1"); + List gtidStrs = Arrays.asList("gtid0", "gtid1"); + Map expectedGtidPerShard = Collect.hashMapOf("s0", "gtid0", "s1", "gtid1"); + ShardEpochMap expectedEpochPerShard = new ShardEpochMap(Collect.hashMapOf("s0", 3L, "s1", 4L)); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); + final int gen = 1; + final int numTasks = 2; + + Map serverOffsets = Map.of( + SourceInfo.VGTID_KEY, vgtid0.toString(), + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpochPerShard.toString()); + + Function customConfig = (builder) -> builder.with( + VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class); + + // Store offsets for the server, see if we can read those as our previous and use them + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, 1, serverOffsets, null, + customConfig); + assertThat(offsets.size() == numTasks); Map gtidPerShard = new HashMap<>(); for (int tid = 0; tid < numTasks; tid++) { String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); - String gtidStr = vgtids.get(key); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); assertThat(gtidStr != null); Vgtid vgtid = Vgtid.of(gtidStr); assertThat(vgtid.getShardGtids().size() == 1); @@ -634,6 +849,53 @@ public void testTaskConfigsOffsetMigrationDouble() { assertEquals(expectedGtidPerShard, gtidPerShard); } + @Test + public void testTaskConfigsOffsetDoubleSubsetValidation() { + List shards = Arrays.asList("s0", "s1"); + List shardSubset = Arrays.asList("s0"); + List gtidStrs = Arrays.asList("gtid0", "gtid1"); + + ShardEpochMap expectedEpochPerShard = new ShardEpochMap(Collect.hashMapOf("s0", 3L, "s1", 4L)); + + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); + final int gen = 1; + final int numTasks = 2; + + Map serverOffsets = Map.of( + SourceInfo.VGTID_KEY, vgtid0.toString(), + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpochPerShard.toString()); + + assertThatThrownBy(() -> { + getOffsetFromStorage(numTasks, shardSubset, gen, 1, serverOffsets, null); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("We will lose gtid positions for some shards if we continue"); + } + + @Test + public void testTaskConfigsOffsetDoubleSubsetEpochValidation() { + List shardSubset = Arrays.asList("s0"); + ShardEpochMap expectedEpochPerShard = new ShardEpochMap(Collect.hashMapOf("s0", 3L, "s1", 4L)); + + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Collections.emptyList(), Collections.emptyList()); + final int gen = 1; + final int numTasks = 2; + + Map serverOffsets = Map.of( + SourceInfo.VGTID_KEY, vgtid0.toString(), + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpochPerShard.toString()); + + Function customConfig = (builder) -> builder.with( + VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class); + + assertThatThrownBy(() -> { + getOffsetFromStorage(numTasks, shardSubset, gen, 1, serverOffsets, null, + customConfig); + }) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("We will lose epochs for some shards if we continue"); + } + @Test public void testTaskConfigsOffsetRestartDouble() { List shards = Arrays.asList("s0", "s1"); @@ -649,15 +911,65 @@ public void testTaskConfigsOffsetRestartDouble() { List gtidStrs1 = List.of("gtid1"); Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs0); Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs1); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen), vgtid1.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, serverVgtid.toString(), prevVgtids); - assertThat(vgtids.size() == numTasks); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen), getVgtidOffset(vgtid1.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, getVgtidOffset(serverVgtid.toString()), prevVgtids); + assertThat(offsets.size() == numTasks); + Map gtidPerShard = new HashMap<>(); + for (int tid = 0; tid < numTasks; tid++) { + String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); + assertThat(gtidStr != null); + Vgtid vgtid = Vgtid.of(gtidStr); + assertThat(vgtid.getShardGtids().size() == 1); + for (int i = 0; i < vgtid.getShardGtids().size(); i++) { + Vgtid.ShardGtid shardGtid = vgtid.getShardGtids().get(i); + gtidPerShard.put(shardGtid.getShard(), shardGtid.getGtid()); + } + } + Map expectedGtidPerShard = Collect.hashMapOf("s0", "gtid0", "s1", "gtid1"); + assertEquals(expectedGtidPerShard, gtidPerShard); + } + + @Test + public void testTaskConfigsOffsetRestartDoubleOrderMetadata() { + List shards = Arrays.asList("s0", "s1"); + // Note we are not able to fetch old0/old1 since prevGtids takes precedence over serverVgtid + List gtidStrs = Arrays.asList("old0", "old1"); + Vgtid serverVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); + final int gen = 1; + final int numTasks = 2; + final int prevNumTasks = 1; + List shards0 = List.of("s0"); + List shards1 = List.of("s1"); + List gtidStrs0 = List.of("gtid0"); + List gtidStrs1 = List.of("gtid1"); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs0); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs1); + + ShardEpochMap expectedEpochPerShard = new ShardEpochMap(Collect.hashMapOf("s0", 3L, "s1", 4L)); + + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), getVgtidEpochOffset(vgtid0.toString(), expectedEpochPerShard.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen), getVgtidEpochOffset(vgtid1.toString(), expectedEpochPerShard.toString())); + + Function customConfig = (builder) -> builder.with( + VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class); + + Map serverOffsets = Map.of( + SourceInfo.VGTID_KEY, serverVgtid.toString(), + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpochPerShard.toString()); + + Map> offsets = getOffsetFromStorage( + numTasks, shards, gen, prevNumTasks, serverOffsets, taskOffsets, + customConfig); + + assertThat(offsets.size() == numTasks); Map gtidPerShard = new HashMap<>(); for (int tid = 0; tid < numTasks; tid++) { String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); - String gtidStr = vgtids.get(key); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); assertThat(gtidStr != null); Vgtid vgtid = Vgtid.of(gtidStr); assertThat(vgtid.getShardGtids().size() == 1); @@ -665,6 +977,9 @@ public void testTaskConfigsOffsetRestartDouble() { Vgtid.ShardGtid shardGtid = vgtid.getShardGtids().get(i); gtidPerShard.put(shardGtid.getShard(), shardGtid.getGtid()); } + + String epoch = offsets.get(key).get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); + assertThat(epoch).isEqualTo(expectedEpochPerShard.toString()); } Map expectedGtidPerShard = Collect.hashMapOf("s0", "gtid0", "s1", "gtid1"); assertEquals(expectedGtidPerShard, gtidPerShard); @@ -683,14 +998,14 @@ public void testTaskConfigsOffsetRestartDoubleIncomplete() { Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs0); // Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs1); // Note that we omit the vgtid1 in prevVgtids so it will fallback to the serverVgtid - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen), vgtid0.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, serverVgtid.toString(), prevVgtids); - assertThat(vgtids.size() == numTasks); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), getVgtidOffset(vgtid0.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, getVgtidOffset(serverVgtid.toString()), prevVgtids); + assertThat(offsets.size() == numTasks); Map gtidPerShard = new HashMap<>(); for (int tid = 0; tid < numTasks; tid++) { String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); - String gtidStr = vgtids.get(key); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); assertThat(gtidStr != null); Vgtid vgtid = Vgtid.of(gtidStr); assertThat(vgtid.getShardGtids().size() == 1); @@ -704,6 +1019,61 @@ public void testTaskConfigsOffsetRestartDoubleIncomplete() { assertEquals(expectedGtidPerShard, gtidPerShard); } + @Test + public void testTaskConfigsOffsetRestartDoubleIncompleteOrderMetadata() { + List shards = Arrays.asList("s0", "s1"); + List gtidStrs = Arrays.asList("old0", "old1"); + Vgtid serverVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards, gtidStrs); + final int gen = 1; + final int numTasks = 2; + final int prevNumTasks = 1; + List shards0 = List.of("s0"); + List gtidStrs0 = List.of("gtid0"); + Vgtid taskVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs0); + // Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs1); + // Note that we omit the vgtid1 in prevVgtids so it will fallback to the serverVgtid + + ShardEpochMap serverEpoch = new ShardEpochMap(Collect.hashMapOf("s0", 3L, "s1", 4L)); + ShardEpochMap taskEpoch = new ShardEpochMap(Collect.hashMapOf("s0", 5L)); + + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), getVgtidEpochOffset(taskVgtid.toString(), taskEpoch.toString())); + + Function customConfig = (builder) -> builder.with( + VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class); + + Map serverOffsets = Map.of( + SourceInfo.VGTID_KEY, serverVgtid.toString(), + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, serverEpoch.toString()); + + Map> offsets = getOffsetFromStorage( + numTasks, shards, gen, prevNumTasks, serverOffsets, taskOffsets, + customConfig); + + assertThat(offsets.size() == numTasks); + Map gtidPerShard = new HashMap<>(); + Map epochPerShard = new HashMap<>(); + for (int tid = 0; tid < numTasks; tid++) { + String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); + assertThat(gtidStr != null); + Vgtid vgtid = Vgtid.of(gtidStr); + assertThat(vgtid.getShardGtids().size() == 1); + String epoch = offsets.get(key).get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); + for (int i = 0; i < vgtid.getShardGtids().size(); i++) { + Vgtid.ShardGtid shardGtid = vgtid.getShardGtids().get(i); + String shard = shardGtid.getShard(); + gtidPerShard.put(shard, shardGtid.getGtid()); + epochPerShard.put(shard, ShardEpochMap.of(epoch).get(shard)); + } + } + // Note we got gtid0 from prevGtids, but got old1 from serverGtid + Map expectedGtidPerShard = Collect.hashMapOf("s0", "gtid0", "s1", "old1"); + Map expectedEpochPerShard = Collect.hashMapOf("s0", 5L, "s1", 4L); + assertThat(gtidPerShard).isEqualTo(expectedGtidPerShard); + assertThat(epochPerShard).isEqualTo(expectedEpochPerShard); + } + @Test public void testTaskConfigsOffsetMigrationQuad() { List shards = Arrays.asList("s0", "s1", "s2", "s3"); @@ -718,16 +1088,16 @@ public void testTaskConfigsOffsetMigrationQuad() { final int gen = 2; final int numTasks = 4; final int prevNumTasks = 2; - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen - 1), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, prevNumTasks, gen - 1), vgtid1.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen - 1), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, prevNumTasks, gen - 1), getVgtidOffset(vgtid1.toString())); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, null, prevVgtids); - assertThat(vgtids.size() == numTasks); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, null, prevVgtids); + assertThat(offsets.size() == numTasks); Map gtidPerShard = new HashMap<>(); for (int tid = 0; tid < numTasks; tid++) { String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); - String gtidStr = vgtids.get(key); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); assertThat(gtidStr != null); Vgtid vgtid = Vgtid.of(gtidStr); assertThat(vgtid.getShardGtids().size() == 1); @@ -739,6 +1109,60 @@ public void testTaskConfigsOffsetMigrationQuad() { assertEquals(expectedGtidPerShard, gtidPerShard); } + @Test + public void testTaskConfigsOffsetMigrationQuadOrderMetadata() { + List shards = Arrays.asList("s0", "s1", "s2", "s3"); + Map expectedGtidPerShard = Collect.hashMapOf( + "s0", "gtid0", "s1", "gtid1", "s2", "gtid2", "s3", "gtid3"); + Map expectedEpochPerShard = Collect.hashMapOf( + "s0", 5L, "s1", 4L, "s2", 6L, "s3", 7L); + + List shards0 = Arrays.asList("s0", "s2"); + List shards1 = Arrays.asList("s1", "s3"); + List gtidStrs0 = Arrays.asList("gtid0", "gtid2"); + List gtidStrs1 = Arrays.asList("gtid1", "gtid3"); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards0, gtidStrs0); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, shards1, gtidStrs1); + + ShardEpochMap prevEpoch0 = new ShardEpochMap(Collect.hashMapOf("s0", 5L, "s2", 6L)); + ShardEpochMap prevEpoch1 = new ShardEpochMap(Collect.hashMapOf("s1", 4L, "s3", 7L)); + + final int gen = 2; + final int numTasks = 4; + final int prevNumTasks = 2; + + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen - 1), getVgtidEpochOffset(vgtid0.toString(), prevEpoch0.toString()), + VitessConnector.getTaskKeyName(1, prevNumTasks, gen - 1), getVgtidEpochOffset(vgtid1.toString(), prevEpoch1.toString())); + + Function customConfig = (builder) -> builder.with( + VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class); + + Map> offsets = getOffsetFromStorage( + numTasks, shards, gen, prevNumTasks, null, taskOffsets, customConfig); + + assertThat(offsets.size() == numTasks); + Map gtidPerShard = new HashMap<>(); + Map epochPerShard = new HashMap<>(); + for (int tid = 0; tid < numTasks; tid++) { + String key = VitessConnector.getTaskKeyName(tid, numTasks, gen); + String gtidStr = offsets.get(key).get(SourceInfo.VGTID_KEY); + assertThat(gtidStr).isNotNull(); + Vgtid vgtid = Vgtid.of(gtidStr); + assertThat(vgtid.getShardGtids().size() == 1); + String epoch = offsets.get(key).get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH); + for (int i = 0; i < vgtid.getShardGtids().size(); i++) { + Vgtid.ShardGtid shardGtid = vgtid.getShardGtids().get(i); + String shard = shardGtid.getShard(); + gtidPerShard.put(shard, shardGtid.getGtid()); + epochPerShard.put(shard, ShardEpochMap.of(epoch).get(shard)); + } + } + + assertThat(gtidPerShard).isEqualTo(expectedGtidPerShard); + assertThat(epochPerShard).isEqualTo(expectedEpochPerShard); + } + @Test public void testEmptyOffsetStorage() { final int numTasks = 2; @@ -749,13 +1173,54 @@ public void testEmptyOffsetStorage() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s1", "s3"), Arrays.asList(Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID)); - final Map expectedVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen), vgtid1.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen, -1, null, null); - Testing.print(String.format("vgtids: %s", vgtids)); - assertEquals(vgtids.size(), 2); - assertArrayEquals(vgtids.values().toArray(), expectedVgtids.values().toArray()); + final Map> expectedVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen), getVgtidOffset(vgtid1.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, -1, null, null); + Testing.print(String.format("offsets: %s", offsets)); + assertEquals(offsets.size(), 2); + assertArrayEquals(offsets.values().toArray(), expectedVgtids.values().toArray()); + } + + @Test + public void testEmptyOffsetStorageOrderMetadata() { + final int numTasks = 2; + final int gen = 0; + final List shards = Arrays.asList("s0", "s1", "s2", "s3"); + + // Define Vgtid for each task + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList(Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID)); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList(Vgtid.CURRENT_GTID, Vgtid.CURRENT_GTID)); + + // Define expected epoch map for each shard (using zero for this test) + ShardEpochMap expectedEpochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 0L, "s2", 0L)); + ShardEpochMap expectedEpochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 0L, "s3", 0L)); + + // Create expected offsets with Vgtids and epochs + final Map> expectedOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen), + getVgtidEpochOffset(vgtid0.toString(), expectedEpochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen), + getVgtidEpochOffset(vgtid1.toString(), expectedEpochMap1.toString())); + + // Retrieve offsets from storage + Map> offsets = getOffsetFromStorage(numTasks, shards, gen, -1, null, null, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class)); + + // Debug print for verification + Testing.print(String.format("offsets: %s", offsets)); + + // Assertions + assertEquals(expectedOffsets.size(), offsets.size()); + for (Map.Entry> entry : expectedOffsets.entrySet()) { + String taskKey = entry.getKey(); + Map expectedValues = entry.getValue(); + Map actualValues = offsets.get(taskKey); + + assertThat(expectedValues).isEqualTo(actualValues); + } } @Test @@ -764,8 +1229,8 @@ public void testPreviousOffsetStorage() { final int prevNumTasks = 1; Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); final int numTasks = 2; final List shards = Arrays.asList("s0", "s1", "s2", "s3"); @@ -774,13 +1239,63 @@ public void testPreviousOffsetStorage() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); - final Map expectedVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); - Testing.print(String.format("vgtids: %s", vgtids)); - assertEquals(vgtids.size(), 2); - assertArrayEquals(vgtids.values().toArray(), expectedVgtids.values().toArray()); + final Map> expectedVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidOffset(vgtid1.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); + Testing.print(String.format("offsets: %s", offsets)); + assertEquals(offsets.size(), 2); + assertArrayEquals(offsets.values().toArray(), expectedVgtids.values().toArray()); + } + + @Test + public void testPreviousOffsetStorageOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + + // Define Vgtid for previous tasks + Vgtid prevVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); + ShardEpochMap prevEpochMap = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s1", 1L, "s2", 1L, "s3", 1L)); + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidEpochOffset(prevVgtid.toString(), prevEpochMap.toString())); + + final int numTasks = 2; + final List shards = Arrays.asList("s0", "s1", "s2", "s3"); + + // Define Vgtids for current tasks + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList("gt0", "gt2")); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); + + // Define epoch maps for current tasks + ShardEpochMap epochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s2", 1L)); + ShardEpochMap epochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 1L, "s3", 1L)); + + // Create expected offsets with Vgtids and epochs + final Map> expectedOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), + getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), + getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); + + // Retrieve offsets from storage + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, prevNumTasks, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class)); + + // Debug print for verification + Testing.print(String.format("offsets: %s", offsets)); + + // Assertions + assertEquals(expectedOffsets.size(), offsets.size()); + for (Map.Entry> entry : expectedOffsets.entrySet()) { + String taskKey = entry.getKey(); + Map expectedValues = entry.getValue(); + Map actualValues = offsets.get(taskKey); + + assertThat(actualValues).isEqualTo(expectedValues); + } } @Test @@ -789,8 +1304,8 @@ public void testExpandingShards() { final int prevNumTasks = 1; Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1"), Arrays.asList("gt0", "gt1")); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); final int numTasks = 2; final List shards = Arrays.asList("s0", "s1", "s2", "s3"); @@ -800,13 +1315,64 @@ public void testExpandingShards() { Arrays.asList("s1", "s3"), Arrays.asList("gt1", "current")); // We still expect the code will use the current db shards: s0, s1, s2, s3 - final Map expectedVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); - Testing.print(String.format("vgtids: %s", vgtids)); - assertEquals(vgtids.size(), 2); - assertArrayEquals(vgtids.values().toArray(), expectedVgtids.values().toArray()); + final Map> expectedVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidOffset(vgtid1.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); + Testing.print(String.format("offsets: %s", offsets)); + assertEquals(offsets.size(), 2); + assertArrayEquals(offsets.values().toArray(), expectedVgtids.values().toArray()); + } + + @Test + public void testExpandingShardsOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + + // Define Vgtid for previous tasks + Vgtid prevVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1"), Arrays.asList("gt0", "gt1")); + ShardEpochMap prevEpochMap = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s1", 1L)); + + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidEpochOffset(prevVgtid.toString(), prevEpochMap.toString())); + + final int numTasks = 2; + final List shards = Arrays.asList("s0", "s1", "s2", "s3"); + + // Define Vgtids for current tasks + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList("gt0", "current")); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList("gt1", "current")); + + // Define epoch maps for current tasks + ShardEpochMap epochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s2", 0L)); + ShardEpochMap epochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 1L, "s3", 0L)); + + // Create expected offsets with Vgtids and epochs + final Map> expectedOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), + getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), + getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); + + // Retrieve offsets from storage + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, prevNumTasks, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class)); + + // Debug print for verification + Testing.print(String.format("offsets: %s", offsets)); + + // Assertions + assertEquals(expectedOffsets.size(), offsets.size()); + for (Map.Entry> entry : expectedOffsets.entrySet()) { + String taskKey = entry.getKey(); + Map expectedValues = entry.getValue(); + Map actualValues = offsets.get(taskKey); + + assertThat(actualValues).isEqualTo(expectedValues); + } } @Test @@ -816,8 +1382,8 @@ public void testContractingShards() { Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "current", "current")); // We still expect the code will use the current db shards: s0, s1, s2, s3 - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid0.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid0.toString())); final int numTasks = 1; final List shards = Arrays.asList("s0", "s1"); @@ -833,14 +1399,34 @@ public void testContractingShards() { } } + @Test + public void testContractingShardsOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + + // Define Vgtid for previous tasks + Vgtid prevVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "current", "current")); + ShardEpochMap prevEpochMap = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s1", 1L, "s2", 1L, "s3", 1L)); + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidEpochOffset(prevVgtid.toString(), prevEpochMap.toString())); + + final int numTasks = 1; + final List shards = Arrays.asList("s0", "s1"); + + assertThatThrownBy(() -> getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class))) + .isInstanceOf(IllegalArgumentException.class).hasMessageContaining("Previous shards: [s3, s0, s1, s2] is the superset of current shards: [s0, s1]."); + } + @Test public void testCurrentOffsetStorageShardSplit() { final int gen = 0; final int prevNumTasks = 1; Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); final int numTasks = 2; // "s3" split into "s30", "s31" @@ -850,16 +1436,68 @@ public void testCurrentOffsetStorageShardSplit() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); // We still expect the code will use the old shards "s3" instead of "s30" and "s31" - final Map expectedVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); + final Map> currentGenVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidOffset(vgtid1.toString())); // Add in current gen maps - prevVgtids.putAll(expectedVgtids); + prevVgtids.putAll(currentGenVgtids); + + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); + Testing.print(String.format("offsets: %s", offsets)); + assertEquals(offsets.size(), 2); + // We want to assert on vgtid= + assertArrayEquals(offsets.values().toArray(), currentGenVgtids.values().toArray()); + } + + @Test + public void testCurrentOffsetStorageShardSplitOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + + // Define Vgtid for previous tasks + Vgtid prevVgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); + ShardEpochMap prevEpochMap = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s1", 1L, "s2", 1L, "s3", 1L)); + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidEpochOffset(prevVgtid.toString(), prevEpochMap.toString())); + + final int numTasks = 2; + // "s3" split into "s30", "s31" + final List shards = Arrays.asList("s0", "s1", "s2", "s30", "s31"); + + // Define Vgtids for current tasks + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList("gt0", "gt2")); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); + + // Define epoch maps for current tasks + ShardEpochMap epochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s2", 1L)); + ShardEpochMap epochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 1L, "s3", 1L)); + + // Create expected offsets with Vgtids and epochs + final Map> expectedOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), + getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), + getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); - Testing.print(String.format("vgtids: %s", vgtids)); - assertEquals(vgtids.size(), 2); - assertArrayEquals(vgtids.values().toArray(), expectedVgtids.values().toArray()); + // Retrieve offsets from storage + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class)); + + // Debug print for verification + Testing.print(String.format("offsets: %s", offsets)); + + // Assertions + assertEquals(expectedOffsets.size(), offsets.size()); + for (Map.Entry> entry : expectedOffsets.entrySet()) { + String taskKey = entry.getKey(); + Map expectedValues = entry.getValue(); + Map actualValues = offsets.get(taskKey); + + assertThat(actualValues).isEqualTo(expectedValues); + } } @Test @@ -868,8 +1506,8 @@ public void testCurrentOffsetStorageShardSplitIncomplete() { final int prevNumTasks = 1; Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); final int numTasks = 2; // "s3" split into "s30", "s31" @@ -879,29 +1517,61 @@ public void testCurrentOffsetStorageShardSplitIncomplete() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); // Put in current gen, but missing one task - prevVgtids.put(VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); + prevVgtids.put(VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidOffset(vgtid1.toString())); // We still expect the code will use the old shards "s3" instead of "s30" and "s31" final Map expectedVgtids = Collect.hashMapOf( VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString(), VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); try { - Map vgtids = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); fail("This call should not reach here."); } catch (IllegalArgumentException ex) { - Testing.print(String.format("Got expected exception: {}", ex)); + System.out.println(String.format("Got expected exception: {}", ex)); } } + @Test + public void testCurrentOffsetStorageShardSplitIncompleteOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); + + final int numTasks = 2; + // "s3" split into "s30", "s31" + final List shards = Arrays.asList("s0", "s1", "s2", "s30", "s31"); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList("gt0", "gt2")); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); + ShardEpochMap epochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s2", 1L)); + ShardEpochMap epochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 1L, "s3", 1L)); + + // Put in current gen, but missing one task + taskOffsets.put(VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); + + // We still expect the code will use the old shards "s3" instead of "s30" and "s31" + final Map expectedTaskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); + assertThatThrownBy(() -> getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class))) + .isInstanceOf(IllegalArgumentException.class).hasMessageContaining( + "No offset found for VitessPartition [sourcePartition={server=test_server, task_key=task0_2_1}]"); + } + @Test public void testCurrentOffsetStorageIncomplete() { final int gen = 0; final int prevNumTasks = 1; Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); - final Map prevVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, prevNumTasks, gen), vgtid.toString()); + final Map> prevVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidOffset(vgtid.toString())); final int numTasks = 2; final List shards = Arrays.asList("s0", "s1", "s2", "s3"); @@ -910,16 +1580,50 @@ public void testCurrentOffsetStorageIncomplete() { Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); // Put in current gen, but missing one task - prevVgtids.put(VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString()); + prevVgtids.put(VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidOffset(vgtid0.toString())); // We still expect the code will use the current db shards: s0, s1, s2, s3 - final Map expectedVgtids = Collect.hashMapOf( - VitessConnector.getTaskKeyName(0, numTasks, gen + 1), vgtid0.toString(), - VitessConnector.getTaskKeyName(1, numTasks, gen + 1), vgtid1.toString()); - Map vgtids = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); - Testing.print(String.format("vgtids: %s", vgtids)); - assertEquals(vgtids.size(), 2); - assertArrayEquals(vgtids.values().toArray(), expectedVgtids.values().toArray()); + final Map> expectedVgtids = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidOffset(vgtid0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidOffset(vgtid1.toString())); + Map> offsets = getOffsetFromStorage(numTasks, shards, gen + 1, 1, null, prevVgtids); + Testing.print(String.format("offsets: %s", offsets)); + assertEquals(offsets.size(), 2); + assertArrayEquals(offsets.values().toArray(), expectedVgtids.values().toArray()); + } + + @Test + public void testCurrentOffsetStorageIncompleteOrderMetadata() { + final int gen = 0; + final int prevNumTasks = 1; + Vgtid vgtid = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s1", "s2", "s3"), Arrays.asList("gt0", "gt1", "gt2", "gt3")); + // Define epoch maps for current tasks + ShardEpochMap epochMap = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s1", 1L, "s2", 1L, "s3", 1L)); + + final Map> taskOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, prevNumTasks, gen), getVgtidEpochOffset(vgtid.toString(), epochMap.toString())); + + final int numTasks = 2; + final List shards = Arrays.asList("s0", "s1", "s2", "s3"); + Vgtid vgtid0 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s0", "s2"), Arrays.asList("gt0", "gt2")); + Vgtid vgtid1 = VitessReplicationConnection.buildVgtid(TEST_UNSHARDED_KEYSPACE, + Arrays.asList("s1", "s3"), Arrays.asList("gt1", "gt3")); + ShardEpochMap epochMap0 = new ShardEpochMap(Collect.hashMapOf("s0", 1L, "s2", 1L)); + ShardEpochMap epochMap1 = new ShardEpochMap(Collect.hashMapOf("s1", 1L, "s3", 1L)); + // Put in current gen, but missing one task + taskOffsets.put(VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString())); + + // We still expect the code will use the current db shards: s0, s1, s2, s3 + final Map> expectedOffsets = Collect.hashMapOf( + VitessConnector.getTaskKeyName(0, numTasks, gen + 1), getVgtidEpochOffset(vgtid0.toString(), epochMap0.toString()), + VitessConnector.getTaskKeyName(1, numTasks, gen + 1), getVgtidEpochOffset(vgtid1.toString(), epochMap1.toString())); + Map> offsets = getOffsetFromStorage( + numTasks, shards, gen + 1, 1, null, taskOffsets, + builder -> builder.with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class)); + Testing.print(String.format("offsets: %s", offsets)); + assertThat(offsets).isEqualTo(expectedOffsets); } @Test @@ -947,36 +1651,35 @@ public void testTableIncludeList() { assertEquals(expectedTables, includedTables); } - private void storeOffsets(OffsetBackingStore offsetStore, String serverVgtid, Map prevVgtids) { - if (serverVgtid == null && (prevVgtids == null || prevVgtids.isEmpty())) { + private boolean isEmptyOffsets(Map offsets) { + return offsets == null || offsets.isEmpty(); + } + + private void storeOffsets(OffsetBackingStore offsetStore, Map serverOffsets, Map> taskOffsets) { + if (isEmptyOffsets(serverOffsets) && isEmptyOffsets(taskOffsets)) { Testing.print("Empty gtids to store to offset."); return; } - final String engineName = "testOffset"; - final Converter keyConverter = new JsonConverter(); - Map converterConfig = Collect.hashMapOf("schemas.enable", false); - keyConverter.configure(converterConfig, true); - final Converter valueConverter = new JsonConverter(); - valueConverter.configure(converterConfig, false); - OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, engineName, - keyConverter, valueConverter); - if (serverVgtid != null) { - Testing.print(String.format("Server vgtids: %s", serverVgtid)); + OffsetStorageWriter offsetWriter = getWriter(offsetStore); + + if (!isEmptyOffsets(serverOffsets)) { + Testing.print(String.format("Server offsets: %s", serverOffsets)); Map sourcePartition = Collect.hashMapOf( VitessPartition.SERVER_PARTITION_KEY, TEST_SERVER); - Map offset = Collect.hashMapOf(SourceInfo.VGTID_KEY, serverVgtid); - offsetWriter.offset(sourcePartition, offset); + offsetWriter.offset(sourcePartition, serverOffsets); } - if (prevVgtids != null) { - Testing.print(String.format("Previous vgtids: %s", prevVgtids)); - for (String key : prevVgtids.keySet()) { + + if (!isEmptyOffsets(taskOffsets)) { + Testing.print(String.format("Task offsets: %s", taskOffsets)); + for (String task : taskOffsets.keySet()) { Map sourcePartition = Collect.hashMapOf( VitessPartition.SERVER_PARTITION_KEY, TEST_SERVER, - VitessPartition.TASK_KEY_PARTITION_KEY, key); - Map offset = Collect.hashMapOf(SourceInfo.VGTID_KEY, prevVgtids.get(key)); + VitessPartition.TASK_KEY_PARTITION_KEY, task); + Map offset = taskOffsets.get(task); offsetWriter.offset(sourcePartition, offset); } } + offsetWriter.beginFlush(); Future f = offsetWriter.doFlush(null); try { @@ -987,6 +1690,18 @@ private void storeOffsets(OffsetBackingStore offsetStore, String serverVgtid, Ma } } + private static OffsetStorageWriter getWriter(OffsetBackingStore offsetStore) { + final String engineName = "testOffset"; + final Converter keyConverter = new JsonConverter(); + Map converterConfig = Collect.hashMapOf("schemas.enable", false); + keyConverter.configure(converterConfig, true); + final Converter valueConverter = new JsonConverter(); + valueConverter.configure(converterConfig, false); + OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetStore, engineName, + keyConverter, valueConverter); + return offsetWriter; + } + private static List> getConfigWithOffsetsHelper(List> initialTaskConfigs) { List> taskConfigs = new ArrayList(); for (Map config : initialTaskConfigs) { @@ -998,9 +1713,18 @@ private static List> getConfigWithOffsetsHelper(List getTaskOffsets(OffsetBackingStore offsetStore, int numTasks, List shards, - int gen, int prevNumTasks) { - final Configuration config = TestHelper.defaultConfig(false, true, numTasks, gen, prevNumTasks, null, VitessConnectorConfig.SnapshotMode.NEVER).build(); + private Map> getTaskOffsets(OffsetBackingStore offsetStore, int numTasks, List shards, + int gen, int prevNumTasks, Function customConfig) { + final Configuration config = customConfig.apply( + TestHelper.defaultConfig( + false, + true, + numTasks, + gen, + prevNumTasks, + null, + VitessConnectorConfig.SnapshotMode.NEVER)) + .build(); final String engineName = "testOffset"; final Converter keyConverter = new JsonConverter(); Map converterConfig = Collect.hashMapOf("schemas.enable", false); @@ -1042,8 +1766,9 @@ public Map configs() { }; List> taskConfigs = connector.taskConfigs(numTasks, shards); - Map vgtids = new HashMap<>(); + Map> offsetsMap = new HashMap<>(); for (Map taskConfig : taskConfigs) { + Map taskOffsetMap = new HashMap(); VitessConnectorTask task = new VitessConnectorTask(); task.initialize(sourceTaskContext); final VitessConnectorConfig connectorConfig = new VitessConnectorConfig( @@ -1056,24 +1781,96 @@ public Map configs() { Offsets previousOffsets = Offsets.of(offsets); final VitessOffsetContext previousOffset = previousOffsets.getTheOnlyOffset(); - Vgtid vgtid = previousOffset == null ? VitessReplicationConnection.defaultVgtid(connectorConfig) - : previousOffset.getRestartVgtid(); - vgtids.put(taskConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG), vgtid.toString()); + Vgtid vgtid; + if (previousOffset == null) { + vgtid = VitessReplicationConnection.defaultVgtid(connectorConfig); + } + else { + vgtid = previousOffset.getRestartVgtid(); + } + taskOffsetMap.put(SourceInfo.VGTID_KEY, vgtid.toString()); + if (VitessOffsetRetriever.isShardEpochMapEnabled(connectorConfig) && previousOffset == null) { + ShardEpochMap shardEpochMap = VitessReplicationConnection.defaultShardEpochMap(connectorConfig); + taskOffsetMap.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, shardEpochMap.toString()); + } + else if (VitessOffsetRetriever.isShardEpochMapEnabled(connectorConfig)) { + ShardEpochMap shardEpochMap = ShardEpochMap.of((String) previousOffset.getOffset().get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)); + taskOffsetMap.put(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, shardEpochMap.toString()); + } + offsetsMap.put(taskConfig.get(VitessConnectorConfig.VITESS_TASK_KEY_CONFIG), taskOffsetMap); } connector.stop(); offsetReader.close(); - return vgtids; + return offsetsMap; } - private Map getOffsetFromStorage(int numTasks, List shards, int gen, int prevNumTasks, - String serverVgtid, Map prevVgtids) { + private Map> getOffsetFromStorage(int numTasks, List shards, int gen, int prevNumTasks, + Map serverOffsets, Map> taskOffsets) { + return getOffsetFromStorage(numTasks, shards, gen, prevNumTasks, serverOffsets, taskOffsets, Function.identity()); + } + + private Map> getOffsetFromStorage(int numTasks, List shards, int gen, int prevNumTasks, + Map serverOffsets, Map> taskOffsets, + Function customConfig) { final OffsetBackingStore offsetStore = KafkaConnectUtil.memoryOffsetBackingStore(); offsetStore.start(); - storeOffsets(offsetStore, serverVgtid, prevVgtids); - Map vgtids = getTaskOffsets(offsetStore, numTasks, shards, gen, prevNumTasks); + storeOffsets(offsetStore, serverOffsets, taskOffsets); + Map> offsets = getTaskOffsets(offsetStore, numTasks, shards, gen, prevNumTasks, customConfig); offsetStore.stop(); - return vgtids; + return offsets; + } + + private Map getVgtidOffset(String vgtid) { + return Map.of(SourceInfo.VGTID_KEY, vgtid); + } + + private Map getVgtidEpochOffset(String vgtid, String epoch) { + return Map.of(SourceInfo.VGTID_KEY, vgtid, VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, epoch); + } + + static class ContextHelper { + OffsetBackingStore offsetStore; + String engineName = "testOffset"; + SourceConnectorContext sourceConnectorContext; + + ContextHelper() { + this.offsetStore = KafkaConnectUtil.memoryOffsetBackingStore(); + this.sourceConnectorContext = initSourceConnectorContext(); + } + + public SourceConnectorContext getSourceConnectorContext() { + return this.sourceConnectorContext; + } + + private SourceConnectorContext initSourceConnectorContext() { + offsetStore.start(); + + final Converter keyConverter = new JsonConverter(); + Map converterConfig = Collect.hashMapOf("schemas.enable", false); + keyConverter.configure(converterConfig, true); + final Converter valueConverter = new JsonConverter(); + valueConverter.configure(converterConfig, false); + final OffsetStorageReaderImpl offsetReader = new OffsetStorageReaderImpl(offsetStore, engineName, + keyConverter, valueConverter); + + SourceConnectorContext sourceConnectorContext = new SourceConnectorContext() { + @Override + public void requestTaskReconfiguration() { + } + + @Override + public void raiseError(Exception e) { + } + + @Override + public OffsetStorageReader offsetStorageReader() { + return offsetReader; + } + + }; + return sourceConnectorContext; + } } } diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java index 3b53ff3b..005434bc 100644 --- a/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetContextTest.java @@ -143,6 +143,7 @@ public void shouldGetInitialVitessOrderedTransactionContext() { TransactionContext transactionContext = context.getTransactionContext(); assertThat(transactionContext).isInstanceOf(VitessOrderedTransactionContext.class); VitessOrderedTransactionContext orderedTransactionContext = (VitessOrderedTransactionContext) transactionContext; - assertThat(orderedTransactionContext.getPreviousVgtid()).isEqualTo(null); + assertThat(orderedTransactionContext.getPreviousVgtid()).isEqualTo( + "[{\"keyspace\":\"test_unsharded_keyspace\",\"shard\":\"0\",\"gtid\":\"current\",\"table_p_ks\":[]}]"); } } diff --git a/src/test/java/io/debezium/connector/vitess/VitessOffsetRetrieverTest.java b/src/test/java/io/debezium/connector/vitess/VitessOffsetRetrieverTest.java new file mode 100644 index 00000000..67403c0d --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/VitessOffsetRetrieverTest.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; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.Test; + +import io.debezium.config.Configuration; +import io.debezium.connector.vitess.pipeline.txmetadata.VitessOrderedTransactionMetadataFactory; + +public class VitessOffsetRetrieverTest { + + @Test + public void isShardEpochMapEnabled() { + Configuration configuration = Configuration.create() + .with(VitessConnectorConfig.TRANSACTION_METADATA_FACTORY, VitessOrderedTransactionMetadataFactory.class).build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(configuration); + assertThat(VitessOffsetRetriever.isShardEpochMapEnabled(connectorConfig)).isTrue(); + } +} diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMapTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMapTest.java new file mode 100644 index 00000000..c775e479 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/ShardEpochMapTest.java @@ -0,0 +1,22 @@ +/* + * 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; + +import io.debezium.connector.vitess.TestHelper; + +public class ShardEpochMapTest { + + @Test + public void of() { + ShardEpochMap epoch = ShardEpochMap.of(TestHelper.TEST_SHARD_TO_EPOCH.toString()); + assertThat(epoch.get(TestHelper.TEST_SHARD1)).isEqualTo(TestHelper.TEST_SHARD1_EPOCH); + } +} 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 e67c3d41..79e2b910 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 @@ -6,18 +6,27 @@ package io.debezium.connector.vitess.pipeline.txmetadata; import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1_EPOCH; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD_TO_EPOCH; import static io.debezium.connector.vitess.TestHelper.VGTID_JSON_TEMPLATE; +import static io.debezium.connector.vitess.VgtidTest.VGTID_BOTH_CURRENT; +import static io.debezium.connector.vitess.VgtidTest.VGTID_JSON; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Java6Assertions.assertThatThrownBy; +import java.util.List; import java.util.Map; import org.assertj.core.api.Assertions; import org.junit.Test; import io.debezium.DebeziumException; +import io.debezium.config.Configuration; +import io.debezium.connector.vitess.TablePrimaryKeysTest; +import io.debezium.connector.vitess.TestHelper; import io.debezium.connector.vitess.Vgtid; import io.debezium.connector.vitess.VgtidTest; +import io.debezium.connector.vitess.VitessConnectorConfig; public class VitessEpochProviderTest { @@ -33,24 +42,56 @@ public class VitessEpochProviderTest { private String txIdVersion5 = "MySQL57/" + String.join(",", host1Tx2); private String txIdVersion8 = "MySQL82/" + String.join(",", host1Tx2); + private List shards = List.of(VgtidTest.TEST_SHARD, VgtidTest.TEST_SHARD2); + + 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); + @Test public void testGetEpochSameHostSet() { - Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId, false); assertThat(epoch).isEqualTo(0); } @Test - public void testGetEpochVgtid() { - VitessEpochProvider provider = new VitessEpochProvider(); - 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); + public void testLoadsEpochFromOffsets() { + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); + provider.load(Map.of(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, TEST_SHARD_TO_EPOCH.toString())); + Long epoch = provider.getEpoch(TEST_SHARD1, VGTID_JSON, VGTID_JSON); + assertThat(epoch).isEqualTo(TEST_SHARD1_EPOCH); + } + + @Test + public void testInitializeConfigEpochWithOffsetStorage() { + Configuration config = Configuration.create() + .with(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG, TestHelper.TEST_SHARD_TO_EPOCH.toString()) + .with(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK, "true") + .build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(config); + VitessEpochProvider provider = VitessEpochProvider.initialize(connectorConfig); + Long epoch = provider.getEpoch(TestHelper.TEST_SHARD1, VGTID_JSON, VGTID_JSON); + assertThat(epoch).isEqualTo(TEST_SHARD1_EPOCH); + } + + @Test + public void testInitializeConfigEpochWithShardList() { + Configuration config = Configuration.create() + .with(VitessConnectorConfig.SHARD, TestHelper.TEST_SHARD1) + .build(); + VitessConnectorConfig connectorConfig = new VitessConnectorConfig(config); + VitessEpochProvider provider = VitessEpochProvider.initialize(connectorConfig); + assertThat(provider.getShardEpochMap().get(TEST_SHARD1)).isEqualTo(0); } @Test public void snapshotIncrementsEpoch() { - VitessEpochProvider provider = new VitessEpochProvider(); + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); String vgtidJsonEmpty = String.format( VGTID_JSON_TEMPLATE, VgtidTest.TEST_KEYSPACE, @@ -59,13 +100,13 @@ public void snapshotIncrementsEpoch() { VgtidTest.TEST_KEYSPACE, VgtidTest.TEST_SHARD2, Vgtid.EMPTY_GTID); - Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, vgtidJsonEmpty, VgtidTest.VGTID_JSON); + Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, vgtidJsonEmpty, VGTID_JSON); assertThat(epoch).isEqualTo(1L); } @Test public void fastForwardVgtidIncrementsEpoch() { - VitessEpochProvider provider = new VitessEpochProvider(); + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); String vgtidJsonCurrent = String.format( VGTID_JSON_TEMPLATE, VgtidTest.TEST_KEYSPACE, @@ -74,34 +115,80 @@ public void fastForwardVgtidIncrementsEpoch() { VgtidTest.TEST_KEYSPACE, VgtidTest.TEST_SHARD2, Vgtid.EMPTY_GTID); - Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, vgtidJsonCurrent, VgtidTest.VGTID_JSON); + Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, vgtidJsonCurrent, VGTID_JSON); assertThat(epoch).isEqualTo(1L); } @Test - public void nullPreviousVgtidWithStoredEpochShouldThrowException() { + public void currentVgtidIncrementsEpochForAllShards() { + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); + Long epochShard1 = provider.getEpoch(VgtidTest.TEST_SHARD, vgtidJsonCurrent, VGTID_JSON); + Long epochShard2 = provider.getEpoch(VgtidTest.TEST_SHARD2, VGTID_JSON, VGTID_JSON); + assertThat(epochShard1).isEqualTo(1L); + assertThat(epochShard2).isEqualTo(1L); + } + + @Test + public void splitShard() { VitessEpochProvider provider = new VitessEpochProvider(); + String singleShardVgtidTemplate = "[" + + "{\"keyspace\":\"%s\",\"shard\":\"%s\",\"gtid\":\"%s\",\"table_p_ks\":%s}" + + "]"; + String vgtid1 = String.format( + singleShardVgtidTemplate, + TestHelper.TEST_SHARDED_KEYSPACE, + TestHelper.TEST_SHARD, + txId, + TablePrimaryKeysTest.TEST_LAST_PKS_JSON); + String vgtid2 = String.format( + VGTID_JSON_TEMPLATE, + TestHelper.TEST_SHARDED_KEYSPACE, + TestHelper.TEST_SHARD1, + txId, + TestHelper.TEST_SHARDED_KEYSPACE, + TestHelper.TEST_SHARD2, + txId); + Long epochShard1 = provider.getEpoch(TestHelper.TEST_SHARD, VGTID_BOTH_CURRENT, vgtid1); + Long epochShard2 = provider.getEpoch(TestHelper.TEST_SHARD1, vgtid1, vgtid2); + assertThat(epochShard1).isEqualTo(0L); + assertThat(epochShard2).isEqualTo(0L); + } + + @Test + public void nullPreviousVgtidWithStoredEpochShouldThrowException() { + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); int expectedEpoch = 1; String shardToEpoch = String.format("{\"%s\": %d}", TEST_SHARD1, expectedEpoch); provider.load(Map.of(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, shardToEpoch)); assertThatThrownBy(() -> { - provider.getEpoch(VgtidTest.TEST_SHARD, null, VgtidTest.VGTID_JSON); - }).isInstanceOf(DebeziumException.class).hasMessageContaining("Previous VGTID is null but shardToEpoch map is not null"); + provider.getEpoch(VgtidTest.TEST_SHARD, null, VGTID_JSON); + }).isInstanceOf(DebeziumException.class).hasMessageContaining("Previous vgtid string cannot be null"); } @Test - public void nullPreviousVgtidWithoutStoredEpochShouldReturnZero() { + public void missingEpochWithPreviousVgtidShouldThrowException() { VitessEpochProvider provider = new VitessEpochProvider(); + int expectedEpoch = 1; + String shardToEpoch = String.format("{\"%s\": %d}", TEST_SHARD1, expectedEpoch); + provider.load(Map.of(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, shardToEpoch)); + assertThatThrownBy(() -> { + provider.getEpoch(VgtidTest.TEST_SHARD, VGTID_JSON, VGTID_JSON); + }).isInstanceOf(DebeziumException.class).hasMessageContaining("Previous epoch cannot be null"); + } + + @Test + public void matchingGtidReturnsInitialEpoch() { + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); int expectedEpoch = 0; - Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, null, VgtidTest.VGTID_JSON); + Long epoch = provider.getEpoch(VgtidTest.TEST_SHARD, VGTID_JSON, VGTID_JSON); assertThat(epoch).isEqualTo(expectedEpoch); } @Test public void testInvalidCurrentGtid() { Long expectedEpoch = 0L; - VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); + Long epoch = provider.getEpoch("-80", VGTID_JSON, VGTID_JSON); assertThat(epoch).isEqualTo(expectedEpoch); String vgtidJsonCurrent = String.format( VGTID_JSON_TEMPLATE, @@ -112,15 +199,15 @@ public void testInvalidCurrentGtid() { VgtidTest.TEST_SHARD2, Vgtid.EMPTY_GTID); assertThatThrownBy(() -> { - provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonCurrent); + provider.getEpoch("-80", VGTID_JSON, vgtidJsonCurrent); }).isInstanceOf(DebeziumException.class).hasMessageContaining("Invalid"); } @Test public void testInvalidEmptyGtid() { Long expectedEpoch = 0L; - VitessEpochProvider provider = new VitessEpochProvider(); - Long epoch = provider.getEpoch("-80", VgtidTest.VGTID_JSON, VgtidTest.VGTID_JSON); + VitessEpochProvider provider = new VitessEpochProvider(ShardEpochMap.init(shards)); + Long epoch = provider.getEpoch("-80", VGTID_JSON, VGTID_JSON); assertThat(epoch).isEqualTo(expectedEpoch); String vgtidJsonEmpty = String.format( VGTID_JSON_TEMPLATE, @@ -131,32 +218,32 @@ public void testInvalidEmptyGtid() { VgtidTest.TEST_SHARD2, Vgtid.EMPTY_GTID); assertThatThrownBy(() -> { - provider.getEpoch("-80", VgtidTest.VGTID_JSON, vgtidJsonEmpty); + provider.getEpoch("-80", VGTID_JSON, vgtidJsonEmpty); }).isInstanceOf(DebeziumException.class).hasMessageContaining("Invalid"); } @Test public void testGetEpochShrunkHostSet() { - Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txIdShrunk); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txIdShrunk, false); assertThat(epoch).isEqualTo(1); } @Test public void testGetEpochExpandHostSet() { - Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, previousTxId, txId, false); assertThat(epoch).isEqualTo(0); } @Test public void testGetEpochDisjointThrowsException() { Assertions.assertThatThrownBy(() -> { - VitessEpochProvider.getEpochForGtid(0L, previousTxId, "foo:1-2,bar:2-4"); + VitessEpochProvider.getEpochForGtid(0L, previousTxId, "foo:1-2,bar:2-4", false); }).isInstanceOf(RuntimeException.class); } @Test public void testVersionUpgradeDoesNotAffectEpoch() { - Long epoch = VitessEpochProvider.getEpochForGtid(0L, txIdVersion5, txIdVersion8); + Long epoch = VitessEpochProvider.getEpochForGtid(0L, txIdVersion5, txIdVersion8, false); 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 71f37c37..acbc412e 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 @@ -5,6 +5,9 @@ */ package io.debezium.connector.vitess.pipeline.txmetadata; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD1_EPOCH; +import static io.debezium.connector.vitess.TestHelper.TEST_SHARD_TO_EPOCH; import static org.assertj.core.api.Assertions.assertThat; import java.math.BigDecimal; @@ -17,7 +20,9 @@ import org.junit.Test; import io.debezium.connector.vitess.SourceInfo; +import io.debezium.connector.vitess.TestHelper; import io.debezium.connector.vitess.VgtidTest; +import io.debezium.connector.vitess.VitessConnectorConfig; public class VitessOrderedTransactionContextTest { @@ -31,14 +36,13 @@ public void shouldInit() { @Test public void shouldLoad() { String expectedId = VgtidTest.VGTID_JSON; - String expectedEpoch = "{\"-80\": 5}"; Map offsets = Map.of( - VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, expectedEpoch, + VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH, TEST_SHARD_TO_EPOCH.toString(), SourceInfo.VGTID_KEY, expectedId); VitessOrderedTransactionContext context = VitessOrderedTransactionContext.load(offsets); assertThat(context.previousVgtid).isEqualTo(expectedId); - context.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, "-80")); - assertThat(context.transactionEpoch).isEqualTo(5); + context.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, TEST_SHARD1)); + assertThat(context.transactionEpoch).isEqualTo(TEST_SHARD1_EPOCH); } @Test @@ -54,9 +58,18 @@ public void shouldLoadWithNull() { @Test public void shouldUpdateEpoch() { - VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); - String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; + VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig(true, + false, + 0, + 0, + 0, + null, + VitessConnectorConfig.SnapshotMode.NEVER).with( + VitessConnectorConfig.VGTID, expectedTxId) + .build()); + VitessOrderedTransactionContext metadata = VitessOrderedTransactionContext.initialize(config); + BigDecimal expectedRank = new BigDecimal("7"); long expectedEpoch = 0; String expectedShard = "-80"; @@ -78,7 +91,8 @@ public void shouldUpdateEpoch() { @Test public void shouldUpdateRank() { - VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); + VitessOrderedTransactionContext metadata = VitessOrderedTransactionContext.initialize( + new VitessConnectorConfig(TestHelper.defaultConfig().build())); String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; String expectedShard = "-80"; @@ -95,16 +109,25 @@ public void shouldUpdateRank() { @Test public void shouldStoreOffsets() { - VitessOrderedTransactionContext metadata = new VitessOrderedTransactionContext(); - String expectedTxId = "[{\"keyspace\": \"foo\", \"gtid\": \"host1:1-3,host2:3-4\", \"shard\": \"-80\"}]"; + VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig(true, + false, + 0, + 0, + 0, + null, + VitessConnectorConfig.SnapshotMode.NEVER).with( + VitessConnectorConfig.VGTID, expectedTxId) + .build()); + VitessOrderedTransactionContext metadata = VitessOrderedTransactionContext.initialize(config); + String expectedShard = "-80"; VitessTransactionInfo transactionInfo = new VitessTransactionInfo(expectedTxId, expectedShard); metadata.beginTransaction(transactionInfo); Map offsets = new HashMap(); - String expectedEpoch = "{\"-80\":0}"; + String expectedEpoch = "{\"-80\":0,\"80-\":0}"; Map actualOffsets = metadata.store(offsets); assertThat(actualOffsets.get(VitessOrderedTransactionContext.OFFSET_TRANSACTION_EPOCH)).isEqualTo(expectedEpoch); } diff --git a/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactoryTest.java b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactoryTest.java new file mode 100644 index 00000000..c50a3602 --- /dev/null +++ b/src/test/java/io/debezium/connector/vitess/pipeline/txmetadata/VitessOrderedTransactionMetadataFactoryTest.java @@ -0,0 +1,44 @@ +/* + * 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; + +import io.debezium.config.Configuration; +import io.debezium.connector.vitess.TestHelper; +import io.debezium.connector.vitess.VgtidTest; +import io.debezium.connector.vitess.VitessConnectorConfig; +import io.debezium.pipeline.txmetadata.TransactionContext; + +public class VitessOrderedTransactionMetadataFactoryTest { + + @Test + public void shouldGetDefaultTransactionContext() { + Configuration config = TestHelper.defaultConfig().build(); + TransactionContext context = new VitessOrderedTransactionMetadataFactory(config).getTransactionContext(); + assertThat(context).isInstanceOf(VitessOrderedTransactionContext.class); + } + + @Test + public void shouldGetTransactionContextWithShardEpochMapFromConfig() { + Configuration config = Configuration.create() + .with(VitessConnectorConfig.VITESS_TASK_SHARD_EPOCH_MAP_CONFIG, TestHelper.TEST_SHARD_TO_EPOCH) + .with(VitessConnectorConfig.OFFSET_STORAGE_PER_TASK, "true") + .with(VitessConnectorConfig.VITESS_TASK_VGTID_CONFIG, VgtidTest.VGTID_JSON) + .build(); + TransactionContext context = new VitessOrderedTransactionMetadataFactory(config).getTransactionContext(); + VitessOrderedTransactionContext orderedTransactionContext = (VitessOrderedTransactionContext) context; + orderedTransactionContext.beginTransaction(new VitessTransactionInfo(VgtidTest.VGTID_JSON, TestHelper.TEST_SHARD1)); + assertThat(orderedTransactionContext.getTransactionEpoch()).isEqualTo(TestHelper.TEST_SHARD1_EPOCH); + } + + @Test + public void getTransactionStructMaker() { + } +} 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 c5e7dc1b..c265f8d8 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 @@ -29,7 +29,7 @@ public class VitessOrderedTransactionStructMakerTest { public void prepareTxStruct() { VitessConnectorConfig config = new VitessConnectorConfig(TestHelper.defaultConfig().build()); VitessOrderedTransactionStructMaker maker = new VitessOrderedTransactionStructMaker(Configuration.empty()); - TransactionContext transactionContext = new VitessOrderedTransactionContext(); + TransactionContext transactionContext = VitessOrderedTransactionContext.initialize(config); 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.addTransactionBlock(context, 0, null);