Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[mysql] Fix issue #1944: Fix GTIDs on startup to correctly recover from checkpoint #2220

Merged
merged 2 commits into from
Jun 20, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import io.debezium.connector.AbstractSourceInfo;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.mysql.GtidSet;
import io.debezium.connector.mysql.GtidUtils;
import io.debezium.connector.mysql.MySqlChangeEventSourceMetricsFactory;
import io.debezium.connector.mysql.MySqlConnection;
import io.debezium.connector.mysql.MySqlConnectorConfig;
Expand Down Expand Up @@ -229,10 +230,21 @@ private boolean checkGtidSet(MySqlOffsetContext offset) {
"Connector used GTIDs previously, but MySQL does not know of any GTIDs or they are not enabled");
return false;
}
// GTIDs are enabled
GtidSet gtidSet = new GtidSet(gtidStr);

// Get the GTID set that is available in the server ...
GtidSet availableGtidSet = new GtidSet(availableGtidStr);

// GTIDs are enabled
LOG.info("Merging server GTID set {} with restored GTID set {}", availableGtidSet, gtidStr);

// Based on the current server's GTID, the GTID in MySqlOffsetContext is adjusted to ensure
// the completeness of
// the GTID. This is done to address the issue of being unable to recover from a checkpoint
// in certain startup
// modes.
GtidSet gtidSet = GtidUtils.fixRestoredGtidSet(availableGtidSet, new GtidSet(gtidStr));
LOG.info("Merged GTID set is {}", gtidSet);

if (gtidSet.isContainedWithin(availableGtidSet)) {
LOG.info(
"MySQL current GTID set {} does contain the GTID set {} required by the connector.",
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* Copyright 2022 Ververica Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package io.debezium.connector.mysql;

import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;

/** Utils for handling GTIDs. */
public class GtidUtils {

/**
* This method corrects the GTID set that has been restored from a state or checkpoint using the
* GTID set fetched from the server via SHOW MASTER STATUS. During the correction process, the
* restored GTID set is adjusted according to the server's GTID set to ensure it does not exceed
* the latter. For each UUID in the restored GTID set, if it exists in the server's GTID set,
* then it will be adjusted according to the server's GTID set; if it does not exist in the
* server's GTID set, it will be directly added to the new GTID set.
*/
public static GtidSet fixRestoredGtidSet(GtidSet serverGtidSet, GtidSet restoredGtidSet) {
Map<String, GtidSet.UUIDSet> newSet = new HashMap<>();
serverGtidSet.getUUIDSets().forEach(uuidSet -> newSet.put(uuidSet.getUUID(), uuidSet));
for (GtidSet.UUIDSet uuidSet : restoredGtidSet.getUUIDSets()) {
GtidSet.UUIDSet serverUuidSet = newSet.get(uuidSet.getUUID());
if (serverUuidSet != null) {
long restoredIntervalEnd = getIntervalEnd(uuidSet);
List<com.github.shyiko.mysql.binlog.GtidSet.Interval> newIntervals =
new ArrayList<>();
for (GtidSet.Interval serverInterval : serverUuidSet.getIntervals()) {
if (serverInterval.getEnd() <= restoredIntervalEnd) {
newIntervals.add(
new com.github.shyiko.mysql.binlog.GtidSet.Interval(
serverInterval.getStart(), serverInterval.getEnd()));
} else if (serverInterval.getStart() <= restoredIntervalEnd
&& serverInterval.getEnd() > restoredIntervalEnd) {
newIntervals.add(
new com.github.shyiko.mysql.binlog.GtidSet.Interval(
serverInterval.getStart(), restoredIntervalEnd));
}
}
newSet.put(
uuidSet.getUUID(),
new GtidSet.UUIDSet(
new com.github.shyiko.mysql.binlog.GtidSet.UUIDSet(
uuidSet.getUUID(), newIntervals)));
} else {
newSet.put(uuidSet.getUUID(), uuidSet);
}
}
return new GtidSet(newSet);
}

/**
* This method merges one GTID set (toMerge) into another (base), without overwriting the
* existing elements in the base GTID set.
*/
public static GtidSet mergeGtidSetInto(GtidSet base, GtidSet toMerge) {
Map<String, GtidSet.UUIDSet> newSet = new HashMap<>();
base.getUUIDSets().forEach(uuidSet -> newSet.put(uuidSet.getUUID(), uuidSet));
for (GtidSet.UUIDSet uuidSet : toMerge.getUUIDSets()) {
if (!newSet.containsKey(uuidSet.getUUID())) {
newSet.put(uuidSet.getUUID(), uuidSet);
}
}
return new GtidSet(newSet);
}

private static long getIntervalEnd(GtidSet.UUIDSet uuidSet) {
return uuidSet.getIntervals().stream()
.mapToLong(GtidSet.Interval::getEnd)
.max()
.getAsLong();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,7 +83,15 @@

import static io.debezium.util.Strings.isNullOrEmpty;

/** Copied from Debezium project to add more error details for some exceptions in line 1465. */
/**
* Copied from Debezium project to fix
* https://github.com/ververica/flink-cdc-connectors/issues/1944.
*
* <p>Line 1428-1434 : Adjust GTID merging logic to support recovering from job which previously
* specifying starting offset on start.
*
* <p>Line 1486 : Add more error details for some exceptions.
*/
public class MySqlStreamingChangeEventSource
implements StreamingChangeEventSource<MySqlPartition, MySqlOffsetContext> {

Expand Down Expand Up @@ -115,6 +123,7 @@ public class MySqlStreamingChangeEventSource
private final MySqlConnection connection;
private final EventDispatcher<MySqlPartition, TableId> eventDispatcher;
private final ErrorHandler errorHandler;
private boolean isRestoredFromCheckpoint = false;

@SingleThreadAccess("binlog client thread")
private Instant eventTimestamp;
Expand Down Expand Up @@ -1406,11 +1415,23 @@ public GtidSet filterGtidSet(
LOGGER.info(
"Relevant GTID set available on server: {}", relevantAvailableServerGtidSet);

// Since the GTID recorded in the checkpoint represents the CDC-executed records, in
// certain scenarios
// (such as when the startup mode is earliest/timestamp/binlogfile), the recorded GTID
// may not start from
// the beginning. For example, A:300-500. However, during job recovery, we usually only
// need to focus on
// the last consumed point instead of consuming A:1-299. Therefore, some adjustments
// need to be made to the
// recorded offset in the checkpoint, and the available GTID for other MySQL instances
// should be completed.
mergedGtidSet =
relevantAvailableServerGtidSet
.retainAll(uuid -> knownGtidSet.forServerWithId(uuid) != null)
.with(purgedServerGtid)
.with(filteredGtidSet);
GtidUtils.fixRestoredGtidSet(
GtidUtils.mergeGtidSetInto(
relevantAvailableServerGtidSet.retainAll(
uuid -> knownGtidSet.forServerWithId(uuid) != null),
purgedServerGtid),
filteredGtidSet);
} else {
mergedGtidSet = availableServerGtidSet.with(filteredGtidSet);
}
Expand Down Expand Up @@ -1542,6 +1563,10 @@ public void onEventDeserializationFailure(BinaryLogClient client, Exception ex)
}
}

protected void setRestoredFromCheckpoint() {
this.isRestoredFromCheckpoint = true;
}

@FunctionalInterface
private interface TableIdProvider<E extends EventData> {
TableId getTableId(E data);
Expand Down
Loading