From 1d6a1ec6c15a40e53c9792390acdb8975b1560c7 Mon Sep 17 00:00:00 2001 From: Chris Cranford Date: Mon, 13 Nov 2023 19:31:32 -0500 Subject: [PATCH] DBZ-7083 Introduce strategy for MySQL and MariaDB --- debezium-connector-mysql/pom.xml | 26 +- .../mysql/AbstractMySqlFieldReader.java | 2 +- .../io/debezium/connector/mysql/GtidSet.java | 474 +----------- .../mysql/MySqlChangeEventSourceFactory.java | 7 +- .../connector/mysql/MySqlConnection.java | 690 ------------------ .../connector/mysql/MySqlConnector.java | 24 +- .../connector/mysql/MySqlConnectorConfig.java | 111 ++- .../connector/mysql/MySqlConnectorTask.java | 87 +-- .../connector/mysql/MySqlOffsetContext.java | 4 +- .../mysql/MySqlSnapshotChangeEventSource.java | 13 +- .../MySqlStreamingChangeEventSource.java | 307 +------- .../connector/mysql/MySqlValueConverters.java | 21 +- .../AbstractBinaryLogClientConfigurator.java | 295 ++++++++ .../AbstractConnectionConfiguration.java | 167 +++++ .../strategy/AbstractConnectorConnection.java | 502 +++++++++++++ .../AbstractHistoryRecordComparator.java} | 23 +- .../strategy/BinaryLogClientConfigurator.java | 29 + .../strategy/ConnectionConfiguration.java | 42 ++ .../mysql/strategy/ConnectorAdapter.java | 63 ++ .../MariaDbBinaryLogClientConfigurator.java | 59 ++ .../strategy/mariadb/MariaDbConnection.java | 87 +++ .../MariaDbConnectionConfiguration.java | 37 + .../mariadb/MariaDbConnectorAdapter.java | 101 +++ .../strategy/mariadb/MariaDbGtidSet.java | 55 ++ .../MariaDbHistoryRecordComparator.java | 27 + .../hybrid/MariaDbHybridConnectorAdapter.java | 43 ++ .../MySqlBinaryLogClientConfigurator.java | 20 + .../mysql/strategy/mysql/MySqlConnection.java | 146 ++++ .../mysql/MySqlConnectionConfiguration.java | 37 + .../strategy/mysql/MySqlConnectorAdapter.java | 118 +++ .../mysql/strategy/mysql/MySqlGtidSet.java | 472 ++++++++++++ .../mysql/MySqlHistoryRecordComparator.java | 27 + ...yIncrementalSnapshotChangeEventSource.java | 9 +- ...SqlReadOnlyIncrementalSnapshotContext.java | 25 +- .../debezium/connector/mysql/GtidSetTest.java | 47 +- .../mysql/MySqlDatabaseSchemaTest.java | 3 +- .../mysql/MySqlValueConvertersTest.java | 8 +- .../connector/mysql/SourceInfoTest.java | 1 + .../connector/mysql/UniqueDatabase.java | 3 + .../history/KafkaSchemaHistoryTest.java | 2 +- 40 files changed, 2628 insertions(+), 1586 deletions(-) delete mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnection.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractBinaryLogClientConfigurator.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectionConfiguration.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectorConnection.java rename debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/{MySqlHistoryRecordComparator.java => strategy/AbstractHistoryRecordComparator.java} (92%) create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/BinaryLogClientConfigurator.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectionConfiguration.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectorAdapter.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbBinaryLogClientConfigurator.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnection.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectionConfiguration.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectorAdapter.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbGtidSet.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbHistoryRecordComparator.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/hybrid/MariaDbHybridConnectorAdapter.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlBinaryLogClientConfigurator.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnection.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectionConfiguration.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectorAdapter.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlGtidSet.java create mode 100644 debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlHistoryRecordComparator.java rename debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/{ => strategy/mysql}/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java (97%) rename debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/{ => strategy/mysql}/MySqlReadOnlyIncrementalSnapshotContext.java (87%) diff --git a/debezium-connector-mysql/pom.xml b/debezium-connector-mysql/pom.xml index 4a2c3911f..3b356d5d8 100644 --- a/debezium-connector-mysql/pom.xml +++ b/debezium-connector-mysql/pom.xml @@ -176,6 +176,7 @@ 60000 8080 60000 + mysql jdbc:mysql com.mysql.cj.jdbc.Driver + ${mysql.database.adapter} ${mysql.database.protocol} ${mysql.database.jdbc.driver} ${skipLongRunningTests} @@ -696,6 +698,7 @@ ${mysql.port} disabled + ${mysql.database.adapter} ${mysql.database.protocol} ${mysql.database.jdbc.driver} false @@ -716,6 +719,7 @@ ${mysql.port} ${mysql.port} + ${mysql.database.adapter} ${mysql.database.protocol} ${mysql.database.jdbc.driver} @@ -732,6 +736,7 @@ ${mysql.gtid.port} ${mysql.gtid.replica.port} + ${mysql.database.adapter} ${mysql.database.protocol} ${mysql.database.jdbc.driver} @@ -768,6 +773,7 @@ ${mysql.ssl.port} ${mysql.ssl.port} + ${mysql.database.adapter} ${mysql.database.protocol} ${mysql.database.jdbc.driver} @@ -932,23 +938,9 @@ false - debezium/mysql-server-test-database - ${mariadb.server.image.source} - ${version.mariadb.server} - jdbc:mariadb - org.mariadb.jdbc.Driver - - - - - mariadb-ssl - - false - - - debezium/mysql-server-test-database-ssl ${mariadb.server.image.source} ${version.mariadb.server} + mariadb jdbc:mariadb org.mariadb.jdbc.Driver @@ -1016,6 +1008,7 @@ ${mysql.database.protocol} ${mysql.database.jdbc.driver} + ${mysql.database.adapter} false true @@ -1096,6 +1089,7 @@ ${mysql.database.protocol} ${mysql.database.jdbc.driver} + ${mysql.database.adapter} false true @@ -1174,6 +1168,7 @@ ${mysql.database.protocol} ${mysql.database.jdbc.driver} + ${mysql.database.adapter} false true @@ -1252,6 +1247,7 @@ ${mysql.database.protocol} ${mysql.database.jdbc.driver} + ${mysql.database.adapter} false true ${project.basedir}/src/test/resources/ssl/truststore diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/AbstractMySqlFieldReader.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/AbstractMySqlFieldReader.java index 27bf2245c..9aa83532e 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/AbstractMySqlFieldReader.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/AbstractMySqlFieldReader.java @@ -87,7 +87,7 @@ public Object readField(ResultSet rs, int columnIndex, Column column, Table tabl try { String columnData = rs.getString(columnIndex); if (columnData != null) { - return columnData.getBytes(MySqlConnection.getJavaEncodingForMysqlCharSet(column.charsetName())); + return columnData.getBytes(connectorConfig.getConnectorAdapter().getJavaEncodingForCharSet(column.charsetName())); } } catch (UnsupportedEncodingException e) { diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/GtidSet.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/GtidSet.java index 18564a166..36ec0d78d 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/GtidSet.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/GtidSet.java @@ -5,476 +5,64 @@ */ package io.debezium.connector.mysql; -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; import java.util.function.Predicate; -import java.util.regex.Pattern; -import java.util.stream.Collectors; - -import io.debezium.annotation.Immutable; /** - * A set of MySQL GTIDs. This is an improvement of {@link com.github.shyiko.mysql.binlog.GtidSet} that is immutable, - * and more properly supports comparisons. + * Represents a common contract for GTID behavior for MySQL and MariaDB. * - * @author Randall Hauch + * @author Randall Hauch, Chris Cranford */ -@Immutable -public final class GtidSet { - - private final Map uuidSetsByServerId = new TreeMap<>(); // sorts on keys - public static Pattern GTID_DELIMITER = Pattern.compile(":"); - - protected GtidSet(Map uuidSetsByServerId) { - this.uuidSetsByServerId.putAll(uuidSetsByServerId); - } +public interface GtidSet { /** - * @param gtids the string representation of the GTIDs. + * Returns whether this {@link GtidSet} is empty. */ - public GtidSet(String gtids) { - gtids = gtids.replace("\n", "").replace("\r", ""); - new com.github.shyiko.mysql.binlog.GtidSet(gtids).getUUIDSets().forEach(uuidSet -> { - uuidSetsByServerId.put(uuidSet.getUUID(), new UUIDSet(uuidSet)); - }); - StringBuilder sb = new StringBuilder(); - uuidSetsByServerId.values().forEach(uuidSet -> { - if (sb.length() != 0) { - sb.append(','); - } - sb.append(uuidSet.toString()); - }); - } + boolean isEmpty(); /** - * Obtain a copy of this {@link GtidSet} except with only the GTID ranges that have server UUIDs that match the given - * predicate. + * Obtain a copy of this {@link GtidSet} except with only the GTID ranges match the specified predicate. * - * @param sourceFilter the predicate that returns whether a server UUID is to be included + * @param sourceFilter the predicate that returns whether a server identifier is to be included * @return the new GtidSet, or this object if {@code sourceFilter} is null; never null */ - public GtidSet retainAll(Predicate sourceFilter) { - if (sourceFilter == null) { - return this; - } - Map newSets = this.uuidSetsByServerId.entrySet() - .stream() - .filter(entry -> sourceFilter.test(entry.getKey())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - return new GtidSet(newSets); - } + // todo: change to T + GtidSet retainAll(Predicate sourceFilter); /** - * Get an immutable collection of the {@link UUIDSet range of GTIDs for a single server}. - * - * @return the {@link UUIDSet GTID ranges for each server}; never null - */ - public Collection getUUIDSets() { - return Collections.unmodifiableCollection(uuidSetsByServerId.values()); - } - - /** - * Find the {@link UUIDSet} for the server with the specified Uuid. - * - * @param uuid the Uuid of the server - * @return the {@link UUIDSet} for the identified server, or {@code null} if there are no GTIDs from that server. - */ - public UUIDSet forServerWithId(String uuid) { - return uuidSetsByServerId.get(uuid); - } - - /** - * Determine if the GTIDs represented by this object are contained completely within the supplied set of GTIDs. + * Determine whether the GTIDs represented by this object are contained completely within the supplied set. * * @param other the other set of GTIDs; may be null - * @return {@code true} if all of the GTIDs in this set are completely contained within the supplied set of GTIDs, or - * {@code false} otherwise + * @return {@code true} if all GTIDs are present in the provided set, {@code false} otherwise */ - public boolean isContainedWithin(GtidSet other) { - if (other == null) { - return false; - } - if (this.equals(other)) { - return true; - } - for (UUIDSet uuidSet : uuidSetsByServerId.values()) { - UUIDSet thatSet = other.forServerWithId(uuidSet.getUUID()); - if (!uuidSet.isContainedWithin(thatSet)) { - return false; - } - } - return true; - } + boolean isContainedWithin(GtidSet other); /** - * Obtain a copy of this {@link GtidSet} except overwritten with all of the GTID ranges in the supplied {@link GtidSet}. - * @param other the other {@link GtidSet} with ranges to add/overwrite on top of those in this set; - * @return the new GtidSet, or this object if {@code other} is null or empty; never null + * Obtain a copy of this {@link GtidSet} except overwritten with all the GTID ranges in the supplied {@link GtidSet}. + * + * @param other the other {@link GtidSet} with ranges to add/overwrite on top of those in this set + * @return the new {@link GtidSet}, or this object if {@code other} is null or empty; never null */ - public GtidSet with(GtidSet other) { - if (other == null || other.uuidSetsByServerId.isEmpty()) { - return this; - } - Map newSet = new HashMap<>(); - newSet.putAll(this.uuidSetsByServerId); - newSet.putAll(other.uuidSetsByServerId); - return new GtidSet(newSet); - } + GtidSet with(GtidSet other); /** - * Returns a copy with all intervals set to beginning - * @return + * Returns a copy of this with all intervals set to the beginning. */ - public GtidSet getGtidSetBeginning() { - Map newSet = new HashMap<>(); - - for (UUIDSet uuidSet : uuidSetsByServerId.values()) { - newSet.put(uuidSet.getUUID(), uuidSet.asIntervalBeginning()); - } - - return new GtidSet(newSet); - } - - public boolean contains(String gtid) { - String[] split = GTID_DELIMITER.split(gtid); - String sourceId = split[0]; - UUIDSet uuidSet = forServerWithId(sourceId); - if (uuidSet == null) { - return false; - } - long transactionId = Long.parseLong(split[1]); - return uuidSet.contains(transactionId); - } - - public GtidSet subtract(GtidSet other) { - if (other == null) { - return this; - } - Map newSets = this.uuidSetsByServerId.entrySet() - .stream() - .filter(entry -> !entry.getValue().isContainedWithin(other.forServerWithId(entry.getKey()))) - .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), entry.getValue().subtract(other.forServerWithId(entry.getKey())))) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - return new GtidSet(newSets); - } - - @Override - public int hashCode() { - return uuidSetsByServerId.keySet().hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj instanceof GtidSet) { - GtidSet that = (GtidSet) obj; - return this.uuidSetsByServerId.equals(that.uuidSetsByServerId); - } - return false; - } - - @Override - public String toString() { - List gtids = new ArrayList(); - for (UUIDSet uuidSet : uuidSetsByServerId.values()) { - gtids.add(uuidSet.toString()); - } - return String.join(",", gtids); - } + GtidSet getGtidSetBeginning(); /** - * A range of GTIDs for a single server with a specific Uuid. + * Return whether the specified GTID is present in this set. + * + * @param gtid the gtid to check; may not be null + * @return {@code true} if contained by this set, {@code false} otherwise */ - @Immutable - public static class UUIDSet { + boolean contains(String gtid); - private final String uuid; - private final LinkedList intervals = new LinkedList<>(); + /** + * Subtracts the two GTID sets. + * + * @param other ther other set; may be null + * @return a new {@link GtidSet} that contains the difference in GTIDs + */ + GtidSet subtract(GtidSet other); - protected UUIDSet(com.github.shyiko.mysql.binlog.GtidSet.UUIDSet uuidSet) { - this.uuid = uuidSet.getUUID(); - uuidSet.getIntervals().forEach(interval -> { - intervals.add(new Interval(interval.getStart(), interval.getEnd())); - }); - Collections.sort(this.intervals); - if (this.intervals.size() > 1) { - // Collapse adjacent intervals ... - for (int i = intervals.size() - 1; i != 0; --i) { - Interval before = this.intervals.get(i - 1); - Interval after = this.intervals.get(i); - if ((before.getEnd() + 1) == after.getStart()) { - this.intervals.set(i - 1, new Interval(before.getStart(), after.getEnd())); - this.intervals.remove(i); - } - } - } - } - - protected UUIDSet(String uuid, Interval interval) { - this.uuid = uuid; - this.intervals.add(interval); - } - - protected UUIDSet(String uuid, List intervals) { - this.uuid = uuid; - this.intervals.addAll(intervals); - } - - public UUIDSet asIntervalBeginning() { - Interval start = new Interval(intervals.get(0).getStart(), intervals.get(0).getStart()); - return new UUIDSet(this.uuid, start); - } - - /** - * Get the Uuid for the server that generated the GTIDs. - * - * @return the server's Uuid; never null - */ - public String getUUID() { - return uuid; - } - - /** - * Get the intervals of transaction numbers. - * - * @return the immutable transaction intervals; never null - */ - public List getIntervals() { - return Collections.unmodifiableList(intervals); - } - - /** - * Determine if the set of transaction numbers from this server is completely within the set of transaction numbers from - * the set of transaction numbers in the supplied set. - * - * @param other the set to compare with this set - * @return {@code true} if this server's transaction numbers are a subset of the transaction numbers of the supplied set, - * or false otherwise - */ - public boolean isContainedWithin(UUIDSet other) { - if (other == null) { - return false; - } - if (!this.getUUID().equalsIgnoreCase(other.getUUID())) { - // Not even the same server ... - return false; - } - if (this.intervals.isEmpty()) { - return true; - } - if (other.intervals.isEmpty()) { - return false; - } - assert this.intervals.size() > 0; - assert other.intervals.size() > 0; - - // Every interval in this must be within an interval of the other ... - for (Interval thisInterval : this.intervals) { - boolean found = false; - for (Interval otherInterval : other.intervals) { - if (thisInterval.isContainedWithin(otherInterval)) { - found = true; - break; - } - } - if (!found) { - return false; // didn't find a match - } - } - return true; - } - - public boolean contains(long transactionId) { - for (Interval interval : this.intervals) { - if (interval.contains(transactionId)) { - return true; - } - } - return false; - } - - @Override - public int hashCode() { - return uuid.hashCode(); - } - - @Override - public boolean equals(Object obj) { - if (obj == this) { - return true; - } - if (obj instanceof UUIDSet) { - UUIDSet that = (UUIDSet) obj; - return this.getUUID().equalsIgnoreCase(that.getUUID()) && this.getIntervals().equals(that.getIntervals()); - } - return super.equals(obj); - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(uuid).append(':'); - Iterator iter = intervals.iterator(); - if (iter.hasNext()) { - sb.append(iter.next()); - } - while (iter.hasNext()) { - sb.append(':'); - sb.append(iter.next()); - } - return sb.toString(); - } - - public UUIDSet subtract(UUIDSet other) { - if (!uuid.equals(other.getUUID())) { - throw new IllegalArgumentException("UUIDSet subtraction is supported only within a single server UUID"); - } - List result = new ArrayList<>(); - for (Interval interval : intervals) { - result.addAll(interval.removeAll(other.getIntervals())); - } - return new UUIDSet(uuid, result); - } - } - - @Immutable - public static class Interval implements Comparable { - - private final long start; - private final long end; - - public Interval(long start, long end) { - this.start = start; - this.end = end; - } - - /** - * Get the starting transaction number in this interval. - * - * @return this interval's first transaction number - */ - public long getStart() { - return start; - } - - /** - * Get the ending transaction number in this interval. - * - * @return this interval's last transaction number - */ - public long getEnd() { - return end; - } - - /** - * Determine if this interval is completely within the supplied interval. - * - * @param other the interval to compare with - * @return {@code true} if the {@link #getStart() start} is greater than or equal to the supplied interval's - * {@link #getStart() start} and the {@link #getEnd() end} is less than or equal to the supplied interval's - * {@link #getEnd() end}, or {@code false} otherwise - */ - public boolean isContainedWithin(Interval other) { - if (other == this) { - return true; - } - if (other == null) { - return false; - } - return this.getStart() >= other.getStart() && this.getEnd() <= other.getEnd(); - } - - public boolean contains(long transactionId) { - return getStart() <= transactionId && transactionId <= getEnd(); - } - - public boolean contains(Interval other) { - return getStart() <= other.getStart() && getEnd() >= other.getEnd(); - } - - public boolean nonintersecting(Interval other) { - return other.getEnd() < this.getStart() || other.getStart() > this.getEnd(); - } - - public List remove(Interval other) { - if (nonintersecting(other)) { - return Collections.singletonList(this); - } - if (other.contains(this)) { - return Collections.emptyList(); - } - List result = new LinkedList<>(); - if (this.getStart() < other.getStart()) { - Interval part = new Interval(this.getStart(), other.getStart() - 1); - result.add(part); - } - if (other.getEnd() < this.getEnd()) { - Interval part = new Interval(other.getEnd() + 1, this.getEnd()); - result.add(part); - } - return result; - } - - public List removeAll(List otherIntervals) { - List thisIntervals = new LinkedList<>(); - thisIntervals.add(this); - List result = new LinkedList<>(); - result.add(this); - for (Interval other : otherIntervals) { - result = new LinkedList<>(); - for (Interval thisInterval : thisIntervals) { - result.addAll(thisInterval.remove(other)); - } - thisIntervals = result; - } - return result; - } - - @Override - public int compareTo(Interval that) { - if (that == this) { - return 0; - } - long diff = this.start - that.start; - if (diff > Integer.MAX_VALUE) { - return Integer.MAX_VALUE; - } - if (diff < Integer.MIN_VALUE) { - return Integer.MIN_VALUE; - } - return (int) diff; - } - - @Override - public int hashCode() { - return (int) getStart(); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - if (obj instanceof Interval) { - Interval that = (Interval) obj; - return this.getStart() == that.getStart() && this.getEnd() == that.getEnd(); - } - return false; - } - - @Override - public String toString() { - return "" + getStart() + "-" + getEnd(); - } - } } diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlChangeEventSourceFactory.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlChangeEventSourceFactory.java index fe397f164..d27073bc6 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlChangeEventSourceFactory.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlChangeEventSourceFactory.java @@ -11,6 +11,7 @@ import org.apache.kafka.connect.source.SourceRecord; import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; import io.debezium.jdbc.MainConnectionProvidingConnectionFactory; import io.debezium.pipeline.DataChangeEvent; import io.debezium.pipeline.ErrorHandler; @@ -31,7 +32,7 @@ public class MySqlChangeEventSourceFactory implements ChangeEventSourceFactory { private final MySqlConnectorConfig configuration; - private final MainConnectionProvidingConnectionFactory connectionFactory; + private final MainConnectionProvidingConnectionFactory connectionFactory; private final ErrorHandler errorHandler; private final EventDispatcher dispatcher; private final Clock clock; @@ -44,7 +45,7 @@ public class MySqlChangeEventSourceFactory implements ChangeEventSourceFactory queue; - public MySqlChangeEventSourceFactory(MySqlConnectorConfig configuration, MainConnectionProvidingConnectionFactory connectionFactory, + public MySqlChangeEventSourceFactory(MySqlConnectorConfig configuration, MainConnectionProvidingConnectionFactory connectionFactory, ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, MySqlDatabaseSchema schema, MySqlTaskContext taskContext, MySqlStreamingChangeEventSourceMetrics streamingMetrics, ChangeEventQueue queue) { @@ -97,7 +98,7 @@ public StreamingChangeEventSource getStreami if (configuration.isReadOnlyConnection()) { if (connectionFactory.mainConnection().isGtidModeEnabled()) { - return Optional.of(new MySqlReadOnlyIncrementalSnapshotChangeEventSource<>( + return Optional.of(configuration.getConnectorAdapter().createIncrementalSnapshotChangeEventSource( configuration, connectionFactory.mainConnection(), dispatcher, diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnection.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnection.java deleted file mode 100644 index 67d8e6767..000000000 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnection.java +++ /dev/null @@ -1,690 +0,0 @@ -/* - * 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.mysql; - -import static io.debezium.config.CommonConnectorConfig.DATABASE_CONFIG_PREFIX; -import static io.debezium.config.CommonConnectorConfig.DRIVER_CONFIG_PREFIX; - -import java.sql.ResultSet; -import java.sql.SQLException; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.OptionalLong; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.mysql.cj.CharsetMapping; - -import io.debezium.DebeziumException; -import io.debezium.config.CommonConnectorConfig; -import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; -import io.debezium.config.Configuration; -import io.debezium.config.Configuration.Builder; -import io.debezium.config.Field; -import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; -import io.debezium.jdbc.JdbcConfiguration; -import io.debezium.jdbc.JdbcConnection; -import io.debezium.relational.Column; -import io.debezium.relational.Table; -import io.debezium.relational.TableId; -import io.debezium.util.Strings; - -/** - * {@link JdbcConnection} extension to be used with MySQL Server - * - * @author Jiri Pechanec, Randall Hauch - * - */ -public class MySqlConnection extends JdbcConnection { - - private static Logger LOGGER = LoggerFactory.getLogger(MySqlConnection.class); - - private static final String SQL_SHOW_SYSTEM_VARIABLES = "SHOW VARIABLES"; - private static final String SQL_SHOW_SYSTEM_VARIABLES_CHARACTER_SET = "SHOW VARIABLES WHERE Variable_name IN ('character_set_server','collation_server')"; - private static final String SQL_SHOW_SESSION_VARIABLE_SSL_VERSION = "SHOW SESSION STATUS LIKE 'Ssl_version'"; - private static final String QUOTED_CHARACTER = "`"; - - protected static final String URL_PATTERN = "${protocol}://${hostname}:${port}/?useInformationSchema=true&nullCatalogMeansCurrent=false&useUnicode=true&characterEncoding=UTF-8&characterSetResults=UTF-8&zeroDateTimeBehavior=CONVERT_TO_NULL&connectTimeout=${connectTimeout}"; - - private final Map originalSystemProperties = new HashMap<>(); - private final MySqlConnectionConfiguration connectionConfig; - private final MySqlFieldReader mysqlFieldReader; - - // Tracks whether this connection is with MariaDB, calculated lazily as needed. - private Boolean isMariaDb; - - /** - * Creates a new connection using the supplied configuration. - * - * @param connectionConfig {@link MySqlConnectionConfiguration} instance, may not be null. - * @param fieldReader binary or text protocol based readers - */ - public MySqlConnection(MySqlConnectionConfiguration connectionConfig, MySqlFieldReader fieldReader) { - super(connectionConfig.jdbcConfig, connectionConfig.factory(), QUOTED_CHARACTER, QUOTED_CHARACTER); - this.connectionConfig = connectionConfig; - this.mysqlFieldReader = fieldReader; - } - - /** - * Creates a new connection using the supplied configuration. - * - * @param connectionConfig {@link MySqlConnectionConfiguration} instance, may not be null. - */ - public MySqlConnection(MySqlConnectionConfiguration connectionConfig) { - this(connectionConfig, new MySqlTextProtocolFieldReader(null)); - } - - @Override - public void close() throws SQLException { - try { - super.close(); - } - finally { - // Reset the system properties to their original value ... - originalSystemProperties.forEach((name, value) -> { - if (value != null) { - System.setProperty(name, value); - } - else { - System.clearProperty(name); - } - }); - } - } - - /** - * Read the MySQL charset-related system variables. - * - * @return the system variables that are related to server character sets; never null - */ - protected Map readMySqlCharsetSystemVariables() { - // Read the system variables from the MySQL instance and get the current database name ... - LOGGER.debug("Reading MySQL charset-related system variables before parsing DDL history."); - return querySystemVariables(SQL_SHOW_SYSTEM_VARIABLES_CHARACTER_SET); - } - - /** - * Read the MySQL system variables. - * - * @return the system variables that are related to server character sets; never null - */ - protected Map readMySqlSystemVariables() { - // Read the system variables from the MySQL instance and get the current database name ... - LOGGER.debug("Reading MySQL system variables"); - return querySystemVariables(SQL_SHOW_SYSTEM_VARIABLES); - } - - private Map querySystemVariables(String statement) { - final Map variables = new HashMap<>(); - try { - query(statement, rs -> { - while (rs.next()) { - String varName = rs.getString(1); - String value = rs.getString(2); - if (varName != null && value != null) { - variables.put(varName, value); - LOGGER.debug("\t{} = {}", - Strings.pad(varName, 45, ' '), - Strings.pad(value, 45, ' ')); - } - } - }); - } - catch (SQLException e) { - throw new DebeziumException("Error reading MySQL variables: " + e.getMessage(), e); - } - - return variables; - } - - protected String setStatementFor(Map variables) { - StringBuilder sb = new StringBuilder("SET "); - boolean first = true; - List varNames = new ArrayList<>(variables.keySet()); - Collections.sort(varNames); - for (String varName : varNames) { - if (first) { - first = false; - } - else { - sb.append(", "); - } - sb.append(varName).append("="); - String value = variables.get(varName); - if (value == null) { - value = ""; - } - if (value.contains(",") || value.contains(";")) { - value = "'" + value + "'"; - } - sb.append(value); - } - return sb.append(";").toString(); - } - - protected void setSystemProperty(String property, Field field, boolean showValueInError) { - String value = connectionConfig.originalConfig().getString(field); - if (value != null) { - value = value.trim(); - String existingValue = System.getProperty(property); - if (existingValue == null) { - // There was no existing property ... - String existing = System.setProperty(property, value); - originalSystemProperties.put(property, existing); // the existing value may be null - } - else { - existingValue = existingValue.trim(); - if (!existingValue.equalsIgnoreCase(value)) { - // There was an existing property, and the value is different ... - String msg = "System or JVM property '" + property + "' is already defined, but the configuration property '" - + field.name() - + "' defines a different value"; - if (showValueInError) { - msg = "System or JVM property '" + property + "' is already defined as " + existingValue - + ", but the configuration property '" + field.name() + "' defines a different value '" + value + "'"; - } - throw new DebeziumException(msg); - } - // Otherwise, there was an existing property, and the value is exactly the same (so do nothing!) - } - } - } - - /** - * Read the Ssl Version session variable. - * - * @return the session variables that are related to sessions ssl version - */ - protected String getSessionVariableForSslVersion() { - final String SSL_VERSION = "Ssl_version"; - LOGGER.debug("Reading MySQL Session variable for Ssl Version"); - Map sessionVariables = querySystemVariables(SQL_SHOW_SESSION_VARIABLE_SSL_VERSION); - if (!sessionVariables.isEmpty() && sessionVariables.containsKey(SSL_VERSION)) { - return sessionVariables.get(SSL_VERSION); - } - return null; - } - - /** - * Determine whether the MySQL server has GTIDs enabled. - * - * @return {@code false} if the server's {@code gtid_mode} is set and is {@code OFF}, or {@code true} otherwise - */ - public boolean isGtidModeEnabled() { - try { - return queryAndMap("SHOW GLOBAL VARIABLES LIKE 'GTID_MODE'", rs -> { - if (rs.next()) { - return "ON".equalsIgnoreCase(rs.getString(2)); - } - return false; - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at GTID mode: ", e); - } - } - - /** - * Determine the executed GTID set for MySQL. - * - * @return the string representation of MySQL's GTID sets; never null but an empty string if the server does not use GTIDs - */ - public String knownGtidSet() { - try { - return queryAndMap("SHOW MASTER STATUS", rs -> { - if (rs.next() && rs.getMetaData().getColumnCount() > 4) { - return rs.getString(5); // GTID set, may be null, blank, or contain a GTID set - } - return ""; - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at GTID mode: ", e); - } - } - - /** - * Determine the difference between two sets. - * - * @return a subtraction of two GTID sets; never null - */ - public GtidSet subtractGtidSet(GtidSet set1, GtidSet set2) { - try { - return prepareQueryAndMap("SELECT GTID_SUBTRACT(?, ?)", - ps -> { - ps.setString(1, set1.toString()); - ps.setString(2, set2.toString()); - }, - rs -> { - if (rs.next()) { - return new GtidSet(rs.getString(1)); - } - return new GtidSet(""); - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at GTID mode: ", e); - } - } - - /** - * Get the purged GTID values from MySQL (gtid_purged value) - * - * @return A GTID set; may be empty if not using GTIDs or none have been purged yet - */ - public GtidSet purgedGtidSet() { - try { - return queryAndMap("SELECT @@global.gtid_purged", rs -> { - if (rs.next() && rs.getMetaData().getColumnCount() > 0) { - return new GtidSet(rs.getString(1)); // GTID set, may be null, blank, or contain a GTID set - } - return new GtidSet(""); - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at gtid_purged variable: ", e); - } - } - - /** - * Determine if the current user has the named privilege. Note that if the user has the "ALL" privilege this method - * returns {@code true}. - * - * @param grantName the name of the MySQL privilege; may not be null - * @return {@code true} if the user has the named privilege, or {@code false} otherwise - */ - public boolean userHasPrivileges(String grantName) { - try { - return queryAndMap("SHOW GRANTS FOR CURRENT_USER", rs -> { - while (rs.next()) { - String grants = rs.getString(1); - LOGGER.debug(grants); - if (grants == null) { - return false; - } - grants = grants.toUpperCase(); - if (grants.contains("ALL") || grants.contains(grantName.toUpperCase())) { - return true; - } - } - return false; - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at privileges for current user: ", e); - } - } - - /** - * Determine the earliest binlog filename that is still available in the server. - * - * @return the name of the earliest binlog filename, or null if there are none. - */ - public String earliestBinlogFilename() { - // Accumulate the available binlog filenames ... - List logNames = new ArrayList<>(); - try { - LOGGER.info("Checking all known binlogs from MySQL"); - query("SHOW BINARY LOGS", rs -> { - while (rs.next()) { - logNames.add(rs.getString(1)); - } - }); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking for binary logs: ", e); - } - - if (logNames.isEmpty()) { - return null; - } - return logNames.get(0); - } - - /** - * Determine whether the MySQL server has the binlog_row_image set to 'FULL'. - * - * @return {@code true} if the server's {@code binlog_row_image} is set to {@code FULL}, or {@code false} otherwise - */ - protected boolean isBinlogRowImageFull() { - try { - final String rowImage = queryAndMap("SHOW GLOBAL VARIABLES LIKE 'binlog_row_image'", rs -> { - if (rs.next()) { - return rs.getString(2); - } - // This setting was introduced in MySQL 5.6+ with default of 'FULL'. - // For older versions, assume 'FULL'. - return "FULL"; - }); - LOGGER.debug("binlog_row_image={}", rowImage); - return "FULL".equalsIgnoreCase(rowImage); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at BINLOG_ROW_IMAGE mode: ", e); - } - } - - /** - * Determine whether the MySQL server has the row-level binlog enabled. - * - * @return {@code true} if the server's {@code binlog_format} is set to {@code ROW}, or {@code false} otherwise - */ - protected boolean isBinlogFormatRow() { - try { - final String mode = queryAndMap("SHOW GLOBAL VARIABLES LIKE 'binlog_format'", rs -> rs.next() ? rs.getString(2) : ""); - LOGGER.debug("binlog_format={}", mode); - return "ROW".equalsIgnoreCase(mode); - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking at BINLOG_FORMAT mode: ", e); - } - } - - /** - * Query the database server to get the list of the binlog files availble. - * - * @return list of the binlog files - */ - public List availableBinlogFiles() { - List logNames = new ArrayList<>(); - try { - LOGGER.info("Get all known binlogs from MySQL"); - query("SHOW BINARY LOGS", rs -> { - while (rs.next()) { - logNames.add(rs.getString(1)); - } - }); - return logNames; - } - catch (SQLException e) { - throw new DebeziumException("Unexpected error while connecting to MySQL and looking for binary logs: ", e); - } - } - - public OptionalLong getEstimatedTableSize(TableId tableId) { - try { - // Choose how we create statements based on the # of rows. - // This is approximate and less accurate then COUNT(*), - // but far more efficient for large InnoDB tables. - execute("USE `" + tableId.catalog() + "`;"); - return queryAndMap("SHOW TABLE STATUS LIKE '" + tableId.table() + "';", rs -> { - if (rs.next()) { - return OptionalLong.of((rs.getLong(5))); - } - return OptionalLong.empty(); - }); - } - catch (SQLException e) { - LOGGER.debug("Error while getting number of rows in table {}: {}", tableId, e.getMessage(), e); - } - return OptionalLong.empty(); - } - - public boolean isMariaDb() { - if (isMariaDb == null) { - final String version = querySystemVariables(SQL_SHOW_SYSTEM_VARIABLES).get("version"); - isMariaDb = version.toLowerCase().contains("mariadb"); - } - return isMariaDb; - } - - public boolean isTableIdCaseSensitive() { - return !"0".equals(readMySqlSystemVariables().get(MySqlSystemVariables.LOWER_CASE_TABLE_NAMES)); - } - - /** - * Read the MySQL default character sets for exisiting databases. - * - * @return the map of database names with their default character sets; never null - */ - protected Map readDatabaseCollations() { - LOGGER.debug("Reading default database charsets"); - try { - return queryAndMap("SELECT schema_name, default_character_set_name, default_collation_name FROM information_schema.schemata", rs -> { - final Map charsets = new HashMap<>(); - while (rs.next()) { - String dbName = rs.getString(1); - String charset = rs.getString(2); - String collation = rs.getString(3); - if (dbName != null && (charset != null || collation != null)) { - charsets.put(dbName, new DatabaseLocales(charset, collation)); - LOGGER.debug("\t{} = {}, {}", - Strings.pad(dbName, 45, ' '), - Strings.pad(charset, 45, ' '), - Strings.pad(collation, 45, ' ')); - } - } - return charsets; - }); - } - catch (SQLException e) { - throw new DebeziumException("Error reading default database charsets: " + e.getMessage(), e); - } - } - - public MySqlConnectionConfiguration connectionConfig() { - return connectionConfig; - } - - public String connectionString() { - return connectionString(URL_PATTERN); - } - - public static String getJavaEncodingForMysqlCharSet(String mysqlCharsetName) { - return CharsetMappingWrapper.getJavaEncodingForMysqlCharSet(mysqlCharsetName); - } - - /** - * Helper to gain access to protected method - */ - private final static class CharsetMappingWrapper extends CharsetMapping { - static String getJavaEncodingForMysqlCharSet(String mySqlCharsetName) { - return CharsetMapping.getStaticJavaEncodingForMysqlCharset(mySqlCharsetName); - } - } - - public static class MySqlConnectionConfiguration { - - protected static final String JDBC_PROPERTY_CONNECTION_TIME_ZONE = "connectionTimeZone"; - protected static final String JDBC_PROPERTY_MARIADB_TIME_ZONE = "timezone"; - - private final JdbcConfiguration jdbcConfig; - private final ConnectionFactory factory; - private final Configuration config; - - public MySqlConnectionConfiguration(Configuration config) { - // Set up the JDBC connection without actually connecting, with extra MySQL-specific properties - // to give us better JDBC database metadata behavior, including using UTF-8 for the client-side character encoding - // per https://dev.mysql.com/doc/connector-j/5.1/en/connector-j-reference-charsets.html - this.config = config; - final boolean useSSL = sslModeEnabled(); - final Configuration dbConfig = config - .edit() - .withDefault(MySqlConnectorConfig.PORT, MySqlConnectorConfig.PORT.defaultValue()) - .withDefault(MySqlConnectorConfig.JDBC_PROTOCOL, MySqlConnectorConfig.JDBC_PROTOCOL.defaultValue()) - .build() - .subset(DATABASE_CONFIG_PREFIX, true) - .merge(config.subset(DRIVER_CONFIG_PREFIX, true)); - - final Builder jdbcConfigBuilder = dbConfig - .edit() - .with("connectTimeout", Long.toString(getConnectionTimeout().toMillis())) - .with("sslMode", sslMode().getValue()); - - if (useSSL) { - if (!Strings.isNullOrBlank(sslTrustStore())) { - jdbcConfigBuilder.with("trustCertificateKeyStoreUrl", "file:" + sslTrustStore()); - } - if (sslTrustStorePassword() != null) { - jdbcConfigBuilder.with("trustCertificateKeyStorePassword", String.valueOf(sslTrustStorePassword())); - } - if (!Strings.isNullOrBlank(sslKeyStore())) { - jdbcConfigBuilder.with("clientCertificateKeyStoreUrl", "file:" + sslKeyStore()); - } - if (sslKeyStorePassword() != null) { - jdbcConfigBuilder.with("clientCertificateKeyStorePassword", String.valueOf(sslKeyStorePassword())); - } - } - - if (isUsingMariaDbProtocol(config)) { - jdbcConfigBuilder.with(JDBC_PROPERTY_MARIADB_TIME_ZONE, determineConnectionTimeZoneForMariaDbDriver(dbConfig)); - } - else { - jdbcConfigBuilder.with(JDBC_PROPERTY_CONNECTION_TIME_ZONE, determineConnectionTimeZoneForMySqlDriver(dbConfig)); - } - - // Set and remove options to prevent potential vulnerabilities - jdbcConfigBuilder - .with("allowLoadLocalInfile", "false") - .with("allowUrlInLocalInfile", "false") - .with("autoDeserialize", false) - .without("queryInterceptors"); - - this.jdbcConfig = JdbcConfiguration.adapt(jdbcConfigBuilder.build()); - String driverClassName = this.config.getString(MySqlConnectorConfig.JDBC_DRIVER); - Field protocol = MySqlConnectorConfig.JDBC_PROTOCOL; - - factory = JdbcConnection.patternBasedFactory(MySqlConnection.URL_PATTERN, driverClassName, getClass().getClassLoader(), protocol); - } - - private static boolean isUsingMariaDbProtocol(Configuration config) { - final String jdbcProtocol = config.getString(MySqlConnectorConfig.JDBC_PROTOCOL); - return !Strings.isNullOrBlank(jdbcProtocol) && jdbcProtocol.equalsIgnoreCase("jdbc:mariadb"); - } - - private static String determineConnectionTimeZoneForMariaDbDriver(final Configuration dbConfig) { - // Debezium by default expected timezone data delivered in server timezone - String timezone = dbConfig.getString(JDBC_PROPERTY_MARIADB_TIME_ZONE); - return timezone != null ? timezone : "auto"; - } - - private static String determineConnectionTimeZoneForMySqlDriver(final Configuration dbConfig) { - // Debezium by default expects timezoned data delivered in server timezone - String connectionTimeZone = dbConfig.getString(JDBC_PROPERTY_CONNECTION_TIME_ZONE); - - if (connectionTimeZone != null) { - return connectionTimeZone; - } - - return "SERVER"; - } - - public JdbcConfiguration config() { - return jdbcConfig; - } - - public Configuration originalConfig() { - return config; - } - - public ConnectionFactory factory() { - return factory; - } - - public String username() { - return config.getString(MySqlConnectorConfig.USER); - } - - public String password() { - return config.getString(MySqlConnectorConfig.PASSWORD); - } - - public String hostname() { - return config.getString(MySqlConnectorConfig.HOSTNAME); - } - - public int port() { - return config.getInteger(MySqlConnectorConfig.PORT); - } - - public SecureConnectionMode sslMode() { - String mode = config.getString(MySqlConnectorConfig.SSL_MODE); - return SecureConnectionMode.parse(mode); - } - - public boolean sslModeEnabled() { - return sslMode() != SecureConnectionMode.DISABLED; - } - - public String sslKeyStore() { - return config.getString(MySqlConnectorConfig.SSL_KEYSTORE); - } - - public char[] sslKeyStorePassword() { - String password = config.getString(MySqlConnectorConfig.SSL_KEYSTORE_PASSWORD); - return Strings.isNullOrBlank(password) ? null : password.toCharArray(); - } - - public String sslTrustStore() { - return config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE); - } - - public char[] sslTrustStorePassword() { - String password = config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE_PASSWORD); - return Strings.isNullOrBlank(password) ? null : password.toCharArray(); - } - - public Duration getConnectionTimeout() { - return Duration.ofMillis(config.getLong(MySqlConnectorConfig.CONNECTION_TIMEOUT_MS)); - } - - public EventProcessingFailureHandlingMode eventProcessingFailureHandlingMode() { - String mode = config.getString(CommonConnectorConfig.EVENT_PROCESSING_FAILURE_HANDLING_MODE); - if (mode == null) { - mode = config.getString(MySqlConnectorConfig.EVENT_DESERIALIZATION_FAILURE_HANDLING_MODE); - } - return EventProcessingFailureHandlingMode.parse(mode); - } - - public EventProcessingFailureHandlingMode inconsistentSchemaHandlingMode() { - String mode = config.getString(MySqlConnectorConfig.INCONSISTENT_SCHEMA_HANDLING_MODE); - return EventProcessingFailureHandlingMode.parse(mode); - } - } - - @Override - public Object getColumnValue(ResultSet rs, int columnIndex, Column column, Table table) throws SQLException { - return mysqlFieldReader.readField(rs, columnIndex, column, table); - } - - @Override - public String quotedTableIdString(TableId tableId) { - return tableId.toQuotedString('`'); - } - - public static class DatabaseLocales { - private final String charset; - private final String collation; - - public DatabaseLocales(String charset, String collation) { - this.charset = charset; - this.collation = collation; - } - - public void appendToDdlStatement(String dbName, StringBuilder ddl) { - if (charset != null) { - LOGGER.debug("Setting default charset '{}' for database '{}'", charset, dbName); - ddl.append(" CHARSET ").append(charset); - } - else { - LOGGER.info("Default database charset for '{}' not found", dbName); - } - if (collation != null) { - LOGGER.debug("Setting default collation '{}' for database '{}'", collation, dbName); - ddl.append(" COLLATE ").append(collation); - } - else { - LOGGER.info("Default database collation for '{}' not found", dbName); - } - } - } -} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnector.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnector.java index e1fd40369..641eb543e 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnector.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnector.java @@ -20,7 +20,8 @@ import io.debezium.annotation.Immutable; import io.debezium.config.Configuration; import io.debezium.connector.common.RelationalBaseSourceConnector; -import io.debezium.connector.mysql.MySqlConnection.MySqlConnectionConfiguration; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; import io.debezium.relational.RelationalDatabaseConnectorConfig; /** @@ -78,17 +79,25 @@ public ConfigDef config() { @Override protected void validateConnection(Map configValues, Configuration config) { + ConfigValue adapterValue = configValues.get(MySqlConnectorConfig.CONNECTOR_ADAPTER.name()); ConfigValue hostnameValue = configValues.get(RelationalDatabaseConnectorConfig.HOSTNAME.name()); + + ConnectorAdapter adapter = adapter(config); + if (adapter == null) { + LOGGER.error("Failed to resolve connection adapter."); + adapterValue.addErrorMessage("Failed to resolve the connector's connection adapter."); + return; + } + // Try to connect to the database ... - final MySqlConnectionConfiguration connectionConfig = new MySqlConnectionConfiguration(config); - try (MySqlConnection connection = new MySqlConnection(connectionConfig)) { + try (AbstractConnectorConnection connection = adapter.createConnection(config)) { try { connection.connect(); connection.execute("SELECT version()"); - LOGGER.info("Successfully tested connection for {} with user '{}'", connection.connectionString(), connectionConfig.username()); + LOGGER.info("Successfully tested connection for {} with user '{}'", connection.connectionString(), connection.connectionConfig().username()); } catch (SQLException e) { - LOGGER.error("Failed testing connection for {} with user '{}'", connection.connectionString(), connectionConfig.username(), e); + LOGGER.error("Failed testing connection for {} with user '{}'", connection.connectionString(), connection.connectionConfig().username(), e); hostnameValue.addErrorMessage("Unable to connect: " + e.getMessage()); } } @@ -101,4 +110,9 @@ protected void validateConnection(Map configValues, Configu protected Map validateAllFields(Configuration config) { return config.validate(MySqlConnectorConfig.ALL_FIELDS); } + + private static ConnectorAdapter adapter(Configuration config) { + // todo: find a better way to handle this + return new MySqlConnectorConfig(config).getConnectorAdapter(); + } } diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorConfig.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorConfig.java index fb7252126..32892f3bd 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorConfig.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorConfig.java @@ -25,6 +25,10 @@ import io.debezium.config.Field.ValidationOutput; import io.debezium.connector.AbstractSourceInfo; import io.debezium.connector.SourceInfoStructMaker; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; +import io.debezium.connector.mysql.strategy.mariadb.MariaDbConnectorAdapter; +import io.debezium.connector.mysql.strategy.mariadb.hybrid.MariaDbHybridConnectorAdapter; +import io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter; import io.debezium.function.Predicates; import io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode; import io.debezium.jdbc.TemporalPrecisionMode; @@ -507,6 +511,92 @@ public static SecureConnectionMode parse(String value, String defaultValue) { } } + /** + * Set of predefined connector adapter modes. + */ + public enum ConnectorAdapterMode implements EnumeratedValue { + /** + * Expects the target database to be MySQL using the MySQL driver. + * This should also be used if the target database is MySQL compliant but isn't MariaDB. + */ + MYSQL("mysql") { + @Override + protected ConnectorAdapter getAdapter(MySqlConnectorConfig connectorConfig) { + LOGGER.info("Using " + MySqlConnectorAdapter.class.getName()); + return new MySqlConnectorAdapter(connectorConfig); + } + }, + + /** + * Expects the target database to be MariaDB using the MariaDB driver. + */ + MARIADB("mariadb") { + @Override + protected ConnectorAdapter getAdapter(MySqlConnectorConfig connectorConfig) { + LOGGER.info("Using " + MariaDbConnectorAdapter.class.getName()); + return new MariaDbConnectorAdapter(connectorConfig); + } + }, + + /** + * Expects the target database to be MariaDB but uses the MySQL driver. + */ + MARIADB_HYBRID("mariadb-hybrid") { + @Override + protected ConnectorAdapter getAdapter(MySqlConnectorConfig connectorConfig) { + LOGGER.info("Using " + MariaDbHybridConnectorAdapter.class.getName()); + return new MariaDbHybridConnectorAdapter(connectorConfig); + } + }; + + private final String value; + + protected abstract ConnectorAdapter getAdapter(MySqlConnectorConfig connectorConfig); + + ConnectorAdapterMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static ConnectorAdapterMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (ConnectorAdapterMode option : ConnectorAdapterMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is invalid + */ + public static ConnectorAdapterMode parse(String value, String defaultValue) { + ConnectorAdapterMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + /** * {@link Integer#MIN_VALUE Minimum value} used for fetch size hint. * See DBZ-94 for details. @@ -877,6 +967,14 @@ public static SecureConnectionMode parse(String value, String defaultValue) { .withImportance(Importance.LOW) .withDescription("Switched connector to use alternative methods to deliver signals to Debezium instead of writing to signaling table"); + public static final Field CONNECTOR_ADAPTER = Field.create("connector.adapter") + .withDisplayName("Connection adapter to be used") + .withEnum(ConnectorAdapterMode.class, ConnectorAdapterMode.MYSQL) + .withGroup(Field.createGroupEntry(Field.Group.ADVANCED, 28)) + .withWidth(Width.SHORT) + .withImportance(Importance.MEDIUM) + .withDescription("Specifies the connection adapter to be used"); + public static final Field SOURCE_INFO_STRUCT_MAKER = CommonConnectorConfig.SOURCE_INFO_STRUCT_MAKER .withDefault(MySqlSourceInfoStructMaker.class.getName()); @@ -920,7 +1018,8 @@ public static SecureConnectionMode parse(String value, String defaultValue) { ROW_COUNT_FOR_STREAMING_RESULT_SETS, INCREMENTAL_SNAPSHOT_CHUNK_SIZE, INCREMENTAL_SNAPSHOT_ALLOW_SCHEMA_CHANGES, - STORE_ONLY_CAPTURED_DATABASES_DDL) + STORE_ONLY_CAPTURED_DATABASES_DDL, + CONNECTOR_ADAPTER) .events( INCLUDE_SQL_QUERY, TABLE_IGNORE_BUILTIN, @@ -967,6 +1066,7 @@ protected boolean supportsSchemaChangesDuringIncrementalSnapshot() { private final Predicate gtidSourceFilter; private final EventProcessingFailureHandlingMode inconsistentSchemaFailureHandlingMode; private final boolean readOnlyConnection; + private final ConnectorAdapter connectorAdapter; public MySqlConnectorConfig(Configuration config) { super( @@ -999,6 +1099,9 @@ public MySqlConnectorConfig(Configuration config) { : (gtidSetExcludes != null ? Predicates.excludesUuids(gtidSetExcludes) : null); this.storeOnlyCapturedDatabasesDdl = config.getBoolean(STORE_ONLY_CAPTURED_DATABASES_DDL); + + // This should always be last to guarantee the full configuration is passed in the constructor + this.connectorAdapter = ConnectorAdapterMode.parse(config.getString(CONNECTOR_ADAPTER)).getAdapter(this); } public boolean useCursorFetch() { @@ -1156,6 +1259,10 @@ public int bufferSizeForStreamingChangeEventSource() { return config.getInteger(MySqlConnectorConfig.BUFFER_SIZE_FOR_BINLOG_READER); } + public ConnectorAdapter getConnectorAdapter() { + return connectorAdapter; + } + /** * Get the predicate function that will return {@code true} if a GTID source is to be included, or {@code false} if * a GTID source is to be excluded. @@ -1180,7 +1287,7 @@ public long rowCountForLargeTable() { @Override protected HistoryRecordComparator getHistoryRecordComparator() { - return new MySqlHistoryRecordComparator(gtidSourceFilter()); + return connectorAdapter.getHistoryRecordComparator(); } public static boolean isBuiltInDatabase(String databaseName) { diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorTask.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorTask.java index bf9923f91..cf350a142 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorTask.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlConnectorTask.java @@ -19,9 +19,12 @@ import io.debezium.config.Field; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.common.BaseSourceTask; -import io.debezium.connector.mysql.MySqlConnection.MySqlConnectionConfiguration; import io.debezium.connector.mysql.MySqlConnectorConfig.BigIntUnsignedHandlingMode; import io.debezium.connector.mysql.MySqlConnectorConfig.SnapshotMode; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; +import io.debezium.connector.mysql.strategy.mysql.MySqlConnection; +import io.debezium.connector.mysql.strategy.mysql.MySqlConnectionConfiguration; import io.debezium.document.DocumentReader; import io.debezium.jdbc.DefaultMainConnectionProvidingConnectionFactory; import io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode; @@ -56,7 +59,7 @@ public class MySqlConnectorTask extends BaseSourceTask queue; - private volatile MySqlConnection connection; + private volatile AbstractConnectorConnection connection; private volatile ErrorHandler errorHandler; private volatile MySqlDatabaseSchema schema; @@ -81,8 +84,10 @@ public ChangeEventSourceCoordinator start(Co .withDefault("database.useCursorFetch", connectorConfig.useCursorFetch()) .build(); - MainConnectionProvidingConnectionFactory connectionFactory = new DefaultMainConnectionProvidingConnectionFactory<>( - () -> new MySqlConnection(new MySqlConnectionConfiguration(config), getFieldReader(connectorConfig))); + final ConnectorAdapter adapter = connectorConfig.getConnectorAdapter(); + + MainConnectionProvidingConnectionFactory connectionFactory = new DefaultMainConnectionProvidingConnectionFactory<>( + () -> adapter.createConnection(config)); connection = connectionFactory.mainConnection(); @@ -214,7 +219,7 @@ private MySqlValueConverters getValueConverters(MySqlConnectorConfig configurati final boolean timeAdjusterEnabled = configuration.getConfig().getBoolean(MySqlConnectorConfig.ENABLE_TIME_ADJUSTER); return new MySqlValueConverters(decimalMode, timePrecisionMode, bigIntUnsignedMode, configuration.binaryHandlingMode(), timeAdjusterEnabled ? MySqlValueConverters::adjustTemporal : x -> x, - MySqlValueConverters::defaultParsingErrorHandler); + MySqlValueConverters::defaultParsingErrorHandler, configuration.getConnectorAdapter()); } private MySqlFieldReader getFieldReader(MySqlConnectorConfig configuration) { @@ -278,70 +283,6 @@ private void validateBinlogConfiguration(MySqlConnectorConfig config) { } } - /** - * Determine whether the binlog position as set on the {@link MySqlOffsetContext} is available in the server. - * - * @return {@code true} if the server has the binlog coordinates, or {@code false} otherwise - */ - protected boolean isBinlogAvailable(MySqlConnectorConfig config, MySqlOffsetContext offset) { - String gtidStr = offset.gtidSet(); - if (gtidStr != null) { - if (gtidStr.trim().isEmpty()) { - return true; // start at beginning ... - } - String availableGtidStr = connection.knownGtidSet(); - if (availableGtidStr == null || availableGtidStr.trim().isEmpty()) { - // Last offsets had GTIDs but the server does not use them ... - LOGGER.info("Connector used GTIDs previously, but MySQL does not know of any GTIDs or they are not enabled"); - return false; - } - // GTIDs are enabled, and we used them previously, but retain only those GTID ranges for the allowed source UUIDs ... - GtidSet gtidSet = new GtidSet(gtidStr).retainAll(config.gtidSourceFilter()); - // Get the GTID set that is available in the server ... - GtidSet availableGtidSet = new GtidSet(availableGtidStr); - if (gtidSet.isContainedWithin(availableGtidSet)) { - LOGGER.info("MySQL current GTID set {} does contain the GTID set required by the connector {}", availableGtidSet, gtidSet); - final GtidSet knownServerSet = availableGtidSet.retainAll(config.gtidSourceFilter()); - final GtidSet gtidSetToReplicate = connection.subtractGtidSet(knownServerSet, gtidSet); - final GtidSet purgedGtidSet = connection.purgedGtidSet(); - LOGGER.info("Server has already purged {} GTIDs", purgedGtidSet); - final GtidSet nonPurgedGtidSetToReplicate = connection.subtractGtidSet(gtidSetToReplicate, purgedGtidSet); - LOGGER.info("GTIDs known by the server but not processed yet {}, for replication are available only {}", gtidSetToReplicate, nonPurgedGtidSetToReplicate); - if (!gtidSetToReplicate.equals(nonPurgedGtidSetToReplicate)) { - LOGGER.info("Some of the GTIDs needed to replicate have been already purged"); - return false; - } - return true; - } - LOGGER.info("Connector last known GTIDs are {}, but MySQL has {}", gtidSet, availableGtidSet); - return false; - } - - String binlogFilename = offset.getSource().binlogFilename(); - if (binlogFilename == null) { - return true; // start at current position - } - if (binlogFilename.equals("")) { - return true; // start at beginning - } - - // Accumulate the available binlog filenames ... - List logNames = connection.availableBinlogFiles(); - - // And compare with the one we're supposed to use ... - boolean found = logNames.stream().anyMatch(binlogFilename::equals); - if (!found) { - if (LOGGER.isInfoEnabled()) { - LOGGER.info("Connector requires binlog file '{}', but MySQL only has {}", binlogFilename, String.join(", ", logNames)); - } - } - else { - LOGGER.info("MySQL has the binlog file '{}' required by the connector", binlogFilename); - } - - return found; - } - private boolean validateAndLoadSchemaHistory(MySqlConnectorConfig config, MySqlPartition partition, MySqlOffsetContext offset, MySqlDatabaseSchema schema) { if (offset == null) { if (config.getSnapshotMode().shouldSnapshotOnSchemaError()) { @@ -357,7 +298,7 @@ private boolean validateAndLoadSchemaHistory(MySqlConnectorConfig config, MySqlP LOGGER.warn("Database schema history was not found but was expected"); if (config.getSnapshotMode().shouldSnapshotOnSchemaError()) { // But check to see if the server still has those binlog coordinates ... - if (!isBinlogAvailable(config, offset)) { + if (!connection.isBinlogPositionAvailable(config, offset.gtidSet(), offset.getSource().binlogFilename())) { throw new DebeziumException("The connector is trying to read binlog starting at " + offset.getSource() + ", but this is no longer " + "available on the server. Reconfigure the connector to use a snapshot when needed."); } @@ -388,7 +329,7 @@ private boolean validateSnapshotFeasibility(MySqlConnectorConfig config, MySqlOf } else { // But check to see if the server still has those binlog coordinates ... - if (!isBinlogAvailable(config, offset)) { + if (!connection.isBinlogPositionAvailable(config, offset.gtidSet(), offset.getSource().binlogFilename())) { if (!config.getSnapshotMode().shouldSnapshotOnDataError()) { throw new DebeziumException("The connector is trying to read binlog starting at " + offset.getSource() + ", but this is no longer " + "available on the server. Reconfigure the connector to use a snapshot when needed."); @@ -422,10 +363,8 @@ private void resetOffset(MySqlConnectorConfig connectorConfig, MySqlOffsetContex SignalProcessor signalProcessor) { boolean isKafkaChannelEnabled = connectorConfig.getEnabledChannels().contains(KafkaSignalChannel.CHANNEL_NAME); if (previousOffset != null && isKafkaChannelEnabled && connectorConfig.isReadOnlyConnection()) { - MySqlReadOnlyIncrementalSnapshotContext readOnlyIncrementalSnapshotContext = (MySqlReadOnlyIncrementalSnapshotContext) previousOffset - .getIncrementalSnapshotContext(); KafkaSignalChannel kafkaSignal = signalProcessor.getSignalChannel(KafkaSignalChannel.class); - Long signalOffset = readOnlyIncrementalSnapshotContext.getSignalOffset(); + Long signalOffset = connectorConfig.getConnectorAdapter().getReadOnlyIncrementalSnapshotSignalOffset(previousOffset); if (signalOffset != null) { LOGGER.info("Resetting Kafka Signal offset to {}", signalOffset); kafkaSignal.reset(signalOffset); diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlOffsetContext.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlOffsetContext.java index f2eddc1c5..9b64c27a8 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlOffsetContext.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlOffsetContext.java @@ -14,6 +14,7 @@ import org.apache.kafka.connect.errors.ConnectException; import io.debezium.connector.SnapshotRecord; +import io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotContext; import io.debezium.pipeline.CommonOffsetContext; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext; @@ -62,7 +63,8 @@ public MySqlOffsetContext(boolean snapshot, boolean snapshotCompleted, Transacti public MySqlOffsetContext(MySqlConnectorConfig connectorConfig, boolean snapshot, boolean snapshotCompleted, SourceInfo sourceInfo) { this(snapshot, snapshotCompleted, new TransactionContext(), - connectorConfig.isReadOnlyConnection() ? new MySqlReadOnlyIncrementalSnapshotContext<>() : new SignalBasedIncrementalSnapshotContext<>(), + connectorConfig.getConnectorAdapter().getIncrementalSnapshotContext(), + // connectorConfig.isReadOnlyConnection() ? new MySqlReadOnlyIncrementalSnapshotContext<>() : new SignalBasedIncrementalSnapshotContext<>(), sourceInfo); } diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlSnapshotChangeEventSource.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlSnapshotChangeEventSource.java index 8f23d78cc..7eff0a3ad 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlSnapshotChangeEventSource.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlSnapshotChangeEventSource.java @@ -38,8 +38,9 @@ import io.debezium.DebeziumException; import io.debezium.connector.SnapshotRecord; -import io.debezium.connector.mysql.MySqlConnection.DatabaseLocales; import io.debezium.connector.mysql.MySqlOffsetContext.Loader; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection.DatabaseLocales; import io.debezium.data.Envelope; import io.debezium.function.BlockingConsumer; import io.debezium.jdbc.JdbcConnection; @@ -61,7 +62,7 @@ public class MySqlSnapshotChangeEventSource extends RelationalSnapshotChangeEven private static final Logger LOGGER = LoggerFactory.getLogger(MySqlSnapshotChangeEventSource.class); private final MySqlConnectorConfig connectorConfig; - private final MySqlConnection connection; + private final AbstractConnectorConnection connection; private long globalLockAcquiredAt = -1; private long tableLockAcquiredAt = -1; private final RelationalTableFilters filters; @@ -72,7 +73,7 @@ public class MySqlSnapshotChangeEventSource extends RelationalSnapshotChangeEven private final BlockingConsumer> lastEventProcessor; private final Runnable preSnapshotAction; - public MySqlSnapshotChangeEventSource(MySqlConnectorConfig connectorConfig, MainConnectionProvidingConnectionFactory connectionFactory, + public MySqlSnapshotChangeEventSource(MySqlConnectorConfig connectorConfig, MainConnectionProvidingConnectionFactory connectionFactory, MySqlDatabaseSchema schema, EventDispatcher dispatcher, Clock clock, MySqlSnapshotChangeEventSourceMetrics metrics, BlockingConsumer> lastEventProcessor, @@ -359,7 +360,7 @@ protected void readTableStructure(ChangeEventSourceContext sourceContext, if (!snapshottingTask.isBlocking()) { // Record default charset - addSchemaEvent(snapshotContext, "", connection.setStatementFor(connection.readMySqlCharsetSystemVariables())); + addSchemaEvent(snapshotContext, "", connection.setStatementFor(connection.readCharsetSystemVariables())); } for (TableId tableId : capturedSchemaTables) { @@ -584,7 +585,7 @@ protected OptionalLong rowCountForTable(TableId tableId) { @Override protected Statement readTableStatement(JdbcConnection jdbcConnection, OptionalLong rowCount) throws SQLException { - MySqlConnection connection = (MySqlConnection) jdbcConnection; + AbstractConnectorConnection connection = (AbstractConnectorConnection) jdbcConnection; final long largeTableRowCount = connectorConfig.rowCountForLargeTable(); if (rowCount.isEmpty() || largeTableRowCount == 0 || rowCount.getAsLong() <= largeTableRowCount) { return super.readTableStatement(connection, rowCount); @@ -610,7 +611,7 @@ protected Statement readTableStatement(JdbcConnection jdbcConnection, OptionalLo * @return the statement; never null * @throws SQLException if there is a problem creating the statement */ - private Statement createStatementWithLargeResultSet(MySqlConnection connection) throws SQLException { + private Statement createStatementWithLargeResultSet(AbstractConnectorConnection connection) throws SQLException { int fetchSize = connectorConfig.getSnapshotFetchSize(); Statement stmt = connection.connection().createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY); stmt.setFetchSize(fetchSize); diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java index b3ad7f6b2..e152efe0a 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java @@ -5,21 +5,11 @@ */ package io.debezium.connector.mysql; -import static io.debezium.util.Strings.isNullOrEmpty; - import java.io.IOException; -import java.security.GeneralSecurityException; -import java.security.KeyStore; -import java.security.KeyStoreException; -import java.security.NoSuchAlgorithmException; -import java.security.UnrecoverableKeyException; -import java.security.cert.CertificateException; -import java.security.cert.X509Certificate; import java.sql.SQLException; import java.time.Duration; import java.time.Instant; import java.util.EnumMap; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -29,13 +19,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.Predicate; -import javax.net.ssl.KeyManager; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.SSLContext; -import javax.net.ssl.TrustManager; -import javax.net.ssl.TrustManagerFactory; -import javax.net.ssl.X509TrustManager; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; import org.slf4j.event.Level; @@ -57,14 +40,9 @@ import com.github.shyiko.mysql.binlog.event.TransactionPayloadEventData; import com.github.shyiko.mysql.binlog.event.UpdateRowsEventData; import com.github.shyiko.mysql.binlog.event.WriteRowsEventData; -import com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException; import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; -import com.github.shyiko.mysql.binlog.event.deserialization.GtidEventDataDeserializer; -import com.github.shyiko.mysql.binlog.io.ByteArrayInputStream; import com.github.shyiko.mysql.binlog.network.AuthenticationException; -import com.github.shyiko.mysql.binlog.network.DefaultSSLSocketFactory; import com.github.shyiko.mysql.binlog.network.SSLMode; -import com.github.shyiko.mysql.binlog.network.SSLSocketFactory; import com.github.shyiko.mysql.binlog.network.ServerException; import io.debezium.DebeziumException; @@ -72,6 +50,8 @@ import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; import io.debezium.config.Configuration; import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; import io.debezium.data.Envelope.Operation; import io.debezium.function.BlockingConsumer; import io.debezium.pipeline.ErrorHandler; @@ -112,13 +92,13 @@ public class MySqlStreamingChangeEventSource implements StreamingChangeEventSour private final AtomicLong totalRecordCounter = new AtomicLong(); private volatile Map lastOffset = null; private com.github.shyiko.mysql.binlog.GtidSet gtidSet; - private final float heartbeatIntervalFactor = 0.8f; private final Map binaryLogClientThreads = new ConcurrentHashMap<>(4); private final MySqlTaskContext taskContext; private final MySqlConnectorConfig connectorConfig; - private final MySqlConnection connection; + private final AbstractConnectorConnection connection; private final EventDispatcher eventDispatcher; private final ErrorHandler errorHandler; + private final ConnectorAdapter connectorAdapter; @SingleThreadAccess("binlog client thread") private Instant eventTimestamp; @@ -184,7 +164,7 @@ private interface BinlogChangeEmitter { void emit(TableId tableId, T data) throws InterruptedException; } - public MySqlStreamingChangeEventSource(MySqlConnectorConfig connectorConfig, MySqlConnection connection, + public MySqlStreamingChangeEventSource(MySqlConnectorConfig connectorConfig, AbstractConnectorConnection connection, EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, MySqlTaskContext taskContext, MySqlStreamingChangeEventSourceMetrics metrics) { @@ -195,131 +175,21 @@ public MySqlStreamingChangeEventSource(MySqlConnectorConfig connectorConfig, MyS this.eventDispatcher = dispatcher; this.errorHandler = errorHandler; this.metrics = metrics; + this.connectorAdapter = connectorConfig.getConnectorAdapter(); eventDeserializationFailureHandlingMode = connectorConfig.getEventProcessingFailureHandlingMode(); inconsistentSchemaHandlingMode = connectorConfig.inconsistentSchemaFailureHandlingMode(); // Set up the log reader ... - client = taskContext.getBinaryLogClient(); - // BinaryLogClient will overwrite thread names later - client.setThreadFactory( + client = connectorAdapter.getBinaryLogClientConfigurator().configure( + taskContext.getBinaryLogClient(), Threads.threadFactory(MySqlConnector.class, connectorConfig.getLogicalName(), "binlog-client", false, false, - x -> binaryLogClientThreads.put(x.getName(), x))); - client.setServerId(connectorConfig.serverId()); - client.setSSLMode(sslModeFor(connectorConfig.sslMode())); - if (connectorConfig.sslModeEnabled()) { - SSLSocketFactory sslSocketFactory = getBinlogSslSocketFactory(connectorConfig, connection); - if (sslSocketFactory != null) { - client.setSslSocketFactory(sslSocketFactory); - } - } - if (connection.isMariaDb()) { - // This makes sure BEGIN events are emitted via QUERY events rather than GTIDs. - client.setMariaDbSlaveCapability(2); - } - Configuration configuration = connectorConfig.getConfig(); - client.setKeepAlive(configuration.getBoolean(MySqlConnectorConfig.KEEP_ALIVE)); - final long keepAliveInterval = configuration.getLong(MySqlConnectorConfig.KEEP_ALIVE_INTERVAL_MS); - client.setKeepAliveInterval(keepAliveInterval); - // Considering heartbeatInterval should be less than keepAliveInterval, we use the heartbeatIntervalFactor - // multiply by keepAliveInterval and set the result value to heartbeatInterval.The default value of heartbeatIntervalFactor - // is 0.8, and we believe the left time (0.2 * keepAliveInterval) is enough to process the packet received from the MySQL server. - client.setHeartbeatInterval((long) (keepAliveInterval * heartbeatIntervalFactor)); + x -> binaryLogClientThreads.put(x.getName(), x)), + connection); + Configuration configuration = connectorConfig.getConfig(); boolean filterDmlEventsByGtidSource = configuration.getBoolean(MySqlConnectorConfig.GTID_SOURCE_FILTER_DML_EVENTS); gtidDmlSourceFilter = filterDmlEventsByGtidSource ? connectorConfig.gtidSourceFilter() : null; - - // Set up the event deserializer with additional type(s) ... - final Map tableMapEventByTableId = new HashMap(); - EventDeserializer eventDeserializer = new EventDeserializer() { - @Override - public Event nextEvent(ByteArrayInputStream inputStream) throws IOException { - try { - // Delegate to the superclass ... - Event event = super.nextEvent(inputStream); - - // We have to record the most recent TableMapEventData for each table number for our custom deserializers ... - if (event.getHeader().getEventType() == EventType.TABLE_MAP) { - TableMapEventData tableMapEvent = event.getData(); - tableMapEventByTableId.put(tableMapEvent.getTableId(), tableMapEvent); - } - - // DBZ-2663 Handle for transaction payload and capture the table map event and add it to the map - if (event.getHeader().getEventType() == EventType.TRANSACTION_PAYLOAD) { - TransactionPayloadEventData transactionPayloadEventData = (TransactionPayloadEventData) event.getData(); - /** - * Loop over the uncompressed events in the transaction payload event and add the table map - * event in the map of table events - **/ - for (Event uncompressedEvent : transactionPayloadEventData.getUncompressedEvents()) { - if (uncompressedEvent.getHeader().getEventType() == EventType.TABLE_MAP - && uncompressedEvent.getData() != null) { - TableMapEventData tableMapEvent = (TableMapEventData) uncompressedEvent.getData(); - tableMapEventByTableId.put(tableMapEvent.getTableId(), tableMapEvent); - } - } - } - - // DBZ-5126 Clean cache on rotate event to prevent it from growing indefinitely. - if (event.getHeader().getEventType() == EventType.ROTATE && event.getHeader().getTimestamp() != 0) { - tableMapEventByTableId.clear(); - } - return event; - } - // DBZ-217 In case an event couldn't be read we create a pseudo-event for the sake of logging - catch (EventDataDeserializationException edde) { - // DBZ-3095 As of Java 15, when reaching EOF in the binlog stream, the polling loop in - // BinaryLogClient#listenForEventPackets() keeps returning values != -1 from peek(); - // this causes the loop to never finish - // Propagating the exception (either EOF or socket closed) causes the loop to be aborted - // in this case - if (edde.getCause() instanceof IOException) { - throw edde; - } - - EventHeaderV4 header = new EventHeaderV4(); - header.setEventType(EventType.INCIDENT); - header.setTimestamp(edde.getEventHeader().getTimestamp()); - header.setServerId(edde.getEventHeader().getServerId()); - - if (edde.getEventHeader() instanceof EventHeaderV4) { - header.setEventLength(((EventHeaderV4) edde.getEventHeader()).getEventLength()); - header.setNextPosition(((EventHeaderV4) edde.getEventHeader()).getNextPosition()); - header.setFlags(((EventHeaderV4) edde.getEventHeader()).getFlags()); - } - - EventData data = new EventDataDeserializationExceptionData(edde); - return new Event(header, data); - } - } - }; - - // Add our custom deserializers ... - eventDeserializer.setEventDataDeserializer(EventType.STOP, new StopEventDataDeserializer()); - eventDeserializer.setEventDataDeserializer(EventType.GTID, new GtidEventDataDeserializer()); - eventDeserializer.setEventDataDeserializer(EventType.WRITE_ROWS, - new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); - eventDeserializer.setEventDataDeserializer(EventType.UPDATE_ROWS, - new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); - eventDeserializer.setEventDataDeserializer(EventType.DELETE_ROWS, - new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); - eventDeserializer.setEventDataDeserializer(EventType.EXT_WRITE_ROWS, - new RowDeserializers.WriteRowsDeserializer( - tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); - eventDeserializer.setEventDataDeserializer(EventType.EXT_UPDATE_ROWS, - new RowDeserializers.UpdateRowsDeserializer( - tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); - eventDeserializer.setEventDataDeserializer(EventType.EXT_DELETE_ROWS, - new RowDeserializers.DeleteRowsDeserializer( - tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); - eventDeserializer.setEventDataDeserializer(EventType.TRANSACTION_PAYLOAD, - new TransactionPayloadDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); - - if (connection.isMariaDb()) { - eventDeserializer.setCompatibilityMode(EventDeserializer.CompatibilityMode.CHAR_AND_BINARY_AS_BYTE_ARRAY); - } - - client.setEventDeserializer(eventDeserializer); } protected void onEvent(MySqlOffsetContext offsetContext, Event event) { @@ -547,19 +417,8 @@ protected void handleGtidEvent(MySqlOffsetContext offsetContext, Event event) { * @param event the database change data event to be processed; may not be null */ protected void handleRecordingQuery(MySqlOffsetContext offsetContext, Event event) { - final String query; - if (!connection.isMariaDb()) { - // Unwrap the RowsQueryEvent - final RowsQueryEventData lastRowsQueryEventData = unwrapData(event); - query = lastRowsQueryEventData.getQuery(); - } - else { - // Unwrap the AnnotateRowsEventData - final AnnotateRowsEventData annotateRowsEventData = unwrapData(event); - query = annotateRowsEventData.getRowsQuery(); - } // Set the query on the source - offsetContext.setQuery(query); + offsetContext.setQuery(connectorAdapter.getRecordingQueryFromEvent(unwrapData(event))); } /** @@ -975,16 +834,8 @@ public void execute(ChangeEventSourceContext context, MySqlPartition partition, // Conditionally register ROWS_QUERY handler to parse SQL statements. if (connectorConfig.includeSqlQuery()) { - if (!connection.isMariaDb()) { - eventHandlers.put(EventType.ROWS_QUERY, (event) -> handleRecordingQuery(effectiveOffsetContext, event)); - } - else { - // Binlog client explicitly needs to be told to enable ANNOTATE_ROWS events, which is the - // MariaDB equivalent of ROWS_QUERY. This must be done ahead of the connection to make - // sure that the right negotiation bits are set during handshake. - client.setUseSendAnnotateRowsEvent(true); - eventHandlers.put(EventType.ANNOTATE_ROWS, (event) -> handleRecordingQuery(effectiveOffsetContext, event)); - } + final EventType eventType = connectorAdapter.getBinaryLogClientConfigurator().getIncludeSqlQueryEventType(); + eventHandlers.put(eventType, (event) -> handleRecordingQuery(effectiveOffsetContext, event)); } BinaryLogClient.EventListener listener; @@ -1007,19 +858,19 @@ public void execute(ChangeEventSourceContext context, MySqlPartition partition, metrics.setIsGtidModeEnabled(isGtidModeEnabled); // Get the current GtidSet from MySQL so we can get a filtered/merged GtidSet based off of the last Debezium checkpoint. - String availableServerGtidStr = connection.knownGtidSet(); if (isGtidModeEnabled) { // The server is using GTIDs, so enable the handler ... eventHandlers.put(EventType.GTID, (event) -> handleGtidEvent(effectiveOffsetContext, event)); // Now look at the GTID set from the server and what we've previously seen ... - GtidSet availableServerGtidSet = new GtidSet(availableServerGtidStr); + GtidSet availableServerGtidSet = connection.knownGtidSet(); // also take into account purged GTID logs GtidSet purgedServerGtidSet = connection.purgedGtidSet(); LOGGER.info("GTID set purged on server: {}", purgedServerGtidSet); - GtidSet filteredGtidSet = filterGtidSet(effectiveOffsetContext, availableServerGtidSet, purgedServerGtidSet); + GtidSet filteredGtidSet = connection.filterGtidSet(connectorConfig.gtidSourceFilter(), + effectiveOffsetContext.gtidSet(), availableServerGtidSet, purgedServerGtidSet); if (filteredGtidSet != null) { // We've seen at least some GTIDs, so start reading from the filtered GTID set ... LOGGER.info("Registering binlog reader with GTID set: {}", filteredGtidSet); @@ -1130,82 +981,6 @@ public MySqlOffsetContext getOffsetContext() { return effectiveOffsetContext; } - private SSLSocketFactory getBinlogSslSocketFactory(MySqlConnectorConfig connectorConfig, MySqlConnection connection) { - String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); - if (!isNullOrEmpty(acceptedTlsVersion)) { - SSLMode sslMode = sslModeFor(connectorConfig.sslMode()); - LOGGER.info("Enable ssl " + sslMode + " mode for connector " + connectorConfig.getLogicalName()); - - final char[] keyPasswordArray = connection.connectionConfig().sslKeyStorePassword(); - final String keyFilename = connection.connectionConfig().sslKeyStore(); - final char[] trustPasswordArray = connection.connectionConfig().sslTrustStorePassword(); - final String trustFilename = connection.connectionConfig().sslTrustStore(); - KeyManager[] keyManagers = null; - if (keyFilename != null) { - try { - KeyStore ks = connection.loadKeyStore(keyFilename, keyPasswordArray); - - KeyManagerFactory kmf = KeyManagerFactory.getInstance("NewSunX509"); - kmf.init(ks, keyPasswordArray); - - keyManagers = kmf.getKeyManagers(); - } - catch (KeyStoreException | NoSuchAlgorithmException | UnrecoverableKeyException e) { - throw new DebeziumException("Could not load keystore", e); - } - } - TrustManager[] trustManagers; - try { - KeyStore ks = null; - if (trustFilename != null) { - ks = connection.loadKeyStore(trustFilename, trustPasswordArray); - } - - if (ks == null && (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED)) { - trustManagers = new TrustManager[]{ - new X509TrustManager() { - - @Override - public void checkClientTrusted(X509Certificate[] x509Certificates, String s) - throws CertificateException { - } - - @Override - public void checkServerTrusted(X509Certificate[] x509Certificates, String s) - throws CertificateException { - } - - @Override - public X509Certificate[] getAcceptedIssuers() { - return new X509Certificate[0]; - } - } - }; - } - else { - TrustManagerFactory tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); - tmf.init(ks); - trustManagers = tmf.getTrustManagers(); - } - } - catch (KeyStoreException | NoSuchAlgorithmException e) { - throw new DebeziumException("Could not load truststore", e); - } - // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that - // the accepted TLS version is passed to the constructed factory - final KeyManager[] finalKMS = keyManagers; - return new DefaultSSLSocketFactory(acceptedTlsVersion) { - - @Override - protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { - sc.init(finalKMS, trustManagers, null); - } - }; - } - - return null; - } - private void logStreamingSourceState() { logStreamingSourceState(Level.ERROR); } @@ -1229,54 +1004,6 @@ private void logStreamingSourceState(Level severity) { } } - /** - * Apply the include/exclude GTID source filters to the current {@link MySqlOffsetContext#gtidSet() GTID set} and merge them onto the - * currently available GTID set from a MySQL server. - * - * The merging behavior of this method might seem a bit strange at first. It's required in order for Debezium to consume a - * MySQL binlog that has multi-source replication enabled, if a failover has to occur. In such a case, the server that - * Debezium is failed over to might have a different set of sources, but still include the sources required for Debezium - * to continue to function. MySQL does not allow downstream replicas to connect if the GTID set does not contain GTIDs for - * all channels that the server is replicating from, even if the server does have the data needed by the client. To get - * around this, we can have Debezium merge its GTID set with whatever is on the server, so that MySQL will allow it to - * connect. See DBZ-143 for details. - * - * This method does not mutate any state in the context. - * - * @param availableServerGtidSet the GTID set currently available in the MySQL server - * @param purgedServerGtid the GTID set already purged by the MySQL server - * @return A GTID set meant for consuming from a MySQL binlog; may return null if the SourceInfo has no GTIDs and therefore - * none were filtered - */ - public GtidSet filterGtidSet(MySqlOffsetContext offsetContext, GtidSet availableServerGtidSet, GtidSet purgedServerGtid) { - String gtidStr = offsetContext.gtidSet(); - if (gtidStr == null) { - return null; - } - LOGGER.info("Attempting to generate a filtered GTID set"); - LOGGER.info("GTID set from previous recorded offset: {}", gtidStr); - GtidSet filteredGtidSet = new GtidSet(gtidStr); - Predicate gtidSourceFilter = connectorConfig.gtidSourceFilter(); - if (gtidSourceFilter != null) { - filteredGtidSet = filteredGtidSet.retainAll(gtidSourceFilter); - LOGGER.info("GTID set after applying GTID source includes/excludes to previous recorded offset: {}", filteredGtidSet); - } - LOGGER.info("GTID set available on server: {}", availableServerGtidSet); - - final GtidSet knownGtidSet = filteredGtidSet; - LOGGER.info("Using first available positions for new GTID channels"); - final GtidSet relevantAvailableServerGtidSet = (gtidSourceFilter != null) ? availableServerGtidSet.retainAll(gtidSourceFilter) : availableServerGtidSet; - LOGGER.info("Relevant GTID set available on server: {}", relevantAvailableServerGtidSet); - - GtidSet mergedGtidSet = relevantAvailableServerGtidSet - .retainAll(uuid -> knownGtidSet.forServerWithId(uuid) != null) - .with(purgedServerGtid) - .with(filteredGtidSet); - - LOGGER.info("Final merged GTID set to use when connecting to MySQL: {}", mergedGtidSet); - return mergedGtidSet; - } - MySqlStreamingChangeEventSourceMetrics getMetrics() { return metrics; } diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java index ae3594781..ef60d0f06 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlValueConverters.java @@ -39,8 +39,11 @@ import io.debezium.DebeziumException; import io.debezium.annotation.Immutable; +import io.debezium.annotation.VisibleForTesting; import io.debezium.config.CommonConnectorConfig.BinaryHandlingMode; +import io.debezium.config.Configuration; import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; import io.debezium.data.Json; import io.debezium.data.SpecialValueDecimal; import io.debezium.jdbc.JdbcValueConverters; @@ -115,6 +118,7 @@ else if (70 <= year && year <= 99) { } private final ParsingErrorHandler parsingErrorHandler; + private final ConnectorAdapter connectorAdapter; /** * Create a new instance that always uses UTC for the default time zone when_needed converting values without timezone information @@ -128,9 +132,16 @@ else if (70 <= year && year <= 99) { * {@link io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode#PRECISE} is to be used * @param binaryMode how binary columns should be represented */ + @VisibleForTesting public MySqlValueConverters(DecimalMode decimalMode, TemporalPrecisionMode temporalPrecisionMode, BigIntUnsignedMode bigIntUnsignedMode, BinaryHandlingMode binaryMode) { - this(decimalMode, temporalPrecisionMode, bigIntUnsignedMode, binaryMode, x -> x, MySqlValueConverters::defaultParsingErrorHandler); + this(decimalMode, temporalPrecisionMode, bigIntUnsignedMode, binaryMode, x -> x, MySqlValueConverters::defaultParsingErrorHandler, resolveDefaultAdapter()); + } + + private static ConnectorAdapter resolveDefaultAdapter() { + Configuration config = Configuration.empty(); + MySqlConnectorConfig connectorConfig = new MySqlConnectorConfig(config); + return MySqlConnectorConfig.ConnectorAdapterMode.MYSQL.getAdapter(connectorConfig); } /** @@ -145,12 +156,14 @@ public MySqlValueConverters(DecimalMode decimalMode, TemporalPrecisionMode tempo * {@link io.debezium.jdbc.JdbcValueConverters.BigIntUnsignedMode#PRECISE} is to be used * @param binaryMode how binary columns should be represented * @param adjuster a temporal adjuster to make a database specific time modification before conversion + * @param connectorAdapter the connector adapter */ public MySqlValueConverters(DecimalMode decimalMode, TemporalPrecisionMode temporalPrecisionMode, BigIntUnsignedMode bigIntUnsignedMode, BinaryHandlingMode binaryMode, - TemporalAdjuster adjuster, ParsingErrorHandler parsingErrorHandler) { + TemporalAdjuster adjuster, ParsingErrorHandler parsingErrorHandler, ConnectorAdapter connectorAdapter) { super(decimalMode, temporalPrecisionMode, ZoneOffset.UTC, adjuster, bigIntUnsignedMode, binaryMode); this.parsingErrorHandler = parsingErrorHandler; + this.connectorAdapter = connectorAdapter; } @Override @@ -331,10 +344,10 @@ protected Charset charsetFor(Column column) { logger.warn("Column is missing a character set: {}", column); return null; } - String encoding = MySqlConnection.getJavaEncodingForMysqlCharSet(mySqlCharsetName); + String encoding = connectorAdapter.getJavaEncodingForCharSet(mySqlCharsetName); if (encoding == null) { logger.debug("Column uses MySQL character set '{}', which has no mapping to a Java character set, will try it in lowercase", mySqlCharsetName); - encoding = MySqlConnection.getJavaEncodingForMysqlCharSet(mySqlCharsetName.toLowerCase()); + encoding = connectorAdapter.getJavaEncodingForCharSet(mySqlCharsetName.toLowerCase()); } if (encoding == null) { logger.warn("Column uses MySQL character set '{}', which has no mapping to a Java character set", mySqlCharsetName); diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractBinaryLogClientConfigurator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractBinaryLogClientConfigurator.java new file mode 100644 index 000000000..aadae489e --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractBinaryLogClientConfigurator.java @@ -0,0 +1,295 @@ +/* + * 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.mysql.strategy; + +import static io.debezium.util.Strings.isNullOrEmpty; + +import java.io.IOException; +import java.security.GeneralSecurityException; +import java.security.KeyStore; +import java.security.KeyStoreException; +import java.security.NoSuchAlgorithmException; +import java.security.UnrecoverableKeyException; +import java.security.cert.CertificateException; +import java.security.cert.X509Certificate; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ThreadFactory; + +import javax.net.ssl.KeyManager; +import javax.net.ssl.KeyManagerFactory; +import javax.net.ssl.SSLContext; +import javax.net.ssl.TrustManager; +import javax.net.ssl.TrustManagerFactory; +import javax.net.ssl.X509TrustManager; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.Event; +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.EventHeaderV4; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.TableMapEventData; +import com.github.shyiko.mysql.binlog.event.TransactionPayloadEventData; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDataDeserializationException; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; +import com.github.shyiko.mysql.binlog.event.deserialization.GtidEventDataDeserializer; +import com.github.shyiko.mysql.binlog.io.ByteArrayInputStream; +import com.github.shyiko.mysql.binlog.network.DefaultSSLSocketFactory; +import com.github.shyiko.mysql.binlog.network.SSLMode; +import com.github.shyiko.mysql.binlog.network.SSLSocketFactory; + +import io.debezium.DebeziumException; +import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode; +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.EventDataDeserializationExceptionData; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode; +import io.debezium.connector.mysql.RowDeserializers; +import io.debezium.connector.mysql.StopEventDataDeserializer; +import io.debezium.connector.mysql.TransactionPayloadDeserializer; + +/** + * @author Chris Cranford + */ +public abstract class AbstractBinaryLogClientConfigurator implements BinaryLogClientConfigurator { + + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractBinaryLogClientConfigurator.class); + + private final MySqlConnectorConfig connectorConfig; + private final float heartbeatIntervalFactor = 0.8f; + private final EventProcessingFailureHandlingMode eventDeserializationFailureHandlingMode; + + public AbstractBinaryLogClientConfigurator(MySqlConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + this.eventDeserializationFailureHandlingMode = connectorConfig.getEventProcessingFailureHandlingMode(); + } + + @Override + public BinaryLogClient configure(BinaryLogClient client, ThreadFactory threadFactory, AbstractConnectorConnection connection) { + client.setThreadFactory(threadFactory); + client.setServerId(connectorConfig.serverId()); + client.setSSLMode(sslModeFor(connectorConfig.sslMode())); + + if (connectorConfig.sslModeEnabled()) { + SSLSocketFactory sslSocketFactory = getBinlogSslSocketFactory(connectorConfig, connection); + if (sslSocketFactory != null) { + client.setSslSocketFactory(sslSocketFactory); + } + } + + configureReplicaCompatibility(client); + + Configuration configuration = connectorConfig.getConfig(); + client.setKeepAlive(configuration.getBoolean(MySqlConnectorConfig.KEEP_ALIVE)); + final long keepAliveInterval = configuration.getLong(MySqlConnectorConfig.KEEP_ALIVE_INTERVAL_MS); + client.setKeepAliveInterval(keepAliveInterval); + // Considering heartbeatInterval should be less than keepAliveInterval, we use the heartbeatIntervalFactor + // multiply by keepAliveInterval and set the result value to heartbeatInterval.The default value of heartbeatIntervalFactor + // is 0.8, and we believe the left time (0.2 * keepAliveInterval) is enough to process the packet received from the MySQL server. + client.setHeartbeatInterval((long) (keepAliveInterval * heartbeatIntervalFactor)); + + client.setEventDeserializer(createEventDeserializer()); + + return client; + } + + protected EventDeserializer createEventDeserializer() { + // Set up the event deserializer with additional type(s) ... + final Map tableMapEventByTableId = new HashMap<>(); + EventDeserializer eventDeserializer = new EventDeserializer() { + @Override + public Event nextEvent(ByteArrayInputStream inputStream) throws IOException { + try { + // Delegate to the superclass ... + Event event = super.nextEvent(inputStream); + + // We have to record the most recent TableMapEventData for each table number for our custom deserializers ... + if (event.getHeader().getEventType() == EventType.TABLE_MAP) { + TableMapEventData tableMapEvent = event.getData(); + tableMapEventByTableId.put(tableMapEvent.getTableId(), tableMapEvent); + } + + // DBZ-2663 Handle for transaction payload and capture the table map event and add it to the map + if (event.getHeader().getEventType() == EventType.TRANSACTION_PAYLOAD) { + TransactionPayloadEventData transactionPayloadEventData = (TransactionPayloadEventData) event.getData(); + /** + * Loop over the uncompressed events in the transaction payload event and add the table map + * event in the map of table events + **/ + for (Event uncompressedEvent : transactionPayloadEventData.getUncompressedEvents()) { + if (uncompressedEvent.getHeader().getEventType() == EventType.TABLE_MAP + && uncompressedEvent.getData() != null) { + TableMapEventData tableMapEvent = (TableMapEventData) uncompressedEvent.getData(); + tableMapEventByTableId.put(tableMapEvent.getTableId(), tableMapEvent); + } + } + } + + // DBZ-5126 Clean cache on rotate event to prevent it from growing indefinitely. + if (event.getHeader().getEventType() == EventType.ROTATE && event.getHeader().getTimestamp() != 0) { + tableMapEventByTableId.clear(); + } + return event; + } + // DBZ-217 In case an event couldn't be read we create a pseudo-event for the sake of logging + catch (EventDataDeserializationException edde) { + // DBZ-3095 As of Java 15, when reaching EOF in the binlog stream, the polling loop in + // BinaryLogClient#listenForEventPackets() keeps returning values != -1 from peek(); + // this causes the loop to never finish + // Propagating the exception (either EOF or socket closed) causes the loop to be aborted + // in this case + if (edde.getCause() instanceof IOException) { + throw edde; + } + + EventHeaderV4 header = new EventHeaderV4(); + header.setEventType(EventType.INCIDENT); + header.setTimestamp(edde.getEventHeader().getTimestamp()); + header.setServerId(edde.getEventHeader().getServerId()); + + if (edde.getEventHeader() instanceof EventHeaderV4) { + header.setEventLength(((EventHeaderV4) edde.getEventHeader()).getEventLength()); + header.setNextPosition(((EventHeaderV4) edde.getEventHeader()).getNextPosition()); + header.setFlags(((EventHeaderV4) edde.getEventHeader()).getFlags()); + } + + EventData data = new EventDataDeserializationExceptionData(edde); + return new Event(header, data); + } + } + }; + + // Add our custom deserializers ... + eventDeserializer.setEventDataDeserializer(EventType.STOP, new StopEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer(EventType.GTID, new GtidEventDataDeserializer()); + eventDeserializer.setEventDataDeserializer(EventType.WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); + eventDeserializer.setEventDataDeserializer(EventType.UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); + eventDeserializer.setEventDataDeserializer(EventType.DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); + eventDeserializer.setEventDataDeserializer(EventType.EXT_WRITE_ROWS, + new RowDeserializers.WriteRowsDeserializer( + tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer(EventType.EXT_UPDATE_ROWS, + new RowDeserializers.UpdateRowsDeserializer( + tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer(EventType.EXT_DELETE_ROWS, + new RowDeserializers.DeleteRowsDeserializer( + tableMapEventByTableId, eventDeserializationFailureHandlingMode).setMayContainExtraInformation(true)); + eventDeserializer.setEventDataDeserializer(EventType.TRANSACTION_PAYLOAD, + new TransactionPayloadDeserializer(tableMapEventByTableId, eventDeserializationFailureHandlingMode)); + + return eventDeserializer; + } + + @Override + public EventType getIncludeSqlQueryEventType() { + return EventType.ROWS_QUERY; + } + + protected MySqlConnectorConfig getConnectorConfig() { + return connectorConfig; + } + + protected void configureReplicaCompatibility(BinaryLogClient client) { + // default is a no-op + } + + private SSLMode sslModeFor(SecureConnectionMode mode) { + switch (mode) { + case DISABLED: + return SSLMode.DISABLED; + case PREFERRED: + return SSLMode.PREFERRED; + case REQUIRED: + return SSLMode.REQUIRED; + case VERIFY_CA: + return SSLMode.VERIFY_CA; + case VERIFY_IDENTITY: + return SSLMode.VERIFY_IDENTITY; + } + return null; + } + + private SSLSocketFactory getBinlogSslSocketFactory(MySqlConnectorConfig connectorConfig, AbstractConnectorConnection connection) { + String acceptedTlsVersion = connection.getSessionVariableForSslVersion(); + if (!isNullOrEmpty(acceptedTlsVersion)) { + SSLMode sslMode = sslModeFor(connectorConfig.sslMode()); + LOGGER.info("Enable ssl " + sslMode + " mode for connector " + connectorConfig.getLogicalName()); + + final char[] keyPasswordArray = connection.connectionConfig().sslKeyStorePassword(); + final String keyFilename = connection.connectionConfig().sslKeyStore(); + final char[] trustPasswordArray = connection.connectionConfig().sslTrustStorePassword(); + final String trustFilename = connection.connectionConfig().sslTrustStore(); + KeyManager[] keyManagers = null; + if (keyFilename != null) { + try { + KeyStore ks = connection.loadKeyStore(keyFilename, keyPasswordArray); + + KeyManagerFactory kmf = KeyManagerFactory.getInstance("NewSunX509"); + kmf.init(ks, keyPasswordArray); + + keyManagers = kmf.getKeyManagers(); + } + catch (KeyStoreException | NoSuchAlgorithmException | UnrecoverableKeyException e) { + throw new DebeziumException("Could not load keystore", e); + } + } + TrustManager[] trustManagers; + try { + KeyStore ks = null; + if (trustFilename != null) { + ks = connection.loadKeyStore(trustFilename, trustPasswordArray); + } + + if (ks == null && (sslMode == SSLMode.PREFERRED || sslMode == SSLMode.REQUIRED)) { + trustManagers = new TrustManager[]{ + new X509TrustManager() { + @Override + public void checkClientTrusted(X509Certificate[] x509Certificates, String s) + throws CertificateException { + } + + @Override + public void checkServerTrusted(X509Certificate[] x509Certificates, String s) + throws CertificateException { + } + + @Override + public X509Certificate[] getAcceptedIssuers() { + return new X509Certificate[0]; + } + } + }; + } + else { + TrustManagerFactory tmf = TrustManagerFactory.getInstance(TrustManagerFactory.getDefaultAlgorithm()); + tmf.init(ks); + trustManagers = tmf.getTrustManagers(); + } + } + catch (KeyStoreException | NoSuchAlgorithmException e) { + throw new DebeziumException("Could not load truststore", e); + } + // DBZ-1208 Resembles the logic from the upstream BinaryLogClient, only that + // the accepted TLS version is passed to the constructed factory + final KeyManager[] finalKMS = keyManagers; + return new DefaultSSLSocketFactory(acceptedTlsVersion) { + + @Override + protected void initSSLContext(SSLContext sc) throws GeneralSecurityException { + sc.init(finalKMS, trustManagers, null); + } + }; + } + + return null; + } +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectionConfiguration.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectionConfiguration.java new file mode 100644 index 000000000..1dd30273c --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectionConfiguration.java @@ -0,0 +1,167 @@ +/* + * 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.mysql.strategy; + +import static io.debezium.config.CommonConnectorConfig.DATABASE_CONFIG_PREFIX; +import static io.debezium.config.CommonConnectorConfig.DRIVER_CONFIG_PREFIX; + +import java.time.Duration; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.config.Configuration; +import io.debezium.config.Field; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.util.Strings; + +/** + * @author Chris Cranford + */ +public abstract class AbstractConnectionConfiguration implements ConnectionConfiguration { + + public static final String URL_PATTERN = "${protocol}://${hostname}:${port}/?useInformationSchema=true&nullCatalogMeansCurrent=false&useUnicode=true&characterEncoding=UTF-8&characterSetResults=UTF-8&zeroDateTimeBehavior=CONVERT_TO_NULL&connectTimeout=${connectTimeout}"; + + private final JdbcConfiguration jdbcConfig; + private final JdbcConnection.ConnectionFactory factory; + private final Configuration config; + + public AbstractConnectionConfiguration(Configuration config) { + // Set up the JDBC connection without actually connecting, with extra MySQL-specific properties + // to give us better JDBC database metadata behavior, including using UTF-8 for the client-side character encoding + // per https://dev.mysql.com/doc/connector-j/5.1/en/connector-j-reference-charsets.html + this.config = config; + final boolean useSSL = sslModeEnabled(); + final Configuration dbConfig = config + .edit() + .withDefault(MySqlConnectorConfig.PORT, MySqlConnectorConfig.PORT.defaultValue()) + .withDefault(MySqlConnectorConfig.JDBC_PROTOCOL, MySqlConnectorConfig.JDBC_PROTOCOL.defaultValue()) + .build() + .subset(DATABASE_CONFIG_PREFIX, true) + .merge(config.subset(DRIVER_CONFIG_PREFIX, true)); + + final Configuration.Builder jdbcConfigBuilder = dbConfig + .edit() + .with("connectTimeout", Long.toString(getConnectionTimeout().toMillis())) + .with("sslMode", sslMode().getValue()); + + if (useSSL) { + if (!Strings.isNullOrBlank(sslTrustStore())) { + jdbcConfigBuilder.with("trustCertificateKeyStoreUrl", "file:" + sslTrustStore()); + } + if (sslTrustStorePassword() != null) { + jdbcConfigBuilder.with("trustCertificateKeyStorePassword", String.valueOf(sslTrustStorePassword())); + } + if (!Strings.isNullOrBlank(sslKeyStore())) { + jdbcConfigBuilder.with("clientCertificateKeyStoreUrl", "file:" + sslKeyStore()); + } + if (sslKeyStorePassword() != null) { + jdbcConfigBuilder.with("clientCertificateKeyStorePassword", String.valueOf(sslKeyStorePassword())); + } + } + + jdbcConfigBuilder.with(getConnectionTimeZonePropertyName(), resolveConnectionTimeZone(dbConfig)); + + // Set and remove options to prevent potential vulnerabilities + jdbcConfigBuilder + .with("allowLoadLocalInfile", "false") + .with("allowUrlInLocalInfile", "false") + .with("autoDeserialize", false) + .without("queryInterceptors"); + + this.jdbcConfig = JdbcConfiguration.adapt(jdbcConfigBuilder.build()); + String driverClassName = this.config.getString(MySqlConnectorConfig.JDBC_DRIVER); + Field protocol = MySqlConnectorConfig.JDBC_PROTOCOL; + + factory = JdbcConnection.patternBasedFactory(URL_PATTERN, driverClassName, getClass().getClassLoader(), protocol); + + } + + @Override + public JdbcConfiguration config() { + return jdbcConfig; + } + + @Override + public Configuration originalConfig() { + return config; + } + + @Override + public JdbcConnection.ConnectionFactory factory() { + return factory; + } + + @Override + public String username() { + return config.getString(MySqlConnectorConfig.USER); + } + + @Override + public String password() { + return config.getString(MySqlConnectorConfig.PASSWORD); + } + + @Override + public String hostname() { + return config.getString(MySqlConnectorConfig.HOSTNAME); + } + + @Override + public int port() { + return config.getInteger(MySqlConnectorConfig.PORT); + } + + @Override + public MySqlConnectorConfig.SecureConnectionMode sslMode() { + String mode = config.getString(MySqlConnectorConfig.SSL_MODE); + return MySqlConnectorConfig.SecureConnectionMode.parse(mode); + } + + @Override + public boolean sslModeEnabled() { + return sslMode() != MySqlConnectorConfig.SecureConnectionMode.DISABLED; + } + + public String sslKeyStore() { + return config.getString(MySqlConnectorConfig.SSL_KEYSTORE); + } + + public char[] sslKeyStorePassword() { + String password = config.getString(MySqlConnectorConfig.SSL_KEYSTORE_PASSWORD); + return Strings.isNullOrBlank(password) ? null : password.toCharArray(); + } + + public String sslTrustStore() { + return config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE); + } + + public char[] sslTrustStorePassword() { + String password = config.getString(MySqlConnectorConfig.SSL_TRUSTSTORE_PASSWORD); + return Strings.isNullOrBlank(password) ? null : password.toCharArray(); + } + + public Duration getConnectionTimeout() { + return Duration.ofMillis(config.getLong(MySqlConnectorConfig.CONNECTION_TIMEOUT_MS)); + } + + public CommonConnectorConfig.EventProcessingFailureHandlingMode eventProcessingFailureHandlingMode() { + String mode = config.getString(CommonConnectorConfig.EVENT_PROCESSING_FAILURE_HANDLING_MODE); + if (mode == null) { + mode = config.getString(MySqlConnectorConfig.EVENT_DESERIALIZATION_FAILURE_HANDLING_MODE); + } + return CommonConnectorConfig.EventProcessingFailureHandlingMode.parse(mode); + } + + public CommonConnectorConfig.EventProcessingFailureHandlingMode inconsistentSchemaHandlingMode() { + String mode = config.getString(MySqlConnectorConfig.INCONSISTENT_SCHEMA_HANDLING_MODE); + return CommonConnectorConfig.EventProcessingFailureHandlingMode.parse(mode); + } + + protected abstract String getConnectionTimeZonePropertyName(); + + protected abstract String resolveConnectionTimeZone(Configuration dbConfig); +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectorConnection.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectorConnection.java new file mode 100644 index 000000000..46eda9c73 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractConnectorConnection.java @@ -0,0 +1,502 @@ +/* + * 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.mysql.strategy; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.OptionalLong; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.DebeziumException; +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.MySqlFieldReader; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlSystemVariables; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.util.Strings; + +/** + * An abstract common implementation of {@link JdbcConnection} for MySQL and MariaDB. + * + * @author Jiri Pechanec, Randall Hauch, Chris Cranford + */ +public abstract class AbstractConnectorConnection extends JdbcConnection { + + private static final Logger LOGGER = LoggerFactory.getLogger(AbstractConnectorConnection.class); + + private static final String SQL_SHOW_SYSTEM_VARIABLES = "SHOW VARIABLES"; + private static final String SQL_SHOW_SYSTEM_VARIABLES_CHARACTER_SET = "SHOW VARIABLES WHERE Variable_name IN ('character_set_server','collation_server')"; + private static final String SQL_SHOW_SESSION_VARIABLE_SSL_VERSION = "SHOW SESSION STATUS LIKE 'Ssl_version'"; + private static final String QUOTED_CHARACTER = "`"; + + private final ConnectionConfiguration connectionConfig; + // todo: rename to drop the prefix on the interface?? + private final MySqlFieldReader fieldReader; + + /** + * Creates a new connection using the supplied configuration. + * + * @param configuration the connection configuration instance, may not be null + * @param fieldReader the configured snapshot fetch size + */ + public AbstractConnectorConnection(ConnectionConfiguration configuration, MySqlFieldReader fieldReader) { + super(configuration.config(), configuration.factory(), QUOTED_CHARACTER, QUOTED_CHARACTER); + this.connectionConfig = configuration; + this.fieldReader = fieldReader; + } + + @Override + public Object getColumnValue(ResultSet rs, int columnIndex, Column column, Table table) throws SQLException { + return fieldReader.readField(rs, columnIndex, column, table); + } + + @Override + public String quotedTableIdString(TableId tableId) { + return tableId.toQuotedString('`'); + } + + public String connectionString() { + return connectionString(AbstractConnectionConfiguration.URL_PATTERN); + } + + public ConnectionConfiguration connectionConfig() { + return connectionConfig; + } + + /** + * Determine whether the current user has the named privilege. If the user has the "ALL" privilege, this + * method will always return {@code true}. + * + * @param grantName the name of the database privilege; may not be null + * @return {@code true} if the user has the named privilege; {@code false} otherwise + */ + public boolean userHasPrivileges(String grantName) { + try { + return queryAndMap("SHOW GRANTS FOR CURRENT_USER", rs -> { + while (rs.next()) { + String grants = rs.getString(1); + LOGGER.debug(grants); + if (grants == null) { + return false; + } + grants = grants.toUpperCase(); + if (grants.contains("ALL") || grants.contains(grantName.toUpperCase())) { + return true; + } + } + return false; + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while connecting to database and looking at privileges for current user: ", e); + } + } + + /** + * Determine the earliest binlog filename that is still available in the server. + * + * @return the name of the earliest binlog filename, or null if there are none + */ + public String earliestBinlogFilename() { + // Accumulate the available binlog filenames ... + List logNames = new ArrayList<>(); + try { + LOGGER.info("Checking all known binlogs from the database"); + query("SHOW BINARY LOGS", rs -> { + while (rs.next()) { + logNames.add(rs.getString(1)); + } + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while connecting to the database and looking for binary logs: ", e); + } + + if (logNames.isEmpty()) { + return null; + } + return logNames.get(0); + } + + /** + * Query the database server and get the list of binlog files that are currently available. + * + * @return list of binlog files + */ + public List availableBinlogFiles() { + List logNames = new ArrayList<>(); + try { + LOGGER.info("Get all known binlogs"); + query("SHOW BINARY LOGS", rs -> { + while (rs.next()) { + logNames.add(rs.getString(1)); + } + }); + return logNames; + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while connecting to the database and looking for binary logs: ", e); + } + } + + /** + * Get the estimated table size, aka number of rows. + * + * @param tableId the table identifier; should never be null + * @return an optional long-value that may be empty if no data is available or an exception occurred + */ + public OptionalLong getEstimatedTableSize(TableId tableId) { + try { + // Choose how we create statements based on the # of rows. + // This is approximate and less accurate then COUNT(*), + // but far more efficient for large InnoDB tables. + execute("USE `" + tableId.catalog() + "`;"); + return queryAndMap("SHOW TABLE STATUS LIKE '" + tableId.table() + "';", rs -> { + if (rs.next()) { + return OptionalLong.of((rs.getLong(5))); + } + return OptionalLong.empty(); + }); + } + catch (SQLException e) { + LOGGER.debug("Error while getting number of rows in table {}: {}", tableId, e.getMessage(), e); + } + return OptionalLong.empty(); + } + + /** + * Read the charset-related system variables. + * + * @return the system variables that are related to server character sets; never null + */ + public Map readCharsetSystemVariables() { + // Read the system variables from the MySQL instance and get the current database name ... + LOGGER.debug("Reading charset-related system variables before parsing DDL history."); + return querySystemVariables(SQL_SHOW_SYSTEM_VARIABLES_CHARACTER_SET); + } + + /** + * Executes a {@code SET} statement, setting each variable with it's specified value. + * + * @param variables key/value variable names as keys and the value(s) to be set + * @return the constructed {@code SET} database statement; never null + */ + public String setStatementFor(Map variables) { + StringBuilder sb = new StringBuilder("SET "); + boolean first = true; + List varNames = new ArrayList<>(variables.keySet()); + Collections.sort(varNames); + for (String varName : varNames) { + if (first) { + first = false; + } + else { + sb.append(", "); + } + sb.append(varName).append("="); + String value = variables.get(varName); + if (value == null) { + value = ""; + } + if (value.contains(",") || value.contains(";")) { + value = "'" + value + "'"; + } + sb.append(value); + } + return sb.append(";").toString(); + } + + /** + * Determines whether the binlog format used by the database server is {@code binlog_row_image='FULL'}. + * + * @return {@code true} if the {@code binlog_row_image} is set to {@code FULL}, {@code false} otherwise + */ + public boolean isBinlogRowImageFull() { + try { + final String rowImage = queryAndMap("SHOW GLOBAL VARIABLES LIKE 'binlog_row_image'", rs -> { + if (rs.next()) { + return rs.getString(2); + } + // This setting was introduced in MySQL 5.6+ with default of 'FULL'. + // For older versions, assume 'FULL'. + return "FULL"; + }); + LOGGER.debug("binlog_row_image={}", rowImage); + return "FULL".equalsIgnoreCase(rowImage); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while connecting to the database and looking at BINLOG_ROW_IMAGE mode: ", e); + } + } + + /** + * Determine whether the database server has the row-level binlog enabled. + * + * @return {@code true} if the server's {@code binlog_format} is set to {@code ROW}, {@code false} otherwise + */ + public boolean isBinlogFormatRow() { + try { + final String mode = queryAndMap("SHOW GLOBAL VARIABLES LIKE 'binlog_format'", rs -> rs.next() ? rs.getString(2) : ""); + LOGGER.debug("binlog_format={}", mode); + return "ROW".equalsIgnoreCase(mode); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while connecting to the database and looking at BINLOG_FORMAT mode: ", e); + } + } + + /** + * Read the database server's default character sets for existing databases. + * + * @return the map of database names and their default character sets; never null + */ + public Map readDatabaseCollations() { + LOGGER.debug("Reading default database charsets"); + try { + return queryAndMap("SELECT schema_name, default_character_set_name, default_collation_name FROM information_schema.schemata", rs -> { + final Map charsets = new HashMap<>(); + while (rs.next()) { + String dbName = rs.getString(1); + String charset = rs.getString(2); + String collation = rs.getString(3); + if (dbName != null && (charset != null || collation != null)) { + charsets.put(dbName, new DatabaseLocales(charset, collation)); + LOGGER.debug("\t{} = {}, {}", + Strings.pad(dbName, 45, ' '), + Strings.pad(charset, 45, ' '), + Strings.pad(collation, 45, ' ')); + } + } + return charsets; + }); + } + catch (SQLException e) { + throw new DebeziumException("Error reading default database charsets: " + e.getMessage(), e); + } + } + + /** + * Return whether the table identifiers are case-sensitive. + * + * @return {@code true} if the table identifiers are case-sensitive, {@code false} otherwise + */ + public boolean isTableIdCaseSensitive() { + return !"0".equals(readSystemVariables().get(MySqlSystemVariables.LOWER_CASE_TABLE_NAMES)); + } + + /** + * Determine whether the binlog position as set in the offset details is available on the server. + * + * @param config the connector configuration; should not be null + * @param gtid the GTID from the connector offsets; may be null + * @param binlogFileName the binlog file name from the connector offsets; may be null + * @return {@code true} if the binlog position is available, {@code false} otherwise + */ + @SuppressWarnings("BooleanMethodIsAlwaysInverted") + public boolean isBinlogPositionAvailable(MySqlConnectorConfig config, String gtid, String binlogFileName) { + if (gtid != null) { + if (gtid.trim().isEmpty()) { + // Start at the beginning + return true; + } + + GtidSet availableGtidSet = knownGtidSet(); + if (availableGtidSet.isEmpty()) { + // Last offsets had GTIDs but the server does not use them + LOGGER.info("Connector used GTIDs previously, but server does not know of any GTIDs or they are not enabled"); + return false; + } + + // GTIDs are enabled, used previously, retain only the ranges allowed + GtidSet gtidSet = createGtidSet(gtid).retainAll(config.gtidSourceFilter()); + + // Get the GTID set that is available on the server + if (gtidSet.isContainedWithin(availableGtidSet)) { + LOGGER.info("The current GTID set {} does not contain the GTID set required by the connector {}", + availableGtidSet, gtidSet); + + final GtidSet knownServerSet = availableGtidSet.retainAll(config.gtidSourceFilter()); + final GtidSet gtidSetToReplicate = subtractGtidSet(knownServerSet, gtidSet); + final GtidSet purgedGtidSet = purgedGtidSet(); + LOGGER.info("Serer has already purged {} GTIDs", purgedGtidSet); + + final GtidSet nonPurgedGtidSetTemplate = subtractGtidSet(gtidSetToReplicate, purgedGtidSet); + LOGGER.info("GTIDs known by the server but not processed yet {}, for replication are available only {}", + gtidSetToReplicate, nonPurgedGtidSetTemplate); + + if (!gtidSetToReplicate.equals(nonPurgedGtidSetTemplate)) { + LOGGER.info("Some of the GTIDs needed to replicate have been already purged"); + return false; + } + return true; + } + + LOGGER.info("Connector last known GTIDs are {}, but server has {}", gtidSet, availableGtidSet); + return false; + } + + if (Strings.isNullOrBlank(binlogFileName)) { + // Start at the current position + return true; + } + + // Accumulate the available binlog filenames, and compare with the one we're supposed to use + List logNames = availableBinlogFiles(); + boolean found = logNames.stream().anyMatch(binlogFileName::equals); + if (!found && LOGGER.isInfoEnabled()) { + LOGGER.info("Connector requires binlog file '{}', but server only has {}", binlogFileName, String.join(", ", logNames)); + } + else if (found && LOGGER.isInfoEnabled()) { + LOGGER.info("Server has the binlog file '{}' required by the connector", binlogFileName); + } + + return found; + } + + /** + * Determine whether the server has enabled GTID support. + * + * @return {@code false} if the server has not enabled GTIDs, {@code true} otherwise + */ + public abstract boolean isGtidModeEnabled(); + + /** + * Returns the most recent executed GTID set or position. + * + * @return the string representation of the most recent executed GTID set or position; never null but + * will be empty if the server does not support or has not processed any GTID + */ + public abstract GtidSet knownGtidSet(); + + /** + * Determines the difference between two GTID sets. + * + * @param set1 the first set; should never be null + * @param set2 the second set; should never be null + * @return the subtraction of the two sets in a new GtidSet instance; never null + */ + public abstract GtidSet subtractGtidSet(GtidSet set1, GtidSet set2); + + /** + * Get the purged GTID values from the server. + * + * @return A GTID set; may be empty of GTID support is not enabled or if none have been purged + */ + public abstract GtidSet purgedGtidSet(); + + /** + * Apply the include/exclude GTID source filters to the current {@link MySqlOffsetContext#gtidSet() GTID set} and merge them onto the + * currently available GTID set from a MySQL server. + * + * The merging behavior of this method might seem a bit strange at first. It's required in order for Debezium to consume a + * MySQL binlog that has multi-source replication enabled, if a failover has to occur. In such a case, the server that + * Debezium is failed over to might have a different set of sources, but still include the sources required for Debezium + * to continue to function. MySQL does not allow downstream replicas to connect if the GTID set does not contain GTIDs for + * all channels that the server is replicating from, even if the server does have the data needed by the client. To get + * around this, we can have Debezium merge its GTID set with whatever is on the server, so that MySQL will allow it to + * connect. See DBZ-143 for details. + * + * This method does not mutate any state in the context. + * + * @param availableServerGtidSet the GTID set currently available in the MySQL server + * @param purgedServerGtid the GTID set already purged by the MySQL server + * @return A GTID set meant for consuming from a MySQL binlog; may return null if the SourceInfo has no GTIDs and therefore + * none were filtered + */ + + public abstract GtidSet filterGtidSet(Predicate gtidSourceFilter, String offsetGtids, GtidSet availableServerGtidSet, GtidSet purgedServerGtidSet); + + /** + * Read the system variables. + * + * @return all the system variables; never null + */ + protected Map readSystemVariables() { + // Read the system variables from the MySQL instance and get the current database name ... + LOGGER.debug("Reading system variables"); + return querySystemVariables(SQL_SHOW_SYSTEM_VARIABLES); + } + + /** + * Read the SSL version session variable. + * + * @return the session variable value related to the session SSL version + */ + protected String getSessionVariableForSslVersion() { + final String SSL_VERSION = "Ssl_version"; + LOGGER.debug("Reading session variable for Ssl Version"); + Map sessionVariables = querySystemVariables(SQL_SHOW_SESSION_VARIABLE_SSL_VERSION); + if (!sessionVariables.isEmpty() && sessionVariables.containsKey(SSL_VERSION)) { + return sessionVariables.get(SSL_VERSION); + } + return null; + } + + protected abstract GtidSet createGtidSet(String gtids); + + private Map querySystemVariables(String statement) { + final Map variables = new HashMap<>(); + try { + query(statement, rs -> { + while (rs.next()) { + String varName = rs.getString(1); + String value = rs.getString(2); + if (varName != null && value != null) { + variables.put(varName, value); + LOGGER.debug("\t{} = {}", + Strings.pad(varName, 45, ' '), + Strings.pad(value, 45, ' ')); + } + } + }); + } + catch (SQLException e) { + throw new DebeziumException("Error reading MySQL variables: " + e.getMessage(), e); + } + + return variables; + } + + public static class DatabaseLocales { + private final String charset; + private final String collation; + + public DatabaseLocales(String charset, String collation) { + this.charset = charset; + this.collation = collation; + } + + public void appendToDdlStatement(String dbName, StringBuilder ddl) { + if (charset != null) { + LOGGER.debug("Setting default charset '{}' for database '{}'", charset, dbName); + ddl.append(" CHARSET ").append(charset); + } + else { + LOGGER.info("Default database charset for '{}' not found", dbName); + } + if (collation != null) { + LOGGER.debug("Setting default collation '{}' for database '{}'", collation, dbName); + ddl.append(" COLLATE ").append(collation); + } + else { + LOGGER.info("Default database collation for '{}' not found", dbName); + } + } + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlHistoryRecordComparator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractHistoryRecordComparator.java similarity index 92% rename from debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlHistoryRecordComparator.java rename to debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractHistoryRecordComparator.java index b846e16bb..c3fd32cbb 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlHistoryRecordComparator.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/AbstractHistoryRecordComparator.java @@ -3,22 +3,30 @@ * * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 */ -package io.debezium.connector.mysql; +package io.debezium.connector.mysql.strategy; import java.util.function.Predicate; +import io.debezium.annotation.VisibleForTesting; +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.SourceInfo; import io.debezium.document.Document; import io.debezium.relational.history.HistoryRecordComparator; -final class MySqlHistoryRecordComparator extends HistoryRecordComparator { +/** + * @author Chris Cranford + */ +public abstract class AbstractHistoryRecordComparator extends HistoryRecordComparator { private final Predicate gtidSourceFilter; - MySqlHistoryRecordComparator(Predicate gtidSourceFilter) { - super(); + public AbstractHistoryRecordComparator(Predicate gtidSourceFilter) { this.gtidSourceFilter = gtidSourceFilter; } + protected abstract GtidSet createGtidSet(String gtidSet); + /** * Determine whether the first offset is at or before the point in time of the second * offset, where the offsets are given in JSON representation of the maps returned by {@link MySqlOffsetContext#getOffset()}. @@ -36,15 +44,16 @@ final class MySqlHistoryRecordComparator extends HistoryRecordComparator { * @return {@code true} if the recorded position is at or before the desired position; or {@code false} otherwise */ @Override - protected boolean isPositionAtOrBefore(Document recorded, Document desired) { + @VisibleForTesting + public boolean isPositionAtOrBefore(Document recorded, Document desired) { String recordedGtidSetStr = recorded.getString(MySqlOffsetContext.GTID_SET_KEY); String desiredGtidSetStr = desired.getString(MySqlOffsetContext.GTID_SET_KEY); if (desiredGtidSetStr != null) { // The desired position uses GTIDs, so we ideally compare using GTIDs ... if (recordedGtidSetStr != null) { // Both have GTIDs, so base the comparison entirely on the GTID sets. - GtidSet recordedGtidSet = new GtidSet(recordedGtidSetStr); - GtidSet desiredGtidSet = new GtidSet(desiredGtidSetStr); + GtidSet recordedGtidSet = createGtidSet(recordedGtidSetStr); + GtidSet desiredGtidSet = createGtidSet(desiredGtidSetStr); if (gtidSourceFilter != null) { // Apply the GTID source filter before we do any comparisons ... recordedGtidSet = recordedGtidSet.retainAll(gtidSourceFilter); diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/BinaryLogClientConfigurator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/BinaryLogClientConfigurator.java new file mode 100644 index 000000000..0f4887b04 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/BinaryLogClientConfigurator.java @@ -0,0 +1,29 @@ +/* + * 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.mysql.strategy; + +import java.util.concurrent.ThreadFactory; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.EventType; + +/** + * @author Chris Cranford + */ +public interface BinaryLogClientConfigurator { + /** + * Configures the provided Binary Log Client instance. + * + * @param client the client instance ot be configured; should not be null + * @param threadFactory the thread factory to be used; should not be null + * @param connection the connector's JDBC connection; should not be null + * + * @return the configured binary log client instance + */ + BinaryLogClient configure(BinaryLogClient client, ThreadFactory threadFactory, AbstractConnectorConnection connection); + + EventType getIncludeSqlQueryEventType(); +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectionConfiguration.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectionConfiguration.java new file mode 100644 index 000000000..459939a9a --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectionConfiguration.java @@ -0,0 +1,42 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.mysql.strategy; + +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; + +/** + * @author Chris Cranford + */ +public interface ConnectionConfiguration { + JdbcConfiguration config(); + + Configuration originalConfig(); + + JdbcConnection.ConnectionFactory factory(); + + String username(); + + String password(); + + String hostname(); + + int port(); + + MySqlConnectorConfig.SecureConnectionMode sslMode(); + + boolean sslModeEnabled(); + + String sslKeyStore(); + + char[] sslKeyStorePassword(); + + String sslTrustStore(); + + char[] sslTrustStorePassword(); +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectorAdapter.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectorAdapter.java new file mode 100644 index 000000000..ac5e98b39 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/ConnectorAdapter.java @@ -0,0 +1,63 @@ +/* + * 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.mysql.strategy; + +import com.github.shyiko.mysql.binlog.event.EventData; + +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.MySqlDatabaseSchema; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.notification.NotificationService; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; +import io.debezium.pipeline.source.spi.DataChangeEventListener; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.spi.schema.DataCollectionId; +import io.debezium.util.Clock; + +/** + * Provides the MySQL connector with an adapter pattern to support varied configurations + * between MySQL and MariaDB and their drivers. + * + * @author Chris Cranford + */ +public interface ConnectorAdapter { + + // todo: we had to introduce the configuration argument because of the extra database properties + // that are set in the task; we would get a test failure because of those missing if we + // simply created the connection configuration the base data in the MySqlConnectorConfig ctor + AbstractConnectorConnection createConnection(Configuration configuration); + + BinaryLogClientConfigurator getBinaryLogClientConfigurator(); + + // todo: should we consider splitting value converters, it may prove useful in the future + // doing so would imply we won't likely need this method as it can be encapsulated? + String getJavaEncodingForCharSet(String charSetName); + + // todo: for the moment we only expose the few handler deviations + // we may want to simply implement an abstract and concrete streaming impls + + String getRecordingQueryFromEvent(EventData event); + + AbstractHistoryRecordComparator getHistoryRecordComparator(); + + IncrementalSnapshotContext getIncrementalSnapshotContext(); + + Long getReadOnlyIncrementalSnapshotSignalOffset(MySqlOffsetContext previousOffsets); + + IncrementalSnapshotChangeEventSource createIncrementalSnapshotChangeEventSource( + MySqlConnectorConfig connectorConfig, + AbstractConnectorConnection connection, + EventDispatcher dispatcher, + MySqlDatabaseSchema schema, + Clock clock, + SnapshotProgressListener snapshotProgressListener, + DataChangeEventListener dataChangeEventListener, + NotificationService notificationService); +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbBinaryLogClientConfigurator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbBinaryLogClientConfigurator.java new file mode 100644 index 000000000..04a48974f --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbBinaryLogClientConfigurator.java @@ -0,0 +1,59 @@ +/* + * 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.mysql.strategy.mariadb; + +import java.util.concurrent.ThreadFactory; + +import com.github.shyiko.mysql.binlog.BinaryLogClient; +import com.github.shyiko.mysql.binlog.event.EventType; +import com.github.shyiko.mysql.binlog.event.deserialization.EventDeserializer; + +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.strategy.AbstractBinaryLogClientConfigurator; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; + +/** + * An {@link AbstractBinaryLogClientConfigurator} implementation for MariaDB. + * + * @author Chris Cranford + */ +public class MariaDbBinaryLogClientConfigurator extends AbstractBinaryLogClientConfigurator { + + public MariaDbBinaryLogClientConfigurator(MySqlConnectorConfig connectorConfig) { + super(connectorConfig); + } + + @Override + public BinaryLogClient configure(BinaryLogClient client, ThreadFactory threadFactory, AbstractConnectorConnection connection) { + BinaryLogClient result = super.configure(client, threadFactory, connection); + if (getConnectorConfig().includeSqlQuery()) { + // Binlog client explicitly needs to be told to enable ANNOTATE_ROWS events, which is the + // MariaDB equivalent of ROWS_QUERY. This must be done ahead of the connection to make + // sure that the right negotiation bits are set during handshake. + result.setUseSendAnnotateRowsEvent(true); + } + return result; + } + + @Override + public EventType getIncludeSqlQueryEventType() { + return EventType.ANNOTATE_ROWS; + } + + @Override + protected void configureReplicaCompatibility(BinaryLogClient client) { + // This makes sure BEGIN events are emitted via QUERY events rather than GTIDs. + client.setMariaDbSlaveCapability(2); + } + + @Override + protected EventDeserializer createEventDeserializer() { + EventDeserializer eventDeserializer = super.createEventDeserializer(); + eventDeserializer.setCompatibilityMode(EventDeserializer.CompatibilityMode.CHAR_AND_BINARY_AS_BYTE_ARRAY); + return eventDeserializer; + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnection.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnection.java new file mode 100644 index 000000000..8e5c0c5e1 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnection.java @@ -0,0 +1,87 @@ +/* + * 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.mysql.strategy.mariadb; + +import java.sql.SQLException; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.DebeziumException; +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.MySqlFieldReader; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; + +/** + * An {@link AbstractConnectorConnection} for MariaDB. + * + * @author Chris Cranford + */ +public class MariaDbConnection extends AbstractConnectorConnection { + + private static final Logger LOGGER = LoggerFactory.getLogger(MariaDbConnection.class); + + public MariaDbConnection(MariaDbConnectionConfiguration connectionConfig, MySqlFieldReader fieldReader) { + super(connectionConfig, fieldReader); + } + + @Override + public boolean isGtidModeEnabled() { + // MariaDB always has GTID enabled; however, GTID_STRICT_MODE can be enabled or disabled. + // For now we don't enforce this, so it can be a mixture + return true; + } + + @Override + public GtidSet knownGtidSet() { + // MariaDB does not store the executed GTID details in the SHOW MASTER STATUS output like MySQL; + // however, instead makes this information available as a variable. The GTID_BINLOG_POS gives + // the current GTID position of the binary log and can therefore be considered the equivalent to + // MySQL's executed GTID set. + try { + return queryAndMap("SHOW GLOBAL VARIABLES LIKE 'GTID_BINLOG_POS'", rs -> { + if (rs.next()) { + LOGGER.info("knownGtidSet = {}", rs.getString(2)); + return new MariaDbGtidSet(rs.getString(2)); + } + return new MariaDbGtidSet(""); + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while looking at GTID_BINLOG_POS: ", e); + } + } + + @Override + public GtidSet subtractGtidSet(GtidSet set1, GtidSet set2) { + throw new DebeziumException("GtidSet subtraction not yet implemented by MariaDB"); + } + + @Override + public GtidSet purgedGtidSet() { + // MariaDB does not store purged GTID details in a variable like MySQL; however, it stores the + // information in the `gtid_slave_pos` table in the `mysql` database, but this information has + // slightly different semantics. The purging is handled by MariaDB through the binary log's + // expiration settings and the `RESET MASTER` or `PURGE BINARY LOGS` statements. + // + // In order to calculate the purged state, we would need to get the `gtid_binlog_pos` variable + // that shows the current position of the GTID in the binary log, used by the primary, and + // compare this with the `gtid_slave_pos` variable on the replica server, which indicates the + // position of the GTIDs that have been applied. + throw new DebeziumException("Fetching purged GtidSet details is not yet supported"); + } + + @Override + public GtidSet filterGtidSet(Predicate gtidSourceFilter, String offsetGtids, GtidSet availableServerGtidSet, GtidSet purgedServerGtidSet) { + throw new DebeziumException("NYI"); + } + + @Override + protected GtidSet createGtidSet(String gtids) { + return new MariaDbGtidSet(gtids); + } +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectionConfiguration.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectionConfiguration.java new file mode 100644 index 000000000..e54b5d192 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectionConfiguration.java @@ -0,0 +1,37 @@ +/* + * 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.mysql.strategy.mariadb; + +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.strategy.AbstractConnectionConfiguration; +import io.debezium.util.Strings; + +/** + * An {@link AbstractConnectionConfiguration} for MariaDB. + * + * @author Chris Cranford + */ +public class MariaDbConnectionConfiguration extends AbstractConnectionConfiguration { + + private static final String JDBC_PROPERTY_MARIADB_TIME_ZONE = "timezone"; + + public MariaDbConnectionConfiguration(Configuration config) { + super(config); + } + + @Override + protected String getConnectionTimeZonePropertyName() { + return JDBC_PROPERTY_MARIADB_TIME_ZONE; + } + + @Override + protected String resolveConnectionTimeZone(Configuration dbConfig) { + // Debezium by default expected timezone data delivered in server timezone + String timezone = dbConfig.getString(JDBC_PROPERTY_MARIADB_TIME_ZONE); + return !Strings.isNullOrBlank(timezone) ? timezone : "auto"; + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectorAdapter.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectorAdapter.java new file mode 100644 index 000000000..80adf61e9 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbConnectorAdapter.java @@ -0,0 +1,101 @@ +/* + * 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.mysql.strategy.mariadb; + +import com.github.shyiko.mysql.binlog.event.AnnotateRowsEventData; +import com.github.shyiko.mysql.binlog.event.EventData; + +import io.debezium.DebeziumException; +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MariaDbProtocolFieldReader; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.MySqlDatabaseSchema; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.AbstractHistoryRecordComparator; +import io.debezium.connector.mysql.strategy.BinaryLogClientConfigurator; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; +import io.debezium.connector.mysql.strategy.mysql.MySqlConnection; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.notification.NotificationService; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; +import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext; +import io.debezium.pipeline.source.spi.DataChangeEventListener; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.spi.schema.DataCollectionId; +import io.debezium.util.Clock; + +/** + * This connector adapter provides a complete implementation for MariaDB assuming that + * the MariaDB driver is used for connections. + * + * @author Chris Cranford + */ +public class MariaDbConnectorAdapter implements ConnectorAdapter { + + private final MySqlConnectorConfig connectorConfig; + private final MariaDbBinaryLogClientConfigurator binaryLogClientConfigurator; + + public MariaDbConnectorAdapter(MySqlConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + this.binaryLogClientConfigurator = new MariaDbBinaryLogClientConfigurator(connectorConfig); + } + + @Override + public AbstractConnectorConnection createConnection(Configuration configuration) { + MariaDbConnectionConfiguration connectionConfig = new MariaDbConnectionConfiguration(configuration); + return new MariaDbConnection(connectionConfig, new MariaDbProtocolFieldReader(connectorConfig)); + } + + @Override + public BinaryLogClientConfigurator getBinaryLogClientConfigurator() { + return binaryLogClientConfigurator; + } + + @Override + public String getRecordingQueryFromEvent(EventData eventData) { + return ((AnnotateRowsEventData) eventData).getRowsQuery(); + } + + @Override + public String getJavaEncodingForCharSet(String charSetName) { + // todo: this should use a MariaDB specific implementation + return MySqlConnection.getJavaEncodingForCharSet(charSetName); + } + + @Override + public AbstractHistoryRecordComparator getHistoryRecordComparator() { + return new MariaDbHistoryRecordComparator(connectorConfig.gtidSourceFilter()); + } + + @Override + public IncrementalSnapshotContext getIncrementalSnapshotContext() { + if (connectorConfig.isReadOnlyConnection()) { + throw new DebeziumException("NYI"); + } + return new SignalBasedIncrementalSnapshotContext<>(); + } + + @Override + public Long getReadOnlyIncrementalSnapshotSignalOffset(MySqlOffsetContext previousOffsets) { + throw new DebeziumException("NYI"); + } + + @Override + public IncrementalSnapshotChangeEventSource createIncrementalSnapshotChangeEventSource( + MySqlConnectorConfig connectorConfig, + AbstractConnectorConnection connection, + EventDispatcher dispatcher, + MySqlDatabaseSchema schema, + Clock clock, + SnapshotProgressListener snapshotProgressListener, + DataChangeEventListener dataChangeEventListener, + NotificationService notificationService) { + throw new DebeziumException("NYI"); + } +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbGtidSet.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbGtidSet.java new file mode 100644 index 000000000..fca92c338 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbGtidSet.java @@ -0,0 +1,55 @@ +/* + * 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.mysql.strategy.mariadb; + +import java.util.function.Predicate; + +import io.debezium.DebeziumException; +import io.debezium.connector.mysql.GtidSet; + +/** + * @author Chris Cranford + */ +public class MariaDbGtidSet implements GtidSet { + + public MariaDbGtidSet(String gtid) { + } + + @Override + public boolean isEmpty() { + throw new DebeziumException("NYI"); + } + + @Override + public GtidSet retainAll(Predicate sourceFilter) { + throw new DebeziumException("NYI"); + } + + @Override + public boolean isContainedWithin(GtidSet other) { + throw new DebeziumException("NYI"); + } + + @Override + public GtidSet with(GtidSet other) { + throw new DebeziumException("NYI"); + } + + @Override + public GtidSet getGtidSetBeginning() { + throw new DebeziumException("NYI"); + } + + @Override + public boolean contains(String gtid) { + throw new DebeziumException("NYI"); + } + + @Override + public GtidSet subtract(GtidSet other) { + throw new DebeziumException("NYI"); + } +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbHistoryRecordComparator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbHistoryRecordComparator.java new file mode 100644 index 000000000..b2f07779e --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/MariaDbHistoryRecordComparator.java @@ -0,0 +1,27 @@ +/* + * 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.mysql.strategy.mariadb; + +import java.util.function.Predicate; + +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.strategy.AbstractHistoryRecordComparator; + +/** + * @author Chris Cranford + */ +public class MariaDbHistoryRecordComparator extends AbstractHistoryRecordComparator { + + public MariaDbHistoryRecordComparator(Predicate gtidSourceFilter) { + super(gtidSourceFilter); + } + + @Override + protected GtidSet createGtidSet(String gtidSet) { + return new MariaDbGtidSet(gtidSet); + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/hybrid/MariaDbHybridConnectorAdapter.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/hybrid/MariaDbHybridConnectorAdapter.java new file mode 100644 index 000000000..ec005f441 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mariadb/hybrid/MariaDbHybridConnectorAdapter.java @@ -0,0 +1,43 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.mysql.strategy.mariadb.hybrid; + +import com.github.shyiko.mysql.binlog.event.AnnotateRowsEventData; +import com.github.shyiko.mysql.binlog.event.EventData; + +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.strategy.BinaryLogClientConfigurator; +import io.debezium.connector.mysql.strategy.mariadb.MariaDbBinaryLogClientConfigurator; +import io.debezium.connector.mysql.strategy.mysql.MySqlConnectorAdapter; + +/** + * This connector adapter provides a hybrid configuration where the user connects to a + * MariaDB target system; however, uses the MySQL driver. + * + * @author Chris Cranford + */ +public class MariaDbHybridConnectorAdapter extends MySqlConnectorAdapter { + + // todo: Do we want to consider supporting this mode at all? + + private final MariaDbBinaryLogClientConfigurator binaryLogClientConfigurator; + + public MariaDbHybridConnectorAdapter(MySqlConnectorConfig connectorConfig) { + super(connectorConfig); + this.binaryLogClientConfigurator = new MariaDbBinaryLogClientConfigurator(connectorConfig); + } + + @Override + public BinaryLogClientConfigurator getBinaryLogClientConfigurator() { + return binaryLogClientConfigurator; + } + + @Override + public String getRecordingQueryFromEvent(EventData eventData) { + return ((AnnotateRowsEventData) eventData).getRowsQuery(); + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlBinaryLogClientConfigurator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlBinaryLogClientConfigurator.java new file mode 100644 index 000000000..6498f3c0d --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlBinaryLogClientConfigurator.java @@ -0,0 +1,20 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.mysql.strategy.mysql; + +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.strategy.AbstractBinaryLogClientConfigurator; + +/** + * @author Chris Cranford + */ +public class MySqlBinaryLogClientConfigurator extends AbstractBinaryLogClientConfigurator { + + public MySqlBinaryLogClientConfigurator(MySqlConnectorConfig connectorConfig) { + super(connectorConfig); + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnection.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnection.java new file mode 100644 index 000000000..de4c09709 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnection.java @@ -0,0 +1,146 @@ +/* + * 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.mysql.strategy.mysql; + +import java.sql.SQLException; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.mysql.cj.CharsetMapping; + +import io.debezium.DebeziumException; +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.MySqlFieldReader; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; + +/** + * An {@link AbstractConnectorConnection} to be used with MySQL. + * + * @author Jiri Pechanec, Randell Hauch, Chris Cranford + */ +public class MySqlConnection extends AbstractConnectorConnection { + + private static final Logger LOGGER = LoggerFactory.getLogger(MySqlConnection.class); + + public MySqlConnection(MySqlConnectionConfiguration connectionConfig, MySqlFieldReader fieldReader) { + super(connectionConfig, fieldReader); + } + + @Override + public boolean isGtidModeEnabled() { + try { + return queryAndMap("SHOW GLOBAL VARIABLES LIKE 'GTID_MODE'", rs -> { + if (rs.next()) { + return "ON".equalsIgnoreCase(rs.getString(2)); + } + return false; + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while looking at GTID mode: ", e); + } + } + + @Override + public GtidSet knownGtidSet() { + try { + return queryAndMap("SHOW MASTER STATUS", rs -> { + if (rs.next() && rs.getMetaData().getColumnCount() > 4) { + return new MySqlGtidSet(rs.getString(5)); // GTID set, may be null, blank, or contain a GTID set + } + return new MySqlGtidSet(""); + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while looking at GTID mode: ", e); + } + } + + @Override + public GtidSet subtractGtidSet(GtidSet set1, GtidSet set2) { + try { + return prepareQueryAndMap("SELECT GTID_SUBTRACT(?, ?)", + ps -> { + ps.setString(1, set1.toString()); + ps.setString(2, set2.toString()); + }, + rs -> { + if (rs.next()) { + return new MySqlGtidSet(rs.getString(1)); + } + return new MySqlGtidSet(""); + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while executing GTID_SUBTRACT: ", e); + } + } + + @Override + public GtidSet purgedGtidSet() { + try { + return queryAndMap("SELECT @@global.gtid_purged", rs -> { + if (rs.next() && rs.getMetaData().getColumnCount() > 0) { + return new MySqlGtidSet(rs.getString(1)); // GTID set, may be null, blank, or contain a GTID set + } + return new MySqlGtidSet(""); + }); + } + catch (SQLException e) { + throw new DebeziumException("Unexpected error while looking at gtid_purged variable: ", e); + } + } + + @Override + public GtidSet filterGtidSet(Predicate gtidSourceFilter, String offsetGtids, GtidSet availableServerGtidSet, GtidSet purgedServerGtidSet) { + String gtidStr = offsetGtids; + if (gtidStr == null) { + return null; + } + LOGGER.info("Attempting to generate a filtered GTID set"); + LOGGER.info("GTID set from previous recorded offset: {}", gtidStr); + GtidSet filteredGtidSet = new MySqlGtidSet(gtidStr); + if (gtidSourceFilter != null) { + filteredGtidSet = filteredGtidSet.retainAll(gtidSourceFilter); + LOGGER.info("GTID set after applying GTID source includes/excludes to previous recorded offset: {}", filteredGtidSet); + } + LOGGER.info("GTID set available on server: {}", availableServerGtidSet); + + final GtidSet knownGtidSet = filteredGtidSet; + LOGGER.info("Using first available positions for new GTID channels"); + final GtidSet relevantAvailableServerGtidSet = (gtidSourceFilter != null) ? availableServerGtidSet.retainAll(gtidSourceFilter) : availableServerGtidSet; + LOGGER.info("Relevant GTID set available on server: {}", relevantAvailableServerGtidSet); + + GtidSet mergedGtidSet = relevantAvailableServerGtidSet + .retainAll(uuid -> ((MySqlGtidSet) knownGtidSet).forServerWithId(uuid) != null) + .with(purgedServerGtidSet) + .with(filteredGtidSet); + + LOGGER.info("Final merged GTID set to use when connecting to MySQL: {}", mergedGtidSet); + return mergedGtidSet; + } + + @Override + protected GtidSet createGtidSet(String gtids) { + return new MySqlGtidSet(gtids); + } + + public static String getJavaEncodingForCharSet(String charSetName) { + return CharsetMappingWrapper.getJavaEncodingForMysqlCharSet(charSetName); + } + + /** + * Helper to gain access to protected method + */ + private final static class CharsetMappingWrapper extends CharsetMapping { + static String getJavaEncodingForMysqlCharSet(String charSetName) { + return CharsetMapping.getStaticJavaEncodingForMysqlCharset(charSetName); + } + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectionConfiguration.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectionConfiguration.java new file mode 100644 index 000000000..3362dbd48 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectionConfiguration.java @@ -0,0 +1,37 @@ +/* + * 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.mysql.strategy.mysql; + +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.strategy.AbstractConnectionConfiguration; + +/** + * An {@link AbstractConnectionConfiguration} implementation for MySQL. + * + * @author Chris Cranford + */ +public class MySqlConnectionConfiguration extends AbstractConnectionConfiguration { + + private static final String JDBC_PROPERTY_CONNECTION_TIME_ZONE = "connectionTimeZone"; + + public MySqlConnectionConfiguration(Configuration config) { + super(config); + } + + @Override + protected String getConnectionTimeZonePropertyName() { + return JDBC_PROPERTY_CONNECTION_TIME_ZONE; + } + + @Override + protected String resolveConnectionTimeZone(Configuration dbConfig) { + // Debezium by default expects time zoned data delivered in server timezone + String connectionTimeZone = dbConfig.getString(JDBC_PROPERTY_CONNECTION_TIME_ZONE); + return connectionTimeZone != null ? connectionTimeZone : "SERVER"; + // return !Strings.isNullOrBlank(connectionTimeZone) ? connectionTimeZone : "SERVER"; + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectorAdapter.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectorAdapter.java new file mode 100644 index 000000000..6a81611af --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlConnectorAdapter.java @@ -0,0 +1,118 @@ +/* + * 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.mysql.strategy.mysql; + +import com.github.shyiko.mysql.binlog.event.EventData; +import com.github.shyiko.mysql.binlog.event.RowsQueryEventData; + +import io.debezium.config.Configuration; +import io.debezium.connector.mysql.MySqlBinaryProtocolFieldReader; +import io.debezium.connector.mysql.MySqlConnectorConfig; +import io.debezium.connector.mysql.MySqlDatabaseSchema; +import io.debezium.connector.mysql.MySqlFieldReader; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.connector.mysql.MySqlTextProtocolFieldReader; +import io.debezium.connector.mysql.strategy.AbstractConnectorConnection; +import io.debezium.connector.mysql.strategy.AbstractHistoryRecordComparator; +import io.debezium.connector.mysql.strategy.BinaryLogClientConfigurator; +import io.debezium.connector.mysql.strategy.ConnectorAdapter; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.notification.NotificationService; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotChangeEventSource; +import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; +import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext; +import io.debezium.pipeline.source.spi.DataChangeEventListener; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.relational.TableId; +import io.debezium.spi.schema.DataCollectionId; +import io.debezium.util.Clock; + +/** + * This connector adapter provides a complete implementation for MySQL assuming that the + * MySQL driver is used for connections. + * + * @author Chris Cranford + */ +public class MySqlConnectorAdapter implements ConnectorAdapter { + + private final MySqlConnectorConfig connectorConfig; + private final MySqlBinaryLogClientConfigurator binaryLogClientConfigurator; + + public MySqlConnectorAdapter(MySqlConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + this.binaryLogClientConfigurator = new MySqlBinaryLogClientConfigurator(connectorConfig); + } + + @Override + public AbstractConnectorConnection createConnection(Configuration configuration) { + final MySqlConnectionConfiguration connectionConfig = new MySqlConnectionConfiguration(configuration); + return new MySqlConnection(connectionConfig, resolveFieldReader()); + } + + @Override + public BinaryLogClientConfigurator getBinaryLogClientConfigurator() { + return binaryLogClientConfigurator; + } + + @Override + public String getJavaEncodingForCharSet(String charSetName) { + return MySqlConnection.getJavaEncodingForCharSet(charSetName); + } + + @Override + public String getRecordingQueryFromEvent(EventData eventData) { + return ((RowsQueryEventData) eventData).getQuery(); + } + + @Override + public AbstractHistoryRecordComparator getHistoryRecordComparator() { + return new MySqlHistoryRecordComparator(connectorConfig.gtidSourceFilter()); + } + + @Override + public IncrementalSnapshotContext getIncrementalSnapshotContext() { + if (connectorConfig.isReadOnlyConnection()) { + return new MySqlReadOnlyIncrementalSnapshotContext<>(); + } + return new SignalBasedIncrementalSnapshotContext<>(); + } + + @Override + @SuppressWarnings("unchecked") + public Long getReadOnlyIncrementalSnapshotSignalOffset(MySqlOffsetContext previousOffsets) { + return ((MySqlReadOnlyIncrementalSnapshotContext) previousOffsets.getIncrementalSnapshotContext()).getSignalOffset(); + } + + @Override + public IncrementalSnapshotChangeEventSource createIncrementalSnapshotChangeEventSource( + MySqlConnectorConfig connectorConfig, + AbstractConnectorConnection connection, + EventDispatcher dispatcher, + MySqlDatabaseSchema schema, + Clock clock, + SnapshotProgressListener snapshotProgressListener, + DataChangeEventListener dataChangeEventListener, + NotificationService notificationService) { + return new MySqlReadOnlyIncrementalSnapshotChangeEventSource<>( + connectorConfig, + connection, + dispatcher, + schema, + clock, + snapshotProgressListener, + dataChangeEventListener, + notificationService); + } + + private MySqlFieldReader resolveFieldReader() { + // todo: this null check is needed for the connection validation (try to rework) + return connectorConfig != null && connectorConfig.useCursorFetch() + ? new MySqlBinaryProtocolFieldReader(connectorConfig) + : new MySqlTextProtocolFieldReader(connectorConfig); + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlGtidSet.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlGtidSet.java new file mode 100644 index 000000000..cd07376eb --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlGtidSet.java @@ -0,0 +1,472 @@ +/* + * 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.mysql.strategy.mysql; + +import java.util.AbstractMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.function.Predicate; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import io.debezium.annotation.Immutable; +import io.debezium.connector.mysql.GtidSet; + +/** + * Represents a set of MySQL GTIDs. + * + * This is an improvement ove {@link com.github.shyiko.mysql.binlog.GtidSet} that is immutable, and + * more properly supports comparisons. + * + * @author Chris Cranford, Randall Hauch + */ +@Immutable +public class MySqlGtidSet implements GtidSet { + + private final Map uuidSetsByServerId = new TreeMap<>(); // sorts on keys + public static Pattern GTID_DELIMITER = Pattern.compile(":"); + + public MySqlGtidSet(String gtids) { + if (gtids != null) { + gtids = gtids.replace("\n", "").replace("\r", ""); + new com.github.shyiko.mysql.binlog.GtidSet(gtids).getUUIDSets().forEach(uuidSet -> { + uuidSetsByServerId.put(uuidSet.getUUID(), new UUIDSet(uuidSet)); + }); + StringBuilder sb = new StringBuilder(); + uuidSetsByServerId.values().forEach(uuidSet -> { + if (sb.length() != 0) { + sb.append(','); + } + sb.append(uuidSet.toString()); + }); + } + } + + protected MySqlGtidSet(Map uuidSetsByServerId) { + this.uuidSetsByServerId.putAll(uuidSetsByServerId); + } + + @Override + public boolean isEmpty() { + return uuidSetsByServerId.isEmpty(); + } + + public MySqlGtidSet retainAll(Predicate sourceFilter) { + if (sourceFilter == null) { + return this; + } + Map newSets = this.uuidSetsByServerId.entrySet() + .stream() + .filter(entry -> sourceFilter.test(entry.getKey())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return new MySqlGtidSet(newSets); + } + + @Override + public boolean isContainedWithin(GtidSet other) { + if (other == null) { + return false; + } + if (this.equals(other)) { + return true; + } + final MySqlGtidSet theOther = (MySqlGtidSet) other; + for (UUIDSet uuidSet : uuidSetsByServerId.values()) { + UUIDSet thatSet = theOther.forServerWithId(uuidSet.getUUID()); + if (!uuidSet.isContainedWithin(thatSet)) { + return false; + } + } + return true; + } + + @Override + public GtidSet with(GtidSet other) { + final MySqlGtidSet theOther = (MySqlGtidSet) other; + if (theOther == null || theOther.uuidSetsByServerId.isEmpty()) { + return this; + } + Map newSet = new HashMap<>(); + newSet.putAll(this.uuidSetsByServerId); + newSet.putAll(theOther.uuidSetsByServerId); + return new MySqlGtidSet(newSet); + } + + @Override + public MySqlGtidSet getGtidSetBeginning() { + Map newSet = new HashMap<>(); + + for (UUIDSet uuidSet : uuidSetsByServerId.values()) { + newSet.put(uuidSet.getUUID(), uuidSet.asIntervalBeginning()); + } + + return new MySqlGtidSet(newSet); + } + + @Override + public boolean contains(String gtid) { + String[] split = GTID_DELIMITER.split(gtid); + String sourceId = split[0]; + UUIDSet uuidSet = forServerWithId(sourceId); + if (uuidSet == null) { + return false; + } + long transactionId = Long.parseLong(split[1]); + return uuidSet.contains(transactionId); + } + + @Override + public MySqlGtidSet subtract(GtidSet other) { + if (other == null) { + return this; + } + final MySqlGtidSet theOther = (MySqlGtidSet) other; + Map newSets = this.uuidSetsByServerId.entrySet() + .stream() + .filter(entry -> !entry.getValue().isContainedWithin(theOther.forServerWithId(entry.getKey()))) + .map(entry -> new AbstractMap.SimpleEntry<>(entry.getKey(), entry.getValue().subtract(theOther.forServerWithId(entry.getKey())))) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + return new MySqlGtidSet(newSets); + } + + /** + * Get an immutable collection of the {@link UUIDSet range of GTIDs for a single server}. + * + * @return the {@link UUIDSet GTID ranges for each server}; never null + */ + public Collection getUUIDSets() { + return Collections.unmodifiableCollection(uuidSetsByServerId.values()); + } + + /** + * Find the {@link UUIDSet} for the server with the specified Uuid. + * + * @param uuid the Uuid of the server + * @return the {@link UUIDSet} for the identified server, or {@code null} if there are no GTIDs from that server. + */ + public UUIDSet forServerWithId(String uuid) { + return uuidSetsByServerId.get(uuid); + } + + @Override + public int hashCode() { + return uuidSetsByServerId.keySet().hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof GtidSet) { + MySqlGtidSet that = (MySqlGtidSet) obj; + return this.uuidSetsByServerId.equals(that.uuidSetsByServerId); + } + return false; + } + + @Override + public String toString() { + List gtids = new ArrayList(); + for (UUIDSet uuidSet : uuidSetsByServerId.values()) { + gtids.add(uuidSet.toString()); + } + return String.join(",", gtids); + } + + /** + * A range of GTIDs for a single server with a specific Uuid. + */ + @Immutable + public static class UUIDSet { + + private final String uuid; + private final LinkedList intervals = new LinkedList<>(); + + protected UUIDSet(com.github.shyiko.mysql.binlog.GtidSet.UUIDSet uuidSet) { + this.uuid = uuidSet.getUUID(); + uuidSet.getIntervals().forEach(interval -> { + intervals.add(new Interval(interval.getStart(), interval.getEnd())); + }); + Collections.sort(this.intervals); + if (this.intervals.size() > 1) { + // Collapse adjacent intervals ... + for (int i = intervals.size() - 1; i != 0; --i) { + Interval before = this.intervals.get(i - 1); + Interval after = this.intervals.get(i); + if ((before.getEnd() + 1) == after.getStart()) { + this.intervals.set(i - 1, new Interval(before.getStart(), after.getEnd())); + this.intervals.remove(i); + } + } + } + } + + protected UUIDSet(String uuid, Interval interval) { + this.uuid = uuid; + this.intervals.add(interval); + } + + protected UUIDSet(String uuid, List intervals) { + this.uuid = uuid; + this.intervals.addAll(intervals); + } + + public UUIDSet asIntervalBeginning() { + Interval start = new Interval(intervals.get(0).getStart(), intervals.get(0).getStart()); + return new UUIDSet(this.uuid, start); + } + + /** + * Get the Uuid for the server that generated the GTIDs. + * + * @return the server's Uuid; never null + */ + public String getUUID() { + return uuid; + } + + /** + * Get the intervals of transaction numbers. + * + * @return the immutable transaction intervals; never null + */ + public List getIntervals() { + return Collections.unmodifiableList(intervals); + } + + /** + * Determine if the set of transaction numbers from this server is completely within the set of transaction numbers from + * the set of transaction numbers in the supplied set. + * + * @param other the set to compare with this set + * @return {@code true} if this server's transaction numbers are a subset of the transaction numbers of the supplied set, + * or false otherwise + */ + public boolean isContainedWithin(UUIDSet other) { + if (other == null) { + return false; + } + if (!this.getUUID().equalsIgnoreCase(other.getUUID())) { + // Not even the same server ... + return false; + } + if (this.intervals.isEmpty()) { + return true; + } + if (other.intervals.isEmpty()) { + return false; + } + assert this.intervals.size() > 0; + assert other.intervals.size() > 0; + + // Every interval in this must be within an interval of the other ... + for (Interval thisInterval : this.intervals) { + boolean found = false; + for (Interval otherInterval : other.intervals) { + if (thisInterval.isContainedWithin(otherInterval)) { + found = true; + break; + } + } + if (!found) { + return false; // didn't find a match + } + } + return true; + } + + public boolean contains(long transactionId) { + for (Interval interval : this.intervals) { + if (interval.contains(transactionId)) { + return true; + } + } + return false; + } + + @Override + public int hashCode() { + return uuid.hashCode(); + } + + @Override + public boolean equals(Object obj) { + if (obj == this) { + return true; + } + if (obj instanceof UUIDSet) { + UUIDSet that = (UUIDSet) obj; + return this.getUUID().equalsIgnoreCase(that.getUUID()) && this.getIntervals().equals(that.getIntervals()); + } + return super.equals(obj); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(uuid).append(':'); + Iterator iter = intervals.iterator(); + if (iter.hasNext()) { + sb.append(iter.next()); + } + while (iter.hasNext()) { + sb.append(':'); + sb.append(iter.next()); + } + return sb.toString(); + } + + public UUIDSet subtract(UUIDSet other) { + if (!uuid.equals(other.getUUID())) { + throw new IllegalArgumentException("UUIDSet subtraction is supported only within a single server UUID"); + } + List result = new ArrayList<>(); + for (Interval interval : intervals) { + result.addAll(interval.removeAll(other.getIntervals())); + } + return new UUIDSet(uuid, result); + } + } + + @Immutable + public static class Interval implements Comparable { + + private final long start; + private final long end; + + public Interval(long start, long end) { + this.start = start; + this.end = end; + } + + /** + * Get the starting transaction number in this interval. + * + * @return this interval's first transaction number + */ + public long getStart() { + return start; + } + + /** + * Get the ending transaction number in this interval. + * + * @return this interval's last transaction number + */ + public long getEnd() { + return end; + } + + /** + * Determine if this interval is completely within the supplied interval. + * + * @param other the interval to compare with + * @return {@code true} if the {@link #getStart() start} is greater than or equal to the supplied interval's + * {@link #getStart() start} and the {@link #getEnd() end} is less than or equal to the supplied interval's + * {@link #getEnd() end}, or {@code false} otherwise + */ + public boolean isContainedWithin(Interval other) { + if (other == this) { + return true; + } + if (other == null) { + return false; + } + return this.getStart() >= other.getStart() && this.getEnd() <= other.getEnd(); + } + + public boolean contains(long transactionId) { + return getStart() <= transactionId && transactionId <= getEnd(); + } + + public boolean contains(Interval other) { + return getStart() <= other.getStart() && getEnd() >= other.getEnd(); + } + + public boolean nonintersecting(Interval other) { + return other.getEnd() < this.getStart() || other.getStart() > this.getEnd(); + } + + public List remove(Interval other) { + if (nonintersecting(other)) { + return Collections.singletonList(this); + } + if (other.contains(this)) { + return Collections.emptyList(); + } + List result = new LinkedList<>(); + if (this.getStart() < other.getStart()) { + Interval part = new Interval(this.getStart(), other.getStart() - 1); + result.add(part); + } + if (other.getEnd() < this.getEnd()) { + Interval part = new Interval(other.getEnd() + 1, this.getEnd()); + result.add(part); + } + return result; + } + + public List removeAll(List otherIntervals) { + List thisIntervals = new LinkedList<>(); + thisIntervals.add(this); + List result = new LinkedList<>(); + result.add(this); + for (Interval other : otherIntervals) { + result = new LinkedList<>(); + for (Interval thisInterval : thisIntervals) { + result.addAll(thisInterval.remove(other)); + } + thisIntervals = result; + } + return result; + } + + @Override + public int compareTo(Interval that) { + if (that == this) { + return 0; + } + long diff = this.start - that.start; + if (diff > Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + if (diff < Integer.MIN_VALUE) { + return Integer.MIN_VALUE; + } + return (int) diff; + } + + @Override + public int hashCode() { + return (int) getStart(); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj instanceof Interval) { + Interval that = (Interval) obj; + return this.getStart() == that.getStart() && this.getEnd() == that.getEnd(); + } + return false; + } + + @Override + public String toString() { + return "" + getStart() + "-" + getEnd(); + } + } +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlHistoryRecordComparator.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlHistoryRecordComparator.java new file mode 100644 index 000000000..4dc5049c8 --- /dev/null +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlHistoryRecordComparator.java @@ -0,0 +1,27 @@ +/* + * 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.mysql.strategy.mysql; + +import java.util.function.Predicate; + +import io.debezium.connector.mysql.GtidSet; +import io.debezium.connector.mysql.strategy.AbstractHistoryRecordComparator; + +/** + * @author Chris Cranford + */ +public class MySqlHistoryRecordComparator extends AbstractHistoryRecordComparator { + + public MySqlHistoryRecordComparator(Predicate gtidSourceFilter) { + super(gtidSourceFilter); + } + + @Override + protected GtidSet createGtidSet(String gtidSet) { + return new MySqlGtidSet(gtidSet); + } + +} diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java similarity index 97% rename from debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java rename to debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java index 7014936b5..d87381f7f 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotChangeEventSource.java @@ -3,7 +3,7 @@ * * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 */ -package io.debezium.connector.mysql; +package io.debezium.connector.mysql.strategy.mysql; import java.sql.SQLException; import java.util.List; @@ -14,6 +14,9 @@ import org.slf4j.LoggerFactory; import io.debezium.DebeziumException; +import io.debezium.connector.mysql.MySqlOffsetContext; +import io.debezium.connector.mysql.MySqlPartition; +import io.debezium.connector.mysql.SourceInfo; import io.debezium.jdbc.JdbcConnection; import io.debezium.pipeline.EventDispatcher; import io.debezium.pipeline.notification.NotificationService; @@ -174,14 +177,14 @@ protected void updateHighWatermark() { getExecutedGtidSet(getContext()::setHighWatermark); } - private void getExecutedGtidSet(Consumer watermark) { + private void getExecutedGtidSet(Consumer watermark) { try { jdbcConnection.query(SHOW_MASTER_STMT, rs -> { if (rs.next()) { if (rs.getMetaData().getColumnCount() > 4) { // This column exists only in MySQL 5.6.5 or later ... final String gtidSet = rs.getString(5); // GTID set, may be null, blank, or contain a GTID set - watermark.accept(new GtidSet(gtidSet)); + watermark.accept(new MySqlGtidSet(gtidSet)); } else { throw new UnsupportedOperationException("Need to add support for executed GTIDs for versions prior to 5.6.5"); diff --git a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotContext.java b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotContext.java similarity index 87% rename from debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotContext.java rename to debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotContext.java index f3093f0e2..662b820d9 100644 --- a/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/MySqlReadOnlyIncrementalSnapshotContext.java +++ b/debezium-connector-mysql/src/main/java/io/debezium/connector/mysql/strategy/mysql/MySqlReadOnlyIncrementalSnapshotContext.java @@ -3,28 +3,27 @@ * * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 */ -package io.debezium.connector.mysql; +package io.debezium.connector.mysql.strategy.mysql; -import static io.debezium.connector.mysql.GtidSet.GTID_DELIMITER; +import static io.debezium.connector.mysql.strategy.mysql.MySqlGtidSet.GTID_DELIMITER; import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import io.debezium.annotation.NotThreadSafe; +import io.debezium.connector.mysql.SourceInfo; import io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotContext; import io.debezium.pipeline.source.snapshot.incremental.IncrementalSnapshotContext; import io.debezium.pipeline.spi.OffsetContext; -@NotThreadSafe public class MySqlReadOnlyIncrementalSnapshotContext extends AbstractIncrementalSnapshotContext { private static final Logger LOGGER = LoggerFactory.getLogger(MySqlReadOnlyIncrementalSnapshotContext.class); - private GtidSet previousLowWatermark; - private GtidSet previousHighWatermark; - private GtidSet lowWatermark; - private GtidSet highWatermark; + private MySqlGtidSet previousLowWatermark; + private MySqlGtidSet previousHighWatermark; + private MySqlGtidSet lowWatermark; + private MySqlGtidSet highWatermark; private Long signalOffset; public static final String SIGNAL_OFFSET = INCREMENTAL_SNAPSHOT_KEY + "_signal_offset"; @@ -53,11 +52,11 @@ public static MySqlReadOnlyIncrementalSnapshotContext load(Map return context; } - public void setLowWatermark(GtidSet lowWatermark) { + public void setLowWatermark(MySqlGtidSet lowWatermark) { this.lowWatermark = lowWatermark; } - public void setHighWatermark(GtidSet highWatermark) { + public void setHighWatermark(MySqlGtidSet highWatermark) { this.highWatermark = highWatermark.subtract(lowWatermark); } @@ -89,10 +88,10 @@ public boolean reachedHighWatermark(String currentGtid) { return true; } String[] gtid = GTID_DELIMITER.split(currentGtid); - GtidSet.UUIDSet uuidSet = getUuidSet(gtid[0]); + MySqlGtidSet.UUIDSet uuidSet = getUuidSet(gtid[0]); if (uuidSet != null) { long maxTransactionId = uuidSet.getIntervals().stream() - .mapToLong(GtidSet.Interval::getEnd) + .mapToLong(MySqlGtidSet.Interval::getEnd) .max() .getAsLong(); if (maxTransactionId <= Long.parseLong(gtid[1])) { @@ -115,7 +114,7 @@ public void closeWindow() { lowWatermark = null; } - private GtidSet.UUIDSet getUuidSet(String serverId) { + private MySqlGtidSet.UUIDSet getUuidSet(String serverId) { return highWatermark.getUUIDSets().isEmpty() ? lowWatermark.forServerWithId(serverId) : highWatermark.forServerWithId(serverId); } diff --git a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/GtidSetTest.java b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/GtidSetTest.java index 7d9758128..09f72dd91 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/GtidSetTest.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/GtidSetTest.java @@ -16,8 +16,9 @@ import org.junit.Test; -import io.debezium.connector.mysql.GtidSet.Interval; -import io.debezium.connector.mysql.GtidSet.UUIDSet; +import io.debezium.connector.mysql.strategy.mysql.MySqlGtidSet; +import io.debezium.connector.mysql.strategy.mysql.MySqlGtidSet.Interval; +import io.debezium.connector.mysql.strategy.mysql.MySqlGtidSet.UUIDSet; import io.debezium.util.Collect; /** @@ -28,11 +29,11 @@ public class GtidSetTest { private static final String UUID1 = "24bc7850-2c16-11e6-a073-0242ac110002"; - private GtidSet gtids; + private MySqlGtidSet gtids; @Test public void shouldCreateSetWithSingleInterval() { - gtids = new GtidSet(UUID1 + ":1-191"); + gtids = new MySqlGtidSet(UUID1 + ":1-191"); asertIntervalCount(UUID1, 1); asertIntervalExists(UUID1, 1, 191); asertFirstInterval(UUID1, 1, 191); @@ -42,7 +43,7 @@ public void shouldCreateSetWithSingleInterval() { @Test public void shouldCollapseAdjacentIntervals() { - gtids = new GtidSet(UUID1 + ":1-191:192-199"); + gtids = new MySqlGtidSet(UUID1 + ":1-191:192-199"); asertIntervalCount(UUID1, 1); asertIntervalExists(UUID1, 1, 199); asertFirstInterval(UUID1, 1, 199); @@ -52,7 +53,7 @@ public void shouldCollapseAdjacentIntervals() { @Test public void shouldNotCollapseNonAdjacentIntervals() { - gtids = new GtidSet(UUID1 + ":1-191:193-199"); + gtids = new MySqlGtidSet(UUID1 + ":1-191:193-199"); asertIntervalCount(UUID1, 2); asertFirstInterval(UUID1, 1, 191); asertLastInterval(UUID1, 193, 199); @@ -61,7 +62,7 @@ public void shouldNotCollapseNonAdjacentIntervals() { @Test public void shouldCreateWithMultipleIntervals() { - gtids = new GtidSet(UUID1 + ":1-191:193-199:1000-1033"); + gtids = new MySqlGtidSet(UUID1 + ":1-191:193-199:1000-1033"); asertIntervalCount(UUID1, 3); asertFirstInterval(UUID1, 1, 191); asertIntervalExists(UUID1, 193, 199); @@ -71,7 +72,7 @@ public void shouldCreateWithMultipleIntervals() { @Test public void shouldCreateWithMultipleIntervalsThatMayBeAdjacent() { - gtids = new GtidSet(UUID1 + ":1-191:192-199:1000-1033:1035-1036:1038-1039"); + gtids = new MySqlGtidSet(UUID1 + ":1-191:192-199:1000-1033:1035-1036:1038-1039"); asertIntervalCount(UUID1, 4); asertFirstInterval(UUID1, 1, 199); asertIntervalExists(UUID1, 1000, 1033); @@ -82,19 +83,19 @@ public void shouldCreateWithMultipleIntervalsThatMayBeAdjacent() { @Test public void shouldCorrectlyDetermineIfSimpleGtidSetIsContainedWithinAnother() { - gtids = new GtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); - assertThat(gtids.isContainedWithin(new GtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"))).isTrue(); - assertThat(gtids.isContainedWithin(new GtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-42"))).isTrue(); - assertThat(gtids.isContainedWithin(new GtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:2-41"))).isFalse(); - assertThat(gtids.isContainedWithin(new GtidSet("7145bf69-d1ca-11e5-a588-0242ac110004:1"))).isFalse(); + gtids = new MySqlGtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); + assertThat(gtids.isContainedWithin(new MySqlGtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"))).isTrue(); + assertThat(gtids.isContainedWithin(new MySqlGtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-42"))).isTrue(); + assertThat(gtids.isContainedWithin(new MySqlGtidSet("7c1de3f2-3fd2-11e6-9cdc-42010af000bc:2-41"))).isFalse(); + assertThat(gtids.isContainedWithin(new MySqlGtidSet("7145bf69-d1ca-11e5-a588-0242ac110004:1"))).isFalse(); } @Test public void shouldCorrectlyDetermineIfComplexGtidSetIsContainedWithinAnother() { - GtidSet connector = new GtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + MySqlGtidSet connector = new MySqlGtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + "7145bf69-d1ca-11e5-a588-0242ac110004:1-3200," + "7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); - GtidSet server = new GtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + MySqlGtidSet server = new MySqlGtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + "7145bf69-d1ca-11e5-a588-0242ac110004:1-3202," + "7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); assertThat(connector.isContainedWithin(server)).isTrue(); @@ -102,12 +103,12 @@ public void shouldCorrectlyDetermineIfComplexGtidSetIsContainedWithinAnother() { @Test public void shouldCorrectlyDetermineIfComplexGtidSetWithVariousLineSeparatorsIsContainedWithinAnother() { - GtidSet connector = new GtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + GtidSet connector = new MySqlGtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + "7145bf69-d1ca-11e5-a588-0242ac110004:1-3200," + "7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); Arrays.stream(new String[]{ "\r\n", "\n", "\r" }) .forEach(separator -> { - GtidSet server = new GtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + separator + + GtidSet server = new MySqlGtidSet("036d85a9-64e5-11e6-9b48-42010af0000c:1-2," + separator + "7145bf69-d1ca-11e5-a588-0242ac110004:1-3202," + separator + "7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-41"); assertThat(connector.isContainedWithin(server)).isTrue(); @@ -122,12 +123,12 @@ public void shouldFilterServerUuids() { Collection keepers = Collect.arrayListOf("036d85a9-64e5-11e6-9b48-42010af0000c", "7c1de3f2-3fd2-11e6-9cdc-42010af000bc", "wont-be-found"); - GtidSet original = new GtidSet(gtidStr); + MySqlGtidSet original = new MySqlGtidSet(gtidStr); assertThat(original.forServerWithId("036d85a9-64e5-11e6-9b48-42010af0000c")).isNotNull(); assertThat(original.forServerWithId("7c1de3f2-3fd2-11e6-9cdc-42010af000bc")).isNotNull(); assertThat(original.forServerWithId("7145bf69-d1ca-11e5-a588-0242ac110004")).isNotNull(); - GtidSet filtered = original.retainAll(keepers::contains); + MySqlGtidSet filtered = original.retainAll(keepers::contains); List actualUuids = filtered.getUUIDSets().stream().map(UUIDSet::getUUID).collect(Collectors.toList()); assertThat(keepers.containsAll(actualUuids)).isTrue(); assertThat(filtered.forServerWithId("7145bf69-d1ca-11e5-a588-0242ac110004")).isNull(); @@ -144,11 +145,11 @@ public void subtract() { String diff = "036d85a9-64e5-11e6-9b48-42010af0000c:21," + "7145bf69-d1ca-11e5-a588-0242ac110004:4500," + "7c1de3f2-3fd2-11e6-9cdc-42010af000bc:1-4:9-11:19-24:66-70:80-100"; - GtidSet gtidSet1 = new GtidSet(gtidStr1); - GtidSet gtidSet2 = new GtidSet(gtidStr2); + MySqlGtidSet gtidSet1 = new MySqlGtidSet(gtidStr1); + MySqlGtidSet gtidSet2 = new MySqlGtidSet(gtidStr2); - GtidSet gtidSetDiff = gtidSet2.subtract(gtidSet1); - GtidSet expectedDiff = new GtidSet(diff); + MySqlGtidSet gtidSetDiff = gtidSet2.subtract(gtidSet1); + MySqlGtidSet expectedDiff = new MySqlGtidSet(diff); assertThat(gtidSetDiff).isEqualTo(expectedDiff); } diff --git a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlDatabaseSchemaTest.java b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlDatabaseSchemaTest.java index 5bffb65e8..c0b896a46 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlDatabaseSchemaTest.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlDatabaseSchemaTest.java @@ -68,7 +68,8 @@ private MySqlDatabaseSchema getSchema(Configuration config) { BigIntUnsignedMode.LONG, BinaryHandlingMode.BYTES, MySqlValueConverters::adjustTemporal, - MySqlValueConverters::defaultParsingErrorHandler); + MySqlValueConverters::defaultParsingErrorHandler, + connectorConfig.getConnectorAdapter()); return new MySqlDatabaseSchema( connectorConfig, mySqlValueConverters, diff --git a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlValueConvertersTest.java b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlValueConvertersTest.java index aa8bd2b62..1043f0bc0 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlValueConvertersTest.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/MySqlValueConvertersTest.java @@ -119,7 +119,7 @@ public void testSkipInvalidJsonValues() { TemporalPrecisionMode.CONNECT, JdbcValueConverters.BigIntUnsignedMode.LONG, BinaryHandlingMode.BYTES, x -> x, (message, exception) -> { errorCount.incrementAndGet(); - }); + }, null); DdlParser parser = new MySqlAntlrDdlParser(); Tables tables = new Tables(); @@ -148,7 +148,7 @@ public void testErrorOnInvalidJsonValues() { TemporalPrecisionMode.CONNECT, JdbcValueConverters.BigIntUnsignedMode.LONG, BinaryHandlingMode.BYTES, x -> x, (message, exception) -> { throw new DebeziumException(message, exception); - }); + }, null); DdlParser parser = new MySqlAntlrDdlParser(); Tables tables = new Tables(); @@ -171,7 +171,7 @@ public void testFallbackDecimalValueScale() { TemporalPrecisionMode.CONNECT, JdbcValueConverters.BigIntUnsignedMode.LONG, BinaryHandlingMode.BYTES, x -> x, (message, exception) -> { throw new DebeziumException(message, exception); - }); + }, null); DdlParser parser = new MySqlAntlrDdlParser(); Tables tables = new Tables(); @@ -194,7 +194,7 @@ public void testZonedDateTimeWithMicrosecondPrecision() { TemporalPrecisionMode.ADAPTIVE_TIME_MICROSECONDS, JdbcValueConverters.BigIntUnsignedMode.LONG, BinaryHandlingMode.BYTES, x -> x, (message, exception) -> { throw new DebeziumException(message, exception); - }); + }, null); DdlParser parser = new MySqlAntlrDdlParser(); Tables tables = new Tables(); diff --git a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/SourceInfoTest.java b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/SourceInfoTest.java index fae573e2f..eccd439ea 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/SourceInfoTest.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/SourceInfoTest.java @@ -26,6 +26,7 @@ import io.debezium.config.CommonConnectorConfig; import io.debezium.config.Configuration; import io.debezium.connector.AbstractSourceInfoStructMaker; +import io.debezium.connector.mysql.strategy.mysql.MySqlHistoryRecordComparator; import io.debezium.data.VerifyRecord; import io.debezium.doc.FixFor; import io.debezium.document.Document; diff --git a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/UniqueDatabase.java b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/UniqueDatabase.java index e633db5f0..eb634523c 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/UniqueDatabase.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/connector/mysql/UniqueDatabase.java @@ -220,6 +220,9 @@ public Configuration.Builder defaultJdbcConfigBuilder() { builder.with(FileSchemaHistory.FILE_PATH, dbHistoryPath); } + String connectorAdapter = System.getProperty("connector.adapter", "mysql"); + builder.with(MySqlConnectorConfig.CONNECTOR_ADAPTER, connectorAdapter); + return builder; } diff --git a/debezium-connector-mysql/src/test/java/io/debezium/relational/history/KafkaSchemaHistoryTest.java b/debezium-connector-mysql/src/test/java/io/debezium/relational/history/KafkaSchemaHistoryTest.java index 47cdfadf3..37f3f4d81 100644 --- a/debezium-connector-mysql/src/test/java/io/debezium/relational/history/KafkaSchemaHistoryTest.java +++ b/debezium-connector-mysql/src/test/java/io/debezium/relational/history/KafkaSchemaHistoryTest.java @@ -31,9 +31,9 @@ import io.debezium.connector.mysql.MySqlConnectorConfig; import io.debezium.connector.mysql.MySqlOffsetContext; import io.debezium.connector.mysql.MySqlPartition; -import io.debezium.connector.mysql.MySqlReadOnlyIncrementalSnapshotContext; import io.debezium.connector.mysql.SourceInfo; import io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser; +import io.debezium.connector.mysql.strategy.mysql.MySqlReadOnlyIncrementalSnapshotContext; import io.debezium.doc.FixFor; import io.debezium.junit.logging.LogInterceptor; import io.debezium.kafka.KafkaCluster;