mysql
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java
new file mode 100644
index 00000000000..1d0f27106b2
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/io/debezium/connector/mysql/MySqlStreamingChangeEventSource.java
@@ -0,0 +1,1580 @@
+/*
+ * 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 com.github.shyiko.mysql.binlog.BinaryLogClient;
+import com.github.shyiko.mysql.binlog.BinaryLogClient.LifecycleListener;
+import com.github.shyiko.mysql.binlog.event.DeleteRowsEventData;
+import com.github.shyiko.mysql.binlog.event.Event;
+import com.github.shyiko.mysql.binlog.event.EventData;
+import com.github.shyiko.mysql.binlog.event.EventHeader;
+import com.github.shyiko.mysql.binlog.event.EventHeaderV4;
+import com.github.shyiko.mysql.binlog.event.EventType;
+import com.github.shyiko.mysql.binlog.event.GtidEventData;
+import com.github.shyiko.mysql.binlog.event.QueryEventData;
+import com.github.shyiko.mysql.binlog.event.RotateEventData;
+import com.github.shyiko.mysql.binlog.event.RowsQueryEventData;
+import com.github.shyiko.mysql.binlog.event.TableMapEventData;
+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;
+import io.debezium.annotation.SingleThreadAccess;
+import io.debezium.config.CommonConnectorConfig.EventProcessingFailureHandlingMode;
+import io.debezium.config.Configuration;
+import io.debezium.connector.mysql.MySqlConnectorConfig.GtidNewChannelPosition;
+import io.debezium.connector.mysql.MySqlConnectorConfig.SecureConnectionMode;
+import io.debezium.connector.mysql.util.ErrorMessageUtils;
+import io.debezium.data.Envelope.Operation;
+import io.debezium.function.BlockingConsumer;
+import io.debezium.pipeline.ErrorHandler;
+import io.debezium.pipeline.EventDispatcher;
+import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
+import io.debezium.relational.TableId;
+import io.debezium.schema.SchemaChangeEvent;
+import io.debezium.util.Clock;
+import io.debezium.util.Metronome;
+import io.debezium.util.Strings;
+import io.debezium.util.Threads;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.slf4j.event.Level;
+
+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 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;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Predicate;
+
+import static io.debezium.util.Strings.isNullOrEmpty;
+
+/**
+ * Copied from mysql-cdc to be compatible with OceanBase Binlog Service.
+ *
+ * Copied from Debezium project(1.9.8.Final) to fix
+ * https://github.com/ververica/flink-cdc-connectors/issues/1944.
+ *
+ *
Line 1427-1433 : Adjust GTID merging logic to support recovering from job which previously
+ * specifying starting offset on start.
+ *
+ *
Line 1485 : Add more error details for some exceptions.
+ */
+public class MySqlStreamingChangeEventSource
+ implements StreamingChangeEventSource {
+
+ private static final Logger LOGGER =
+ LoggerFactory.getLogger(MySqlStreamingChangeEventSource.class);
+
+ private static final String KEEPALIVE_THREAD_NAME = "blc-keepalive";
+
+ private final EnumMap> eventHandlers =
+ new EnumMap<>(EventType.class);
+ private final BinaryLogClient client;
+ private final MySqlStreamingChangeEventSourceMetrics metrics;
+ private final Clock clock;
+ private final EventProcessingFailureHandlingMode eventDeserializationFailureHandlingMode;
+ private final EventProcessingFailureHandlingMode inconsistentSchemaHandlingMode;
+
+ private int startingRowNumber = 0;
+ private long initialEventsToSkip = 0L;
+ private boolean skipEvent = false;
+ private boolean ignoreDmlEventByGtidSource = false;
+ private final Predicate gtidDmlSourceFilter;
+ 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 EventDispatcher eventDispatcher;
+ private final ErrorHandler errorHandler;
+
+ @SingleThreadAccess("binlog client thread")
+ private Instant eventTimestamp;
+
+ /** Describe binlog position. */
+ public static class BinlogPosition {
+ final String filename;
+ final long position;
+
+ public BinlogPosition(String filename, long position) {
+ assert filename != null;
+
+ this.filename = filename;
+ this.position = position;
+ }
+
+ public String getFilename() {
+ return filename;
+ }
+
+ public long getPosition() {
+ return position;
+ }
+
+ @Override
+ public String toString() {
+ return filename + "/" + position;
+ }
+
+ @Override
+ public int hashCode() {
+ final int prime = 31;
+ int result = 1;
+ result = prime * result + filename.hashCode();
+ result = prime * result + (int) (position ^ (position >>> 32));
+ return result;
+ }
+
+ @Override
+ public boolean equals(Object obj) {
+ if (this == obj) {
+ return true;
+ }
+ if (obj == null) {
+ return false;
+ }
+ if (getClass() != obj.getClass()) {
+ return false;
+ }
+ BinlogPosition other = (BinlogPosition) obj;
+ if (!filename.equals(other.filename)) {
+ return false;
+ }
+ if (position != other.position) {
+ return false;
+ }
+ return true;
+ }
+ }
+
+ @FunctionalInterface
+ private interface BinlogChangeEmitter {
+ void emit(TableId tableId, T data) throws InterruptedException;
+ }
+
+ public MySqlStreamingChangeEventSource(
+ MySqlConnectorConfig connectorConfig,
+ MySqlConnection connection,
+ EventDispatcher dispatcher,
+ ErrorHandler errorHandler,
+ Clock clock,
+ MySqlTaskContext taskContext,
+ MySqlStreamingChangeEventSourceMetrics metrics) {
+
+ this.taskContext = taskContext;
+ this.connectorConfig = connectorConfig;
+ this.connection = connection;
+ this.clock = clock;
+ this.eventDispatcher = dispatcher;
+ this.errorHandler = errorHandler;
+ this.metrics = metrics;
+
+ eventDeserializationFailureHandlingMode =
+ connectorConfig.getEventProcessingFailureHandlingMode();
+ inconsistentSchemaHandlingMode = connectorConfig.inconsistentSchemaFailureHandlingMode();
+
+ // Set up the log reader ...
+ client = taskContext.getBinaryLogClient();
+ // BinaryLogClient will overwrite thread names later
+ client.setThreadFactory(
+ 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);
+ }
+ }
+ 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));
+
+ 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-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));
+ eventDeserializer.setEventDataDeserializer(
+ EventType.UPDATE_ROWS,
+ new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId));
+ eventDeserializer.setEventDataDeserializer(
+ EventType.DELETE_ROWS,
+ new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId));
+ eventDeserializer.setEventDataDeserializer(
+ EventType.EXT_WRITE_ROWS,
+ new RowDeserializers.WriteRowsDeserializer(tableMapEventByTableId)
+ .setMayContainExtraInformation(true));
+ eventDeserializer.setEventDataDeserializer(
+ EventType.EXT_UPDATE_ROWS,
+ new RowDeserializers.UpdateRowsDeserializer(tableMapEventByTableId)
+ .setMayContainExtraInformation(true));
+ eventDeserializer.setEventDataDeserializer(
+ EventType.EXT_DELETE_ROWS,
+ new RowDeserializers.DeleteRowsDeserializer(tableMapEventByTableId)
+ .setMayContainExtraInformation(true));
+ client.setEventDeserializer(eventDeserializer);
+ }
+
+ protected void onEvent(MySqlOffsetContext offsetContext, Event event) {
+ long ts = 0;
+
+ if (event.getHeader().getEventType() == EventType.HEARTBEAT) {
+ // HEARTBEAT events have no timestamp but are fired only when
+ // there is no traffic on the connection which means we are caught-up
+ // https://dev.mysql.com/doc/internals/en/heartbeat-event.html
+ metrics.setMilliSecondsBehindSource(ts);
+ return;
+ }
+
+ // MySQL has seconds resolution but mysql-binlog-connector-java returns
+ // a value in milliseconds
+ long eventTs = event.getHeader().getTimestamp();
+
+ if (eventTs == 0) {
+ LOGGER.trace("Received unexpected event with 0 timestamp: {}", event);
+ return;
+ }
+
+ ts = clock.currentTimeInMillis() - eventTs;
+ LOGGER.trace("Current milliseconds behind source: {} ms", ts);
+ metrics.setMilliSecondsBehindSource(ts);
+ }
+
+ protected void ignoreEvent(MySqlOffsetContext offsetContext, Event event) {
+ LOGGER.trace("Ignoring event due to missing handler: {}", event);
+ }
+
+ protected void handleEvent(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) {
+ if (event == null) {
+ return;
+ }
+
+ final EventHeader eventHeader = event.getHeader();
+ // Update the source offset info. Note that the client returns the value in *milliseconds*,
+ // even though the binlog
+ // contains only *seconds* precision ...
+ // HEARTBEAT events have no timestamp; only set the timestamp if the event is not a
+ // HEARTBEAT
+ eventTimestamp =
+ !eventHeader.getEventType().equals(EventType.HEARTBEAT)
+ ? Instant.ofEpochMilli(eventHeader.getTimestamp())
+ : null;
+ offsetContext.setBinlogServerId(eventHeader.getServerId());
+
+ final EventType eventType = eventHeader.getEventType();
+ if (eventType == EventType.ROTATE) {
+ EventData eventData = event.getData();
+ RotateEventData rotateEventData;
+ if (eventData instanceof EventDeserializer.EventDataWrapper) {
+ rotateEventData =
+ (RotateEventData)
+ ((EventDeserializer.EventDataWrapper) eventData).getInternal();
+ } else {
+ rotateEventData = (RotateEventData) eventData;
+ }
+ offsetContext.setBinlogStartPoint(
+ rotateEventData.getBinlogFilename(), rotateEventData.getBinlogPosition());
+ } else if (eventHeader instanceof EventHeaderV4) {
+ EventHeaderV4 trackableEventHeader = (EventHeaderV4) eventHeader;
+ offsetContext.setEventPosition(
+ trackableEventHeader.getPosition(), trackableEventHeader.getEventLength());
+ }
+
+ // If there is a handler for this event, forward the event to it ...
+ try {
+ // Forward the event to the handler ...
+ eventHandlers
+ .getOrDefault(eventType, (e) -> ignoreEvent(offsetContext, e))
+ .accept(event);
+
+ // Generate heartbeat message if the time is right
+ eventDispatcher.dispatchHeartbeatEvent(partition, offsetContext);
+
+ // Capture that we've completed another event ...
+ // Skip HeratBeat event for OceanBase Binlog Service
+ if (eventType != EventType.HEARTBEAT) {
+ offsetContext.completeEvent();
+ }
+
+ // update last offset used for logging
+ lastOffset = offsetContext.getOffset();
+
+ if (skipEvent) {
+ // We're in the mode of skipping events and we just skipped this one, so decrement
+ // our skip count ...
+ --initialEventsToSkip;
+ skipEvent = initialEventsToSkip > 0;
+ }
+ } catch (RuntimeException e) {
+ // There was an error in the event handler, so propagate the failure to Kafka Connect
+ // ...
+ logStreamingSourceState();
+ errorHandler.setProducerThrowable(
+ new DebeziumException("Error processing binlog event", e));
+ // Do not stop the client, since Kafka Connect should stop the connector on it's own
+ // (and doing it here may cause problems the second time it is stopped).
+ // We can clear the listeners though so that we ignore all future events ...
+ eventHandlers.clear();
+ LOGGER.info(
+ "Error processing binlog event, and propagating to Kafka Connect so it stops this connector. Future binlog events read before connector is shutdown will be ignored.");
+ } catch (InterruptedException e) {
+ // Most likely because this reader was stopped and our thread was interrupted ...
+ Thread.currentThread().interrupt();
+ eventHandlers.clear();
+ LOGGER.info("Stopped processing binlog events due to thread interruption");
+ }
+ }
+
+ @SuppressWarnings("unchecked")
+ protected T unwrapData(Event event) {
+ EventData eventData = event.getData();
+ if (eventData instanceof EventDeserializer.EventDataWrapper) {
+ eventData = ((EventDeserializer.EventDataWrapper) eventData).getInternal();
+ }
+ return (T) eventData;
+ }
+
+ /**
+ * Handle the supplied event that signals that mysqld has stopped.
+ *
+ * @param event the server stopped event to be processed; may not be null
+ */
+ protected void handleServerStop(MySqlOffsetContext offsetContext, Event event) {
+ LOGGER.debug("Server stopped: {}", event);
+ }
+
+ /**
+ * Handle the supplied event that is sent by a primary to a replica to let the replica know that
+ * the primary is still alive. Not written to a binary log.
+ *
+ * @param event the server stopped event to be processed; may not be null
+ */
+ protected void handleServerHeartbeat(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ LOGGER.trace("Server heartbeat: {}", event);
+ eventDispatcher.dispatchServerHeartbeatEvent(partition, offsetContext);
+ }
+
+ /**
+ * Handle the supplied event that signals that an out of the ordinary event that occurred on the
+ * master. It notifies the replica that something happened on the primary that might cause data
+ * to be in an inconsistent state.
+ *
+ * @param event the server stopped event to be processed; may not be null
+ */
+ protected void handleServerIncident(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event) {
+ if (event.getData() instanceof EventDataDeserializationExceptionData) {
+ metrics.onErroneousEvent(partition, "source = " + event);
+ EventDataDeserializationExceptionData data = event.getData();
+
+ EventHeaderV4 eventHeader =
+ (EventHeaderV4)
+ data.getCause()
+ .getEventHeader(); // safe cast, instantiated that ourselves
+
+ // logging some additional context but not the exception itself, this will happen in
+ // handleEvent()
+ if (eventDeserializationFailureHandlingMode
+ == EventProcessingFailureHandlingMode.FAIL) {
+ LOGGER.error(
+ "Error while deserializing binlog event at offset {}.{}"
+ + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}",
+ offsetContext.getOffset(),
+ System.lineSeparator(),
+ eventHeader.getPosition(),
+ eventHeader.getNextPosition(),
+ offsetContext.getSource().binlogFilename());
+
+ throw new RuntimeException(data.getCause());
+ } else if (eventDeserializationFailureHandlingMode
+ == EventProcessingFailureHandlingMode.WARN) {
+ LOGGER.warn(
+ "Error while deserializing binlog event at offset {}.{}"
+ + "This exception will be ignored and the event be skipped.{}"
+ + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}",
+ offsetContext.getOffset(),
+ System.lineSeparator(),
+ System.lineSeparator(),
+ eventHeader.getPosition(),
+ eventHeader.getNextPosition(),
+ offsetContext.getSource().binlogFilename(),
+ data.getCause());
+ }
+ } else {
+ LOGGER.error("Server incident: {}", event);
+ }
+ }
+
+ /**
+ * Handle the supplied event with a {@link RotateEventData} that signals the logs are being
+ * rotated. This means that either the server was restarted, or the binlog has transitioned to a
+ * new file. In either case, subsequent table numbers will be different than those seen to this
+ * point.
+ *
+ * @param event the database change data event to be processed; may not be null
+ */
+ protected void handleRotateLogsEvent(MySqlOffsetContext offsetContext, Event event) {
+ LOGGER.debug("Rotating logs: {}", event);
+ RotateEventData command = unwrapData(event);
+ assert command != null;
+ taskContext.getSchema().clearTableMappings();
+ }
+
+ /**
+ * Handle the supplied event with a {@link GtidEventData} that signals the beginning of a GTID
+ * transaction. We don't yet know whether this transaction contains any events we're interested
+ * in, but we have to record it so that we know the position of this event and know we've
+ * processed the binlog to this point.
+ *
+ * Note that this captures the current GTID and complete GTID set, regardless of whether the
+ * connector is {@link MySqlTaskContext#gtidSourceFilter() filtering} the GTID set upon
+ * connection. We do this because we actually want to capture all GTID set values found in the
+ * binlog, whether or not we process them. However, only when we connect do we actually want to
+ * pass to MySQL only those GTID ranges that are applicable per the configuration.
+ *
+ * @param event the GTID event to be processed; may not be null
+ */
+ protected void handleGtidEvent(MySqlOffsetContext offsetContext, Event event) {
+ LOGGER.debug("GTID transaction: {}", event);
+ GtidEventData gtidEvent = unwrapData(event);
+ String gtid = gtidEvent.getGtid();
+ gtidSet.add(gtid);
+ offsetContext.startGtid(gtid, gtidSet.toString()); // rather than use the client's GTID set
+ ignoreDmlEventByGtidSource = false;
+ if (gtidDmlSourceFilter != null && gtid != null) {
+ String uuid = gtid.trim().substring(0, gtid.indexOf(":"));
+ if (!gtidDmlSourceFilter.test(uuid)) {
+ ignoreDmlEventByGtidSource = true;
+ }
+ }
+ metrics.onGtidChange(gtid);
+ }
+
+ /**
+ * Handle the supplied event with an {@link RowsQueryEventData} by recording the original SQL
+ * query that generated the event.
+ *
+ * @param event the database change data event to be processed; may not be null
+ */
+ protected void handleRowsQuery(MySqlOffsetContext offsetContext, Event event) {
+ // Unwrap the RowsQueryEvent
+ final RowsQueryEventData lastRowsQueryEventData = unwrapData(event);
+
+ // Set the query on the source
+ offsetContext.setQuery(lastRowsQueryEventData.getQuery());
+ }
+
+ /**
+ * Handle the supplied event with an {@link QueryEventData} by possibly recording the DDL
+ * statements as changes in the MySQL schemas.
+ *
+ * @param partition the partition in which the even occurred
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while recording the DDL statements
+ */
+ protected void handleQueryEvent(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ QueryEventData command = unwrapData(event);
+ LOGGER.debug("Received query command: {}", event);
+ String sql = command.getSql().trim();
+ if (sql.equalsIgnoreCase("BEGIN")) {
+ // We are starting a new transaction ...
+ offsetContext.startNextTransaction();
+ eventDispatcher.dispatchTransactionStartedEvent(
+ partition, offsetContext.getTransactionId(), offsetContext);
+ offsetContext.setBinlogThread(command.getThreadId());
+ if (initialEventsToSkip != 0) {
+ LOGGER.debug(
+ "Restarting partially-processed transaction; change events will not be created for the first {} events plus {} more rows in the next event",
+ initialEventsToSkip,
+ startingRowNumber);
+ // We are restarting, so we need to skip the events in this transaction that we
+ // processed previously...
+ skipEvent = true;
+ }
+ return;
+ }
+ if (sql.equalsIgnoreCase("COMMIT")) {
+ handleTransactionCompletion(partition, offsetContext, event);
+ return;
+ }
+
+ String upperCasedStatementBegin = Strings.getBegin(sql, 7).toUpperCase();
+
+ if (upperCasedStatementBegin.startsWith("XA ")) {
+ // This is an XA transaction, and we currently ignore these and do nothing ...
+ return;
+ }
+ if (connectorConfig.getDdlFilter().test(sql)) {
+ LOGGER.debug("DDL '{}' was filtered out of processing", sql);
+ return;
+ }
+ if (upperCasedStatementBegin.equals("INSERT ")
+ || upperCasedStatementBegin.equals("UPDATE ")
+ || upperCasedStatementBegin.equals("DELETE ")) {
+ LOGGER.warn(
+ "Received DML '"
+ + sql
+ + "' for processing, binlog probably contains events generated with statement or mixed based replication format");
+ return;
+ }
+ if (sql.equalsIgnoreCase("ROLLBACK")) {
+ // We have hit a ROLLBACK which is not supported
+ LOGGER.warn(
+ "Rollback statements cannot be handled without binlog buffering, the connector will fail. Please check '{}' to see how to enable buffering",
+ MySqlConnectorConfig.BUFFER_SIZE_FOR_BINLOG_READER.name());
+ }
+
+ final List schemaChangeEvents =
+ taskContext
+ .getSchema()
+ .parseStreamingDdl(
+ partition,
+ sql,
+ command.getDatabase(),
+ offsetContext,
+ clock.currentTimeAsInstant());
+ try {
+ for (SchemaChangeEvent schemaChangeEvent : schemaChangeEvents) {
+ if (taskContext.getSchema().skipSchemaChangeEvent(schemaChangeEvent)) {
+ continue;
+ }
+
+ final TableId tableId =
+ schemaChangeEvent.getTables().isEmpty()
+ ? null
+ : schemaChangeEvent.getTables().iterator().next().id();
+ eventDispatcher.dispatchSchemaChangeEvent(
+ partition,
+ tableId,
+ (receiver) -> {
+ try {
+ receiver.schemaChangeEvent(schemaChangeEvent);
+ } catch (Exception e) {
+ throw new DebeziumException(e);
+ }
+ });
+ }
+ } catch (InterruptedException e) {
+ LOGGER.info("Processing interrupted");
+ }
+ }
+
+ private void handleTransactionCompletion(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ // We are completing the transaction ...
+ eventDispatcher.dispatchTransactionCommittedEvent(partition, offsetContext);
+ offsetContext.commitTransaction();
+ offsetContext.setBinlogThread(-1L);
+ skipEvent = false;
+ ignoreDmlEventByGtidSource = false;
+ }
+
+ /**
+ * Handle a change in the table metadata.
+ *
+ * This method should be called whenever we consume a TABLE_MAP event, and every transaction
+ * in the log should include one of these for each table affected by the transaction. Each table
+ * map event includes a monotonically-increasing numeric identifier, and this identifier is used
+ * within subsequent events within the same transaction. This table identifier can change when:
+ *
+ *
+ * - the table structure is modified (e.g., via an {@code ALTER TABLE ...} command); or
+ *
- MySQL rotates to a new binary log file, even if the table structure does not change.
+ *
+ *
+ * @param event the update event; never null
+ */
+ protected void handleUpdateTableMetadata(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ TableMapEventData metadata = unwrapData(event);
+ long tableNumber = metadata.getTableId();
+ String databaseName = metadata.getDatabase();
+ String tableName = metadata.getTable();
+ TableId tableId = new TableId(databaseName, null, tableName);
+ if (taskContext.getSchema().assignTableNumber(tableNumber, tableId)) {
+ LOGGER.debug("Received update table metadata event: {}", event);
+ } else {
+ informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId);
+ }
+ }
+
+ /**
+ * If we receive an event for a table that is monitored but whose metadata we don't know, either
+ * ignore that event or raise a warning or error as per the {@link
+ * MySqlConnectorConfig#INCONSISTENT_SCHEMA_HANDLING_MODE} configuration.
+ */
+ private void informAboutUnknownTableIfRequired(
+ MySqlPartition partition,
+ MySqlOffsetContext offsetContext,
+ Event event,
+ TableId tableId,
+ Operation operation)
+ throws InterruptedException {
+ if (tableId != null
+ && connectorConfig.getTableFilters().dataCollectionFilter().isIncluded(tableId)) {
+ metrics.onErroneousEvent(
+ partition, "source = " + tableId + ", event " + event, operation);
+ EventHeaderV4 eventHeader = event.getHeader();
+
+ if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.FAIL) {
+ LOGGER.error(
+ "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}"
+ + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}",
+ event,
+ offsetContext.getOffset(),
+ tableId,
+ System.lineSeparator(),
+ eventHeader.getPosition(),
+ eventHeader.getNextPosition(),
+ offsetContext.getSource().binlogFilename());
+ throw new DebeziumException(
+ "Encountered change event for table "
+ + tableId
+ + " whose schema isn't known to this connector");
+ } else if (inconsistentSchemaHandlingMode == EventProcessingFailureHandlingMode.WARN) {
+ LOGGER.warn(
+ "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}"
+ + "The event will be ignored.{}"
+ + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}",
+ event,
+ offsetContext.getOffset(),
+ tableId,
+ System.lineSeparator(),
+ System.lineSeparator(),
+ eventHeader.getPosition(),
+ eventHeader.getNextPosition(),
+ offsetContext.getSource().binlogFilename());
+ } else {
+ LOGGER.debug(
+ "Encountered change event '{}' at offset {} for table {} whose schema isn't known to this connector. One possible cause is an incomplete database history topic. Take a new snapshot in this case.{}"
+ + "The event will be ignored.{}"
+ + "Use the mysqlbinlog tool to view the problematic event: mysqlbinlog --start-position={} --stop-position={} --verbose {}",
+ event,
+ offsetContext.getOffset(),
+ tableId,
+ System.lineSeparator(),
+ System.lineSeparator(),
+ eventHeader.getPosition(),
+ eventHeader.getNextPosition(),
+ offsetContext.getSource().binlogFilename());
+ }
+ } else {
+ if (tableId == null) {
+ EventData eventData = unwrapData(event);
+ if (eventData instanceof WriteRowsEventData) {
+ tableId =
+ taskContext
+ .getSchema()
+ .getExcludeTableId(
+ ((WriteRowsEventData) eventData).getTableId());
+ } else if (eventData instanceof UpdateRowsEventData) {
+ tableId =
+ taskContext
+ .getSchema()
+ .getExcludeTableId(
+ ((UpdateRowsEventData) eventData).getTableId());
+ } else if (eventData instanceof DeleteRowsEventData) {
+ tableId =
+ taskContext
+ .getSchema()
+ .getExcludeTableId(
+ ((DeleteRowsEventData) eventData).getTableId());
+ }
+ }
+ LOGGER.trace("Filtered {} event for {}", event.getHeader().getEventType(), tableId);
+ metrics.onFilteredEvent(partition, "source = " + tableId, operation);
+ eventDispatcher.dispatchFilteredEvent(partition, offsetContext);
+ }
+ }
+
+ private void informAboutUnknownTableIfRequired(
+ MySqlPartition partition,
+ MySqlOffsetContext offsetContext,
+ Event event,
+ TableId tableId)
+ throws InterruptedException {
+ informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, null);
+ }
+
+ /**
+ * Generate source records for the supplied event with an {@link WriteRowsEventData}.
+ *
+ * @param partition the partition in which the even occurred
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while blocking
+ */
+ protected void handleInsert(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ handleChange(
+ partition,
+ offsetContext,
+ event,
+ Operation.CREATE,
+ WriteRowsEventData.class,
+ x -> taskContext.getSchema().getTableId(x.getTableId()),
+ WriteRowsEventData::getRows,
+ (tableId, row) ->
+ eventDispatcher.dispatchDataChangeEvent(
+ partition,
+ tableId,
+ new MySqlChangeRecordEmitter(
+ partition,
+ offsetContext,
+ clock,
+ Operation.CREATE,
+ null,
+ row)));
+ }
+
+ /**
+ * Generate source records for the supplied event with an {@link UpdateRowsEventData}.
+ *
+ * @param partition the partition in which the even occurred
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while blocking
+ */
+ protected void handleUpdate(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ handleChange(
+ partition,
+ offsetContext,
+ event,
+ Operation.UPDATE,
+ UpdateRowsEventData.class,
+ x -> taskContext.getSchema().getTableId(x.getTableId()),
+ UpdateRowsEventData::getRows,
+ (tableId, row) ->
+ eventDispatcher.dispatchDataChangeEvent(
+ partition,
+ tableId,
+ new MySqlChangeRecordEmitter(
+ partition,
+ offsetContext,
+ clock,
+ Operation.UPDATE,
+ row.getKey(),
+ row.getValue())));
+ }
+
+ /**
+ * Generate source records for the supplied event with an {@link DeleteRowsEventData}.
+ *
+ * @param partition the partition in which the even occurred
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while blocking
+ */
+ protected void handleDelete(
+ MySqlPartition partition, MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ handleChange(
+ partition,
+ offsetContext,
+ event,
+ Operation.DELETE,
+ DeleteRowsEventData.class,
+ x -> taskContext.getSchema().getTableId(x.getTableId()),
+ DeleteRowsEventData::getRows,
+ (tableId, row) ->
+ eventDispatcher.dispatchDataChangeEvent(
+ partition,
+ tableId,
+ new MySqlChangeRecordEmitter(
+ partition,
+ offsetContext,
+ clock,
+ Operation.DELETE,
+ row,
+ null)));
+ }
+
+ private void handleChange(
+ MySqlPartition partition,
+ MySqlOffsetContext offsetContext,
+ Event event,
+ Operation operation,
+ Class eventDataClass,
+ TableIdProvider tableIdProvider,
+ RowsProvider rowsProvider,
+ BinlogChangeEmitter changeEmitter)
+ throws InterruptedException {
+ if (skipEvent) {
+ // We can skip this because we should already be at least this far ...
+ LOGGER.info("Skipping previously processed row event: {}", event);
+ return;
+ }
+ if (ignoreDmlEventByGtidSource) {
+ LOGGER.debug("Skipping DML event because this GTID source is filtered: {}", event);
+ return;
+ }
+ final T data = unwrapData(event);
+ final TableId tableId = tableIdProvider.getTableId(data);
+ final List rows = rowsProvider.getRows(data);
+ String changeType = operation.name();
+
+ if (tableId != null && taskContext.getSchema().schemaFor(tableId) != null) {
+ int count = 0;
+ int numRows = rows.size();
+ if (startingRowNumber < numRows) {
+ for (int row = startingRowNumber; row != numRows; ++row) {
+ offsetContext.setRowNumber(row, numRows);
+ offsetContext.event(tableId, eventTimestamp);
+ changeEmitter.emit(tableId, rows.get(row));
+ count++;
+ }
+ if (LOGGER.isDebugEnabled()) {
+ if (startingRowNumber != 0) {
+ LOGGER.debug(
+ "Emitted {} {} record(s) for last {} row(s) in event: {}",
+ count,
+ changeType,
+ numRows - startingRowNumber,
+ event);
+ } else {
+ LOGGER.debug(
+ "Emitted {} {} record(s) for event: {}", count, changeType, event);
+ }
+ }
+ offsetContext.changeEventCompleted();
+ } else {
+ // All rows were previously processed ...
+ LOGGER.debug("Skipping previously processed {} event: {}", changeType, event);
+ }
+ } else {
+ informAboutUnknownTableIfRequired(partition, offsetContext, event, tableId, operation);
+ }
+ startingRowNumber = 0;
+ }
+
+ /**
+ * Handle a {@link EventType#VIEW_CHANGE} event.
+ *
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while blocking
+ */
+ protected void viewChange(MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ LOGGER.debug("View Change event: {}", event);
+ // do nothing
+ }
+
+ /**
+ * Handle a {@link EventType#XA_PREPARE} event.
+ *
+ * @param event the database change data event to be processed; may not be null
+ * @throws InterruptedException if this thread is interrupted while blocking
+ */
+ protected void prepareTransaction(MySqlOffsetContext offsetContext, Event event)
+ throws InterruptedException {
+ LOGGER.debug("XA Prepare event: {}", event);
+ // do nothing
+ }
+
+ 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;
+ }
+
+ @Override
+ public void execute(
+ ChangeEventSourceContext context,
+ MySqlPartition partition,
+ MySqlOffsetContext offsetContext)
+ throws InterruptedException {
+ if (!connectorConfig.getSnapshotMode().shouldStream()) {
+ LOGGER.info(
+ "Streaming is disabled for snapshot mode {}",
+ connectorConfig.getSnapshotMode());
+ return;
+ }
+ if (connectorConfig.getSnapshotMode() != MySqlConnectorConfig.SnapshotMode.NEVER) {
+ taskContext.getSchema().assureNonEmptySchema();
+ }
+ final Set skippedOperations = connectorConfig.getSkippedOperations();
+
+ final MySqlOffsetContext effectiveOffsetContext =
+ offsetContext != null ? offsetContext : MySqlOffsetContext.initial(connectorConfig);
+
+ // Register our event handlers ...
+ eventHandlers.put(
+ EventType.STOP, (event) -> handleServerStop(effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.HEARTBEAT,
+ (event) -> handleServerHeartbeat(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.INCIDENT,
+ (event) -> handleServerIncident(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.ROTATE, (event) -> handleRotateLogsEvent(effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.TABLE_MAP,
+ (event) -> handleUpdateTableMetadata(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.QUERY,
+ (event) -> handleQueryEvent(partition, effectiveOffsetContext, event));
+
+ if (!skippedOperations.contains(Operation.CREATE)) {
+ eventHandlers.put(
+ EventType.WRITE_ROWS,
+ (event) -> handleInsert(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.EXT_WRITE_ROWS,
+ (event) -> handleInsert(partition, effectiveOffsetContext, event));
+ }
+
+ if (!skippedOperations.contains(Operation.UPDATE)) {
+ eventHandlers.put(
+ EventType.UPDATE_ROWS,
+ (event) -> handleUpdate(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.EXT_UPDATE_ROWS,
+ (event) -> handleUpdate(partition, effectiveOffsetContext, event));
+ }
+
+ if (!skippedOperations.contains(Operation.DELETE)) {
+ eventHandlers.put(
+ EventType.DELETE_ROWS,
+ (event) -> handleDelete(partition, effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.EXT_DELETE_ROWS,
+ (event) -> handleDelete(partition, effectiveOffsetContext, event));
+ }
+
+ eventHandlers.put(
+ EventType.VIEW_CHANGE, (event) -> viewChange(effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.XA_PREPARE, (event) -> prepareTransaction(effectiveOffsetContext, event));
+ eventHandlers.put(
+ EventType.XID,
+ (event) -> handleTransactionCompletion(partition, effectiveOffsetContext, event));
+
+ // Conditionally register ROWS_QUERY handler to parse SQL statements.
+ if (connectorConfig.includeSqlQuery()) {
+ eventHandlers.put(
+ EventType.ROWS_QUERY,
+ (event) -> handleRowsQuery(effectiveOffsetContext, event));
+ }
+
+ BinaryLogClient.EventListener listener;
+ if (connectorConfig.bufferSizeForStreamingChangeEventSource() == 0) {
+ listener = (event) -> handleEvent(partition, effectiveOffsetContext, event);
+ } else {
+ EventBuffer buffer =
+ new EventBuffer(
+ connectorConfig.bufferSizeForStreamingChangeEventSource(),
+ this,
+ context);
+ listener = (event) -> buffer.add(partition, effectiveOffsetContext, event);
+ }
+ client.registerEventListener(listener);
+
+ client.registerLifecycleListener(new ReaderThreadLifecycleListener(effectiveOffsetContext));
+ client.registerEventListener((event) -> onEvent(effectiveOffsetContext, event));
+ if (LOGGER.isDebugEnabled()) {
+ client.registerEventListener((event) -> logEvent(effectiveOffsetContext, event));
+ }
+
+ final boolean isGtidModeEnabled = connection.isGtidModeEnabled();
+ 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);
+
+ // 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);
+ 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);
+ String filteredGtidSetStr = filteredGtidSet.toString();
+ client.setGtidSet(filteredGtidSetStr);
+ effectiveOffsetContext.setCompletedGtidSet(filteredGtidSetStr);
+ gtidSet = new com.github.shyiko.mysql.binlog.GtidSet(filteredGtidSetStr);
+ } else {
+ // We've not yet seen any GTIDs, so that means we have to start reading the binlog
+ // from the beginning ...
+ client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename());
+ client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition());
+ gtidSet = new com.github.shyiko.mysql.binlog.GtidSet("");
+ }
+ } else {
+ // The server is not using GTIDs, so start reading the binlog based upon where we last
+ // left off ...
+ client.setBinlogFilename(effectiveOffsetContext.getSource().binlogFilename());
+ client.setBinlogPosition(effectiveOffsetContext.getSource().binlogPosition());
+ }
+
+ // We may be restarting in the middle of a transaction, so see how far into the transaction
+ // we have already processed...
+ initialEventsToSkip = effectiveOffsetContext.eventsToSkipUponRestart();
+ LOGGER.info("Skip {} events on streaming start", initialEventsToSkip);
+
+ // Set the starting row number, which is the next row number to be read ...
+ startingRowNumber = effectiveOffsetContext.rowsToSkipUponRestart();
+ LOGGER.info("Skip {} rows on streaming start", startingRowNumber);
+
+ // Only when we reach the first BEGIN event will we start to skip events ...
+ skipEvent = false;
+
+ try {
+ // Start the log reader, which starts background threads ...
+ if (context.isRunning()) {
+ long timeout = connectorConfig.getConnectionTimeout().toMillis();
+ long started = clock.currentTimeInMillis();
+ try {
+ LOGGER.debug(
+ "Attempting to establish binlog reader connection with timeout of {} ms",
+ timeout);
+ client.connect(timeout);
+ // Need to wait for keepalive thread to be running, otherwise it can be left
+ // orphaned
+ // The problem is with timing. When the close is called too early after connect
+ // then
+ // the keepalive thread is not terminated
+ if (client.isKeepAlive()) {
+ LOGGER.info("Waiting for keepalive thread to start");
+ final Metronome metronome = Metronome.parker(Duration.ofMillis(100), clock);
+ int waitAttempts = 50;
+ boolean keepAliveThreadRunning = false;
+ while (!keepAliveThreadRunning && waitAttempts-- > 0) {
+ for (Thread t : binaryLogClientThreads.values()) {
+ if (t.getName().startsWith(KEEPALIVE_THREAD_NAME) && t.isAlive()) {
+ LOGGER.info("Keepalive thread is running");
+ keepAliveThreadRunning = true;
+ }
+ }
+ metronome.pause();
+ }
+ }
+ } catch (TimeoutException e) {
+ // If the client thread is interrupted *before* the client could connect, the
+ // client throws a timeout exception
+ // The only way we can distinguish this is if we get the timeout exception
+ // before the specified timeout has
+ // elapsed, so we simply check this (within 10%) ...
+ long duration = clock.currentTimeInMillis() - started;
+ if (duration > (0.9 * timeout)) {
+ double actualSeconds = TimeUnit.MILLISECONDS.toSeconds(duration);
+ throw new DebeziumException(
+ "Timed out after "
+ + actualSeconds
+ + " seconds while waiting to connect to MySQL at "
+ + connectorConfig.hostname()
+ + ":"
+ + connectorConfig.port()
+ + " with user '"
+ + connectorConfig.username()
+ + "'",
+ e);
+ }
+ // Otherwise, we were told to shutdown, so we don't care about the timeout
+ // exception
+ } catch (AuthenticationException e) {
+ throw new DebeziumException(
+ "Failed to authenticate to the MySQL database at "
+ + connectorConfig.hostname()
+ + ":"
+ + connectorConfig.port()
+ + " with user '"
+ + connectorConfig.username()
+ + "'",
+ e);
+ } catch (Throwable e) {
+ throw new DebeziumException(
+ "Unable to connect to the MySQL database at "
+ + connectorConfig.hostname()
+ + ":"
+ + connectorConfig.port()
+ + " with user '"
+ + connectorConfig.username()
+ + "': "
+ + e.getMessage(),
+ e);
+ }
+ }
+ while (context.isRunning()) {
+ Thread.sleep(100);
+ }
+ } finally {
+ try {
+ client.disconnect();
+ } catch (Exception e) {
+ LOGGER.info("Exception while stopping binary log client", e);
+ }
+ }
+ }
+
+ 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);
+ }
+
+ protected void logEvent(MySqlOffsetContext offsetContext, Event event) {
+ LOGGER.trace("Received event: {}", event);
+ }
+
+ private void logStreamingSourceState(Level severity) {
+ final Object position =
+ client == null
+ ? "N/A"
+ : client.getBinlogFilename() + "/" + client.getBinlogPosition();
+ final String message =
+ "Error during binlog processing. Last offset stored = {}, binlog reader near position = {}";
+ switch (severity) {
+ case WARN:
+ LOGGER.warn(message, lastOffset, position);
+ break;
+ case DEBUG:
+ LOGGER.debug(message, lastOffset, position);
+ break;
+ default:
+ LOGGER.error(message, lastOffset, position);
+ }
+ }
+
+ /**
+ * Apply the include/exclude GTID source filters to the current {@link #source() 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);
+
+ GtidSet mergedGtidSet;
+
+ if (connectorConfig.gtidNewChannelPosition() == GtidNewChannelPosition.EARLIEST) {
+ 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);
+
+ // Since the GTID recorded in the checkpoint represents the CDC-executed records, in
+ // certain scenarios
+ // (such as when the startup mode is earliest/timestamp/binlogfile), the recorded GTID
+ // may not start from
+ // the beginning. For example, A:300-500. However, during job recovery, we usually only
+ // need to focus on
+ // the last consumed point instead of consuming A:1-299. Therefore, some adjustments
+ // need to be made to the
+ // recorded offset in the checkpoint, and the available GTID for other MySQL instances
+ // should be completed.
+ mergedGtidSet =
+ GtidUtils.fixRestoredGtidSet(
+ GtidUtils.mergeGtidSetInto(
+ relevantAvailableServerGtidSet.retainAll(
+ uuid -> knownGtidSet.forServerWithId(uuid) != null),
+ purgedServerGtid),
+ filteredGtidSet);
+ } else {
+ mergedGtidSet = availableServerGtidSet.with(filteredGtidSet);
+ }
+
+ LOGGER.info("Final merged GTID set to use when connecting to MySQL: {}", mergedGtidSet);
+ return mergedGtidSet;
+ }
+
+ MySqlStreamingChangeEventSourceMetrics getMetrics() {
+ return metrics;
+ }
+
+ void rewindBinaryLogClient(ChangeEventSourceContext context, BinlogPosition position) {
+ try {
+ if (context.isRunning()) {
+ LOGGER.debug("Rewinding binlog to position {}", position);
+ client.disconnect();
+ client.setBinlogFilename(position.getFilename());
+ client.setBinlogPosition(position.getPosition());
+ client.connect();
+ }
+ } catch (IOException e) {
+ LOGGER.error("Unexpected error when re-connecting to the MySQL binary log reader", e);
+ }
+ }
+
+ BinlogPosition getCurrentBinlogPosition() {
+ return new BinlogPosition(client.getBinlogFilename(), client.getBinlogPosition());
+ }
+
+ /**
+ * Wraps the specified exception in a {@link DebeziumException}, ensuring that all useful state
+ * is captured inside the new exception's message.
+ *
+ * @param error the exception; may not be null
+ * @return the wrapped Kafka Connect exception
+ */
+ protected DebeziumException wrap(Throwable error) {
+ assert error != null;
+ String msg = error.getMessage();
+ if (error instanceof ServerException) {
+ ServerException e = (ServerException) error;
+ msg = msg + " Error code: " + e.getErrorCode() + "; SQLSTATE: " + e.getSqlState() + ".";
+ } else if (error instanceof SQLException) {
+ SQLException e = (SQLException) error;
+ msg =
+ e.getMessage()
+ + " Error code: "
+ + e.getErrorCode()
+ + "; SQLSTATE: "
+ + e.getSQLState()
+ + ".";
+ }
+ msg = ErrorMessageUtils.optimizeErrorMessage(msg);
+ return new DebeziumException(msg, error);
+ }
+
+ /** LifecycleListener for Reader Thread. */
+ protected final class ReaderThreadLifecycleListener implements LifecycleListener {
+ private final MySqlOffsetContext offsetContext;
+
+ ReaderThreadLifecycleListener(MySqlOffsetContext offsetContext) {
+ this.offsetContext = offsetContext;
+ }
+
+ @Override
+ public void onDisconnect(BinaryLogClient client) {
+ if (LOGGER.isInfoEnabled()) {
+ taskContext.temporaryLoggingContext(
+ connectorConfig,
+ "binlog",
+ () -> {
+ Map offset = lastOffset;
+ if (offset != null) {
+ LOGGER.info(
+ "Stopped reading binlog after {} events, last recorded offset: {}",
+ totalRecordCounter,
+ offset);
+ } else {
+ LOGGER.info(
+ "Stopped reading binlog after {} events, no new offset was recorded",
+ totalRecordCounter);
+ }
+ });
+ }
+ }
+
+ @Override
+ public void onConnect(BinaryLogClient client) {
+ // Set up the MDC logging context for this thread ...
+ taskContext.configureLoggingContext("binlog");
+
+ // The event row number will be used when processing the first event ...
+ LOGGER.info(
+ "Connected to MySQL binlog at {}:{}, starting at {}",
+ connectorConfig.hostname(),
+ connectorConfig.port(),
+ offsetContext);
+ }
+
+ @Override
+ public void onCommunicationFailure(BinaryLogClient client, Exception ex) {
+ LOGGER.debug("A communication failure event arrived", ex);
+ logStreamingSourceState();
+ try {
+ // Stop BinaryLogClient background threads
+ client.disconnect();
+ } catch (final Exception e) {
+ LOGGER.debug("Exception while closing client", e);
+ }
+ errorHandler.setProducerThrowable(wrap(ex));
+ }
+
+ @Override
+ public void onEventDeserializationFailure(BinaryLogClient client, Exception ex) {
+ if (eventDeserializationFailureHandlingMode
+ == EventProcessingFailureHandlingMode.FAIL) {
+ LOGGER.debug("A deserialization failure event arrived", ex);
+ logStreamingSourceState();
+ errorHandler.setProducerThrowable(wrap(ex));
+ } else if (eventDeserializationFailureHandlingMode
+ == EventProcessingFailureHandlingMode.WARN) {
+ LOGGER.warn("A deserialization failure event arrived", ex);
+ logStreamingSourceState(Level.WARN);
+ } else {
+ LOGGER.debug("A deserialization failure event arrived", ex);
+ logStreamingSourceState(Level.DEBUG);
+ }
+ }
+ }
+
+ @FunctionalInterface
+ private interface TableIdProvider {
+ TableId getTableId(E data);
+ }
+
+ @FunctionalInterface
+ private interface RowsProvider {
+ List getRows(E data);
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java
deleted file mode 100644
index 195d505e6af..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSource.java
+++ /dev/null
@@ -1,313 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase;
-
-import org.apache.flink.cdc.common.annotation.PublicEvolving;
-import org.apache.flink.cdc.connectors.base.options.StartupOptions;
-import org.apache.flink.cdc.connectors.oceanbase.source.OceanBaseRichSourceFunction;
-import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
-import org.apache.flink.streaming.api.functions.source.SourceFunction;
-
-import com.oceanbase.clogproxy.client.config.ClientConf;
-import com.oceanbase.clogproxy.client.config.ObReaderConfig;
-import org.apache.commons.lang3.StringUtils;
-
-import java.time.Duration;
-import java.time.Instant;
-import java.time.ZoneId;
-import java.time.format.DateTimeFormatter;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** A builder to build a SourceFunction which can read snapshot and change events of OceanBase. */
-@PublicEvolving
-public class OceanBaseSource {
-
- public static Builder builder() {
- return new Builder<>();
- }
-
- /** Builder class of {@link OceanBaseSource}. */
- public static class Builder {
-
- // common config
- private StartupOptions startupOptions;
- private String username;
- private String password;
- private String tenantName;
- private String databaseName;
- private String tableName;
- private String tableList;
- private String serverTimeZone;
- private Duration connectTimeout;
-
- // snapshot reading config
- private String hostname;
- private Integer port;
- private String compatibleMode;
- private String jdbcDriver;
- private Properties jdbcProperties;
-
- // incremental reading config
- private String logProxyHost;
- private Integer logProxyPort;
- private String logProxyClientId;
- private Long startupTimestamp;
- private String rsList;
- private String configUrl;
- private String workingMode;
- private Properties obcdcProperties;
- private Properties debeziumProperties;
-
- private DebeziumDeserializationSchema deserializer;
-
- public Builder startupOptions(StartupOptions startupOptions) {
- this.startupOptions = startupOptions;
- return this;
- }
-
- public Builder username(String username) {
- this.username = username;
- return this;
- }
-
- public Builder password(String password) {
- this.password = password;
- return this;
- }
-
- public Builder tenantName(String tenantName) {
- this.tenantName = tenantName;
- return this;
- }
-
- public Builder databaseName(String databaseName) {
- this.databaseName = databaseName;
- return this;
- }
-
- public Builder tableName(String tableName) {
- this.tableName = tableName;
- return this;
- }
-
- public Builder tableList(String tableList) {
- this.tableList = tableList;
- return this;
- }
-
- public Builder serverTimeZone(String serverTimeZone) {
- this.serverTimeZone = serverTimeZone;
- return this;
- }
-
- public Builder connectTimeout(Duration connectTimeout) {
- this.connectTimeout = connectTimeout;
- return this;
- }
-
- public Builder hostname(String hostname) {
- this.hostname = hostname;
- return this;
- }
-
- public Builder port(int port) {
- this.port = port;
- return this;
- }
-
- public Builder compatibleMode(String compatibleMode) {
- this.compatibleMode = compatibleMode;
- return this;
- }
-
- public Builder jdbcDriver(String jdbcDriver) {
- this.jdbcDriver = jdbcDriver;
- return this;
- }
-
- public Builder jdbcProperties(Properties jdbcProperties) {
- this.jdbcProperties = jdbcProperties;
- return this;
- }
-
- public Builder logProxyHost(String logProxyHost) {
- this.logProxyHost = logProxyHost;
- return this;
- }
-
- public Builder logProxyPort(Integer logProxyPort) {
- this.logProxyPort = logProxyPort;
- return this;
- }
-
- public Builder logProxyClientId(String logProxyClientId) {
- this.logProxyClientId = logProxyClientId;
- return this;
- }
-
- public Builder startupTimestamp(Long startupTimestamp) {
- this.startupTimestamp = startupTimestamp;
- return this;
- }
-
- public Builder rsList(String rsList) {
- this.rsList = rsList;
- return this;
- }
-
- public Builder configUrl(String configUrl) {
- this.configUrl = configUrl;
- return this;
- }
-
- public Builder workingMode(String workingMode) {
- this.workingMode = workingMode;
- return this;
- }
-
- public Builder obcdcProperties(Properties obcdcProperties) {
- this.obcdcProperties = obcdcProperties;
- return this;
- }
-
- public Builder debeziumProperties(Properties debeziumProperties) {
- this.debeziumProperties = debeziumProperties;
- return this;
- }
-
- public Builder deserializer(DebeziumDeserializationSchema deserializer) {
- this.deserializer = deserializer;
- return this;
- }
-
- public SourceFunction build() {
- checkNotNull(username, "username shouldn't be null");
- checkNotNull(password, "password shouldn't be null");
- checkNotNull(hostname, "hostname shouldn't be null");
- checkNotNull(port, "port shouldn't be null");
-
- if (startupOptions == null) {
- startupOptions = StartupOptions.initial();
- }
- if (compatibleMode == null) {
- compatibleMode = "mysql";
- }
- if (jdbcDriver == null) {
- jdbcDriver = "com.mysql.cj.jdbc.Driver";
- }
-
- if (connectTimeout == null) {
- connectTimeout = Duration.ofSeconds(30);
- }
-
- if (serverTimeZone == null) {
- serverTimeZone = ZoneId.systemDefault().getId();
- }
-
- switch (startupOptions.startupMode) {
- case SNAPSHOT:
- break;
- case INITIAL:
- case LATEST_OFFSET:
- startupTimestamp = 0L;
- break;
- case TIMESTAMP:
- checkNotNull(
- startupTimestamp,
- "startupTimestamp shouldn't be null on startup mode 'timestamp'");
- break;
- default:
- throw new UnsupportedOperationException(
- startupOptions.startupMode + " mode is not supported.");
- }
-
- if (StringUtils.isNotEmpty(databaseName) || StringUtils.isNotEmpty(tableName)) {
- if (StringUtils.isEmpty(databaseName) || StringUtils.isEmpty(tableName)) {
- throw new IllegalArgumentException(
- "'database-name' and 'table-name' should be configured at the same time");
- }
- } else {
- checkNotNull(
- tableList,
- "'database-name', 'table-name' or 'table-list' should be configured");
- }
-
- ClientConf clientConf = null;
- ObReaderConfig obReaderConfig = null;
-
- if (!startupOptions.isSnapshotOnly()) {
-
- checkNotNull(logProxyHost);
- checkNotNull(logProxyPort);
- checkNotNull(tenantName);
-
- obReaderConfig = new ObReaderConfig();
- if (StringUtils.isNotEmpty(rsList)) {
- obReaderConfig.setRsList(rsList);
- }
- if (StringUtils.isNotEmpty(configUrl)) {
- obReaderConfig.setClusterUrl(configUrl);
- }
- if (StringUtils.isNotEmpty(workingMode)) {
- obReaderConfig.setWorkingMode(workingMode);
- }
- obReaderConfig.setUsername(username);
- obReaderConfig.setPassword(password);
- obReaderConfig.setStartTimestamp(startupTimestamp);
- obReaderConfig.setTimezone(
- DateTimeFormatter.ofPattern("xxx")
- .format(
- ZoneId.of(serverTimeZone)
- .getRules()
- .getOffset(Instant.now())));
-
- if (obcdcProperties != null && !obcdcProperties.isEmpty()) {
- Map extraConfigs = new HashMap<>();
- obcdcProperties.forEach((k, v) -> extraConfigs.put(k.toString(), v.toString()));
- obReaderConfig.setExtraConfigs(extraConfigs);
- }
- }
-
- return new OceanBaseRichSourceFunction<>(
- startupOptions,
- username,
- password,
- tenantName,
- databaseName,
- tableName,
- tableList,
- serverTimeZone,
- connectTimeout,
- hostname,
- port,
- compatibleMode,
- jdbcDriver,
- jdbcProperties,
- logProxyHost,
- logProxyPort,
- logProxyClientId,
- obReaderConfig,
- debeziumProperties,
- deserializer);
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java
deleted file mode 100644
index 84508fba6d6..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/OceanBaseRichSourceFunction.java
+++ /dev/null
@@ -1,598 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source;
-
-import org.apache.flink.api.common.state.CheckpointListener;
-import org.apache.flink.api.common.state.ListState;
-import org.apache.flink.api.common.state.ListStateDescriptor;
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.api.common.typeutils.base.LongSerializer;
-import org.apache.flink.api.java.typeutils.ResultTypeQueryable;
-import org.apache.flink.cdc.connectors.base.options.StartupOptions;
-import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig;
-import org.apache.flink.cdc.connectors.oceanbase.source.connection.OceanBaseConnection;
-import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfo;
-import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseDatabaseSchema;
-import org.apache.flink.cdc.connectors.oceanbase.source.schema.OceanBaseSchema;
-import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.runtime.state.FunctionInitializationContext;
-import org.apache.flink.runtime.state.FunctionSnapshotContext;
-import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction;
-import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
-import org.apache.flink.util.Collector;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import com.oceanbase.clogproxy.client.LogProxyClient;
-import com.oceanbase.clogproxy.client.config.ClientConf;
-import com.oceanbase.clogproxy.client.config.ObReaderConfig;
-import com.oceanbase.clogproxy.client.exception.LogProxyClientException;
-import com.oceanbase.clogproxy.client.listener.RecordListener;
-import com.oceanbase.clogproxy.client.util.ClientUtil;
-import com.oceanbase.oms.logmessage.DataMessage;
-import com.oceanbase.oms.logmessage.LogMessage;
-import io.debezium.connector.SnapshotRecord;
-import io.debezium.relational.TableId;
-import io.debezium.relational.TableSchema;
-import io.debezium.relational.history.TableChanges;
-import org.apache.commons.lang3.StringUtils;
-import org.apache.kafka.connect.data.Field;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.data.Struct;
-import org.apache.kafka.connect.source.SourceRecord;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.SQLException;
-import java.time.Duration;
-import java.time.Instant;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/**
- * The source implementation for OceanBase that read snapshot events first and then read the change
- * event.
- *
- * @param The type created by the deserializer.
- */
-public class OceanBaseRichSourceFunction extends RichSourceFunction
- implements CheckpointListener, CheckpointedFunction, ResultTypeQueryable {
-
- private static final long serialVersionUID = 2844054619864617340L;
-
- private static final Logger LOG = LoggerFactory.getLogger(OceanBaseRichSourceFunction.class);
-
- private final StartupOptions startupOptions;
- private final String username;
- private final String password;
- private final String tenantName;
- private final String databaseName;
- private final String tableName;
- private final String tableList;
- private final String serverTimeZone;
- private final Duration connectTimeout;
- private final String hostname;
- private final Integer port;
- private final String compatibleMode;
- private final String jdbcDriver;
- private final Properties jdbcProperties;
- private final String logProxyHost;
- private final Integer logProxyPort;
- private final String logProxyClientId;
- private final ObReaderConfig obReaderConfig;
- private final Properties debeziumProperties;
- private final DebeziumDeserializationSchema deserializer;
-
- private final List changeRecordBuffer = new LinkedList<>();
-
- private transient OceanBaseConnectorConfig connectorConfig;
- private transient OceanBaseSourceInfo sourceInfo;
- private transient Set tableSet;
- private transient OceanBaseSchema obSchema;
- private transient OceanBaseDatabaseSchema databaseSchema;
- private transient volatile long resolvedTimestamp;
- private transient volatile Exception logProxyClientException;
- private transient volatile OceanBaseConnection snapshotConnection;
- private transient LogProxyClient logProxyClient;
- private transient ListState offsetState;
- private transient OutputCollector outputCollector;
-
- public OceanBaseRichSourceFunction(
- StartupOptions startupOptions,
- String username,
- String password,
- String tenantName,
- String databaseName,
- String tableName,
- String tableList,
- String serverTimeZone,
- Duration connectTimeout,
- String hostname,
- Integer port,
- String compatibleMode,
- String jdbcDriver,
- Properties jdbcProperties,
- String logProxyHost,
- Integer logProxyPort,
- String logProxyClientId,
- ObReaderConfig obReaderConfig,
- Properties debeziumProperties,
- DebeziumDeserializationSchema deserializer) {
- this.startupOptions = checkNotNull(startupOptions);
- this.username = checkNotNull(username);
- this.password = checkNotNull(password);
- this.tenantName = tenantName;
- this.databaseName = databaseName;
- this.tableName = tableName;
- this.tableList = tableList;
- this.serverTimeZone = checkNotNull(serverTimeZone);
- this.connectTimeout = checkNotNull(connectTimeout);
- this.hostname = checkNotNull(hostname);
- this.port = checkNotNull(port);
- this.compatibleMode = checkNotNull(compatibleMode);
- this.jdbcDriver = checkNotNull(jdbcDriver);
- this.jdbcProperties = jdbcProperties;
- this.logProxyHost = logProxyHost;
- this.logProxyPort = logProxyPort;
- this.logProxyClientId = logProxyClientId;
- this.obReaderConfig = obReaderConfig;
- this.debeziumProperties = debeziumProperties;
- this.deserializer = checkNotNull(deserializer);
- }
-
- @Override
- public void open(final Configuration config) throws Exception {
- super.open(config);
- this.outputCollector = new OutputCollector<>();
- this.connectorConfig =
- new OceanBaseConnectorConfig(compatibleMode, serverTimeZone, debeziumProperties);
- this.sourceInfo = new OceanBaseSourceInfo(connectorConfig, tenantName);
- }
-
- @Override
- public void run(SourceContext ctx) throws Exception {
- outputCollector.context = ctx;
- try {
- LOG.info("Start to initial table whitelist");
- initTableWhiteList();
-
- if (resolvedTimestamp <= 0 && !startupOptions.isStreamOnly()) {
- sourceInfo.setSnapshot(SnapshotRecord.TRUE);
- long startTimestamp = getSnapshotConnection().getCurrentTimestampS();
- LOG.info("Snapshot reading started from timestamp: {}", startTimestamp);
- readSnapshotRecords();
- sourceInfo.setSnapshot(SnapshotRecord.FALSE);
- LOG.info("Snapshot reading finished");
- resolvedTimestamp = startTimestamp;
- } else {
- LOG.info("Snapshot reading skipped");
- }
-
- if (!startupOptions.isSnapshotOnly()) {
- sourceInfo.setSnapshot(SnapshotRecord.FALSE);
- LOG.info("Change events reading started");
- readChangeRecords();
- }
- } finally {
- cancel();
- }
- }
-
- private OceanBaseConnection getSnapshotConnection() {
- if (snapshotConnection == null) {
- snapshotConnection =
- new OceanBaseConnection(
- hostname,
- port,
- username,
- password,
- connectTimeout,
- compatibleMode,
- jdbcDriver,
- jdbcProperties,
- getClass().getClassLoader());
- }
- return snapshotConnection;
- }
-
- private void closeSnapshotConnection() {
- if (snapshotConnection != null) {
- try {
- snapshotConnection.close();
- } catch (SQLException e) {
- LOG.error("Failed to close snapshotConnection", e);
- }
- snapshotConnection = null;
- }
- }
-
- private TableId tableId(String databaseName, String tableName) {
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- return new TableId(databaseName, null, tableName);
- }
- return new TableId(null, databaseName, tableName);
- }
-
- private void initTableWhiteList() {
- if (tableSet != null && !tableSet.isEmpty()) {
- return;
- }
-
- final Set localTableSet = new HashSet<>();
-
- if (StringUtils.isNotBlank(tableList)) {
- for (String s : tableList.split(",")) {
- if (StringUtils.isNotBlank(s)) {
- String[] arr = s.split("\\.");
- TableId tableId = tableId(arr[0].trim(), arr[1].trim());
- localTableSet.add(tableId);
- }
- }
- }
-
- if (StringUtils.isNotBlank(databaseName) && StringUtils.isNotBlank(tableName)) {
- try {
- List tableIds = getSnapshotConnection().getTables(databaseName, tableName);
- LOG.info("Pattern matched tables: {}", tableIds);
- localTableSet.addAll(tableIds);
- } catch (SQLException e) {
- LOG.error(
- String.format(
- "Query table list by 'databaseName' %s and 'tableName' %s failed.",
- databaseName, tableName),
- e);
- throw new FlinkRuntimeException(e);
- }
- }
-
- if (localTableSet.isEmpty()) {
- throw new FlinkRuntimeException("No valid table found");
- }
-
- LOG.info("Table list: {}", localTableSet);
- this.tableSet = localTableSet;
- // for some 4.x versions, it will be treated as 'tenant.*.*'
- if (this.obReaderConfig != null) {
- this.obReaderConfig.setTableWhiteList(
- localTableSet.stream()
- .map(tableId -> String.format("%s.%s", tenantName, tableId.toString()))
- .collect(Collectors.joining("|")));
- }
- }
-
- private TableSchema getTableSchema(TableId tableId) {
- if (databaseSchema == null) {
- databaseSchema =
- new OceanBaseDatabaseSchema(connectorConfig, t -> tableSet.contains(t), false);
- }
- TableSchema tableSchema = databaseSchema.schemaFor(tableId);
- if (tableSchema != null) {
- return tableSchema;
- }
-
- if (obSchema == null) {
- obSchema = new OceanBaseSchema();
- }
- TableChanges.TableChange tableChange =
- obSchema.getTableSchema(getSnapshotConnection(), tableId);
- databaseSchema.refresh(tableChange.getTable());
- return databaseSchema.schemaFor(tableId);
- }
-
- protected void readSnapshotRecords() {
- tableSet.forEach(this::readSnapshotRecordsByTable);
- }
-
- private void readSnapshotRecordsByTable(TableId tableId) {
- String fullName = getSnapshotConnection().quotedTableIdString(tableId);
- sourceInfo.tableEvent(tableId);
- try (OceanBaseConnection connection = getSnapshotConnection()) {
- LOG.info("Start to read snapshot from {}", connection.quotedTableIdString(tableId));
- connection.query(
- "SELECT * FROM " + fullName,
- rs -> {
- TableSchema tableSchema = getTableSchema(tableId);
- List fields = tableSchema.valueSchema().fields();
-
- while (rs.next()) {
- Object[] fieldValues = new Object[fields.size()];
- for (Field field : fields) {
- fieldValues[field.index()] = rs.getObject(field.name());
- }
- Struct value = tableSchema.valueFromColumnData(fieldValues);
- Instant now = Instant.now();
- Struct struct =
- tableSchema
- .getEnvelopeSchema()
- .read(value, sourceInfo.struct(), now);
- try {
- deserializer.deserialize(
- new SourceRecord(
- null,
- null,
- tableId.identifier(),
- null,
- null,
- null,
- struct.schema(),
- struct),
- outputCollector);
- } catch (Exception e) {
- LOG.error("Deserialize snapshot record failed ", e);
- throw new FlinkRuntimeException(e);
- }
- }
- });
- LOG.info("Read snapshot from {} finished", fullName);
- } catch (SQLException e) {
- LOG.error("Read snapshot from table " + fullName + " failed", e);
- throw new FlinkRuntimeException(e);
- }
- }
-
- protected void readChangeRecords() throws InterruptedException, TimeoutException {
- if (resolvedTimestamp > 0) {
- obReaderConfig.updateCheckpoint(Long.toString(resolvedTimestamp));
- LOG.info("Restore from timestamp: {}", resolvedTimestamp);
- }
-
- ClientConf clientConf =
- ClientConf.builder()
- .clientId(
- logProxyClientId != null
- ? logProxyClientId
- : String.format(
- "%s_%s_%s",
- ClientUtil.generateClientId(),
- Thread.currentThread().getId(),
- tenantName))
- .maxReconnectTimes(0)
- .connectTimeoutMs((int) connectTimeout.toMillis())
- .build();
-
- logProxyClient = new LogProxyClient(logProxyHost, logProxyPort, obReaderConfig, clientConf);
-
- final CountDownLatch latch = new CountDownLatch(1);
-
- logProxyClient.addListener(
- new RecordListener() {
-
- boolean started = false;
-
- @Override
- public void notify(LogMessage message) {
- switch (message.getOpt()) {
- case HEARTBEAT:
- case BEGIN:
- if (!started) {
- started = true;
- latch.countDown();
- }
- break;
- case INSERT:
- case UPDATE:
- case DELETE:
- if (!started) {
- break;
- }
- SourceRecord record = getChangeRecord(message);
- if (record != null) {
- changeRecordBuffer.add(record);
- }
- break;
- case COMMIT:
- changeRecordBuffer.forEach(
- r -> {
- try {
- deserializer.deserialize(r, outputCollector);
- } catch (Exception e) {
- throw new FlinkRuntimeException(e);
- }
- });
- changeRecordBuffer.clear();
- long timestamp = Long.parseLong(message.getSafeTimestamp());
- if (timestamp > resolvedTimestamp) {
- resolvedTimestamp = timestamp;
- }
- break;
- case DDL:
- // TODO record ddl and remove expired table schema
- LOG.trace(
- "Ddl: {}",
- message.getFieldList().get(0).getValue().toString());
- break;
- default:
- throw new UnsupportedOperationException(
- "Unsupported type: " + message.getOpt());
- }
- }
-
- @Override
- public void onException(LogProxyClientException e) {
- logProxyClientException = e;
- logProxyClient.stop();
- }
- });
-
- LOG.info(
- "Try to start LogProxyClient with client id: {}, config: {}",
- clientConf.getClientId(),
- obReaderConfig);
- logProxyClient.start();
-
- if (!latch.await(connectTimeout.getSeconds(), TimeUnit.SECONDS)) {
- throw new TimeoutException(
- "Timeout to receive log messages in LogProxyClient.RecordListener");
- }
- LOG.info("LogProxyClient started successfully");
-
- logProxyClient.join();
-
- if (logProxyClientException != null) {
- throw new RuntimeException("LogProxyClient exception", logProxyClientException);
- }
- }
-
- private SourceRecord getChangeRecord(LogMessage message) {
- String databaseName = message.getDbName().replace(tenantName + ".", "");
- TableId tableId = tableId(databaseName, message.getTableName());
- if (!tableSet.contains(tableId)) {
- return null;
- }
-
- sourceInfo.tableEvent(tableId);
- sourceInfo.setSourceTime(Instant.ofEpochSecond(Long.parseLong(message.getTimestamp())));
- Struct source = sourceInfo.struct();
-
- TableSchema tableSchema = getTableSchema(tableId);
- Struct struct;
- Schema valueSchema = tableSchema.valueSchema();
- List fields = valueSchema.fields();
- Struct before, after;
- Object[] beforeFieldValues, afterFieldValues;
- Map beforeValueMap = new HashMap<>();
- Map afterValueMap = new HashMap<>();
- message.getFieldList()
- .forEach(
- field -> {
- if (field.isPrev()) {
- beforeValueMap.put(field.getFieldname(), getFieldValue(field));
- } else {
- afterValueMap.put(field.getFieldname(), getFieldValue(field));
- }
- });
- switch (message.getOpt()) {
- case INSERT:
- afterFieldValues = new Object[fields.size()];
- for (Field field : fields) {
- afterFieldValues[field.index()] = afterValueMap.get(field.name());
- }
- after = tableSchema.valueFromColumnData(afterFieldValues);
- struct = tableSchema.getEnvelopeSchema().create(after, source, Instant.now());
- break;
- case DELETE:
- beforeFieldValues = new Object[fields.size()];
- for (Field field : fields) {
- beforeFieldValues[field.index()] = beforeValueMap.get(field.name());
- }
- before = tableSchema.valueFromColumnData(beforeFieldValues);
- struct = tableSchema.getEnvelopeSchema().delete(before, source, Instant.now());
- break;
- case UPDATE:
- beforeFieldValues = new Object[fields.size()];
- afterFieldValues = new Object[fields.size()];
- for (Field field : fields) {
- beforeFieldValues[field.index()] = beforeValueMap.get(field.name());
- afterFieldValues[field.index()] = afterValueMap.get(field.name());
- }
- before = tableSchema.valueFromColumnData(beforeFieldValues);
- after = tableSchema.valueFromColumnData(afterFieldValues);
- struct =
- tableSchema
- .getEnvelopeSchema()
- .update(before, after, source, Instant.now());
- break;
- default:
- throw new UnsupportedOperationException();
- }
- return new SourceRecord(
- null, null, tableId.identifier(), null, null, null, struct.schema(), struct);
- }
-
- private Object getFieldValue(DataMessage.Record.Field field) {
- if (field.getValue() == null) {
- return null;
- }
- String encoding = field.getEncoding();
- if ("binary".equalsIgnoreCase(encoding)) {
- return field.getValue().getBytes();
- }
- return field.getValue().toString(encoding);
- }
-
- @Override
- public void notifyCheckpointComplete(long l) {
- // do nothing
- }
-
- @Override
- public TypeInformation getProducedType() {
- return this.deserializer.getProducedType();
- }
-
- @Override
- public void snapshotState(FunctionSnapshotContext context) throws Exception {
- LOG.info(
- "snapshotState checkpoint: {} at resolvedTimestamp: {}",
- context.getCheckpointId(),
- resolvedTimestamp);
- offsetState.clear();
- offsetState.add(resolvedTimestamp);
- }
-
- @Override
- public void initializeState(FunctionInitializationContext context) throws Exception {
- LOG.info("initialize checkpoint");
- offsetState =
- context.getOperatorStateStore()
- .getListState(
- new ListStateDescriptor<>(
- "resolvedTimestampState", LongSerializer.INSTANCE));
- if (context.isRestored()) {
- for (final Long offset : offsetState.get()) {
- resolvedTimestamp = offset;
- LOG.info("Restore State from resolvedTimestamp: {}", resolvedTimestamp);
- return;
- }
- }
- }
-
- @Override
- public void cancel() {
- closeSnapshotConnection();
- if (logProxyClient != null) {
- logProxyClient.stop();
- }
- }
-
- private static class OutputCollector implements Collector {
-
- private SourceContext context;
-
- @Override
- public void collect(T record) {
- context.collect(record);
- }
-
- @Override
- public void close() {
- // do nothing
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java
deleted file mode 100644
index a2b55b60c71..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/config/OceanBaseConnectorConfig.java
+++ /dev/null
@@ -1,88 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.config;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfoStructMaker;
-
-import io.debezium.config.Configuration;
-import io.debezium.connector.SourceInfoStructMaker;
-import io.debezium.relational.ColumnFilterMode;
-import io.debezium.relational.RelationalDatabaseConnectorConfig;
-import io.debezium.relational.TableId;
-import io.debezium.relational.Tables;
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Properties;
-
-/** Debezium connector config. */
-public class OceanBaseConnectorConfig extends RelationalDatabaseConnectorConfig {
-
- protected static final String LOGICAL_NAME = "oceanbase_cdc_connector";
- protected static final int DEFAULT_SNAPSHOT_FETCH_SIZE = Integer.MIN_VALUE;
- protected static final List BUILT_IN_DB_NAMES =
- Collections.unmodifiableList(
- Arrays.asList(
- "information_schema", "mysql", "oceanbase", "LBACSYS", "ORAAUDITOR"));
-
- private final String compatibleMode;
- private final String serverTimeZone;
-
- public OceanBaseConnectorConfig(
- String compatibleMode, String serverTimeZone, Properties properties) {
- super(
- Configuration.from(properties),
- LOGICAL_NAME,
- Tables.TableFilter.fromPredicate(
- tableId ->
- "mysql".equalsIgnoreCase(compatibleMode)
- ? !BUILT_IN_DB_NAMES.contains(tableId.catalog())
- : !BUILT_IN_DB_NAMES.contains(tableId.schema())),
- TableId::identifier,
- DEFAULT_SNAPSHOT_FETCH_SIZE,
- "mysql".equalsIgnoreCase(compatibleMode)
- ? ColumnFilterMode.CATALOG
- : ColumnFilterMode.SCHEMA);
- this.compatibleMode = compatibleMode;
- this.serverTimeZone = serverTimeZone;
- }
-
- public String getCompatibleMode() {
- return compatibleMode;
- }
-
- public String getServerTimeZone() {
- return serverTimeZone;
- }
-
- @Override
- public String getConnectorName() {
- return "oceanbase";
- }
-
- @Override
- public String getContextName() {
- return "OceanBase";
- }
-
- @Override
- protected SourceInfoStructMaker> getSourceInfoStructMaker(Version version) {
- return new OceanBaseSourceInfoStructMaker();
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java
deleted file mode 100644
index 49ad532a13b..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/connection/OceanBaseConnection.java
+++ /dev/null
@@ -1,334 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.connection;
-
-import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import io.debezium.jdbc.JdbcConfiguration;
-import io.debezium.jdbc.JdbcConnection;
-import io.debezium.relational.Column;
-import io.debezium.relational.TableId;
-import io.debezium.relational.Tables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.sql.DatabaseMetaData;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Set;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-/** {@link JdbcConnection} extension to be used with OceanBase server. */
-public class OceanBaseConnection extends JdbcConnection {
-
- private static final Logger LOG = LoggerFactory.getLogger(OceanBaseConnection.class);
-
- private static final Properties DEFAULT_JDBC_PROPERTIES = initializeDefaultJdbcProperties();
- private static final String MYSQL_URL_PATTERN =
- "jdbc:mysql://${hostname}:${port}/?connectTimeout=${connectTimeout}";
- private static final String OB_URL_PATTERN =
- "jdbc:oceanbase://${hostname}:${port}/?connectTimeout=${connectTimeout}";
-
- private static final int TYPE_BINARY_FLOAT = 100;
- private static final int TYPE_BINARY_DOUBLE = 101;
- private static final int TYPE_TIMESTAMP_WITH_TIME_ZONE = -101;
- private static final int TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE = -102;
- private static final int TYPE_INTERVAL_YEAR_TO_MONTH = -103;
- private static final int TYPE_INTERVAL_DAY_TO_SECOND = -104;
-
- private final String compatibleMode;
-
- public OceanBaseConnection(
- String hostname,
- Integer port,
- String user,
- String password,
- Duration timeout,
- String compatibleMode,
- String jdbcDriver,
- Properties jdbcProperties,
- ClassLoader classLoader) {
- super(
- config(hostname, port, user, password, timeout),
- JdbcConnection.patternBasedFactory(
- formatJdbcUrl(jdbcDriver, jdbcProperties), jdbcDriver, classLoader),
- getQuote(compatibleMode) + "",
- getQuote(compatibleMode) + "");
- this.compatibleMode = compatibleMode;
- }
-
- private static JdbcConfiguration config(
- String hostname, Integer port, String user, String password, Duration timeout) {
- return JdbcConfiguration.create()
- .with("hostname", hostname)
- .with("port", port)
- .with("user", user)
- .with("password", password)
- .with("connectTimeout", timeout == null ? 30000 : timeout.toMillis())
- .build();
- }
-
- private static String formatJdbcUrl(String jdbcDriver, Properties jdbcProperties) {
- Properties combinedProperties = new Properties();
- combinedProperties.putAll(DEFAULT_JDBC_PROPERTIES);
- if (jdbcProperties != null) {
- combinedProperties.putAll(jdbcProperties);
- }
- String urlPattern =
- OceanBaseUtils.isOceanBaseDriver(jdbcDriver) ? OB_URL_PATTERN : MYSQL_URL_PATTERN;
- StringBuilder jdbcUrlStringBuilder = new StringBuilder(urlPattern);
- combinedProperties.forEach(
- (key, value) -> {
- jdbcUrlStringBuilder.append("&").append(key).append("=").append(value);
- });
- return jdbcUrlStringBuilder.toString();
- }
-
- private static Properties initializeDefaultJdbcProperties() {
- Properties defaultJdbcProperties = new Properties();
- defaultJdbcProperties.setProperty("useInformationSchema", "true");
- defaultJdbcProperties.setProperty("nullCatalogMeansCurrent", "false");
- defaultJdbcProperties.setProperty("useUnicode", "true");
- defaultJdbcProperties.setProperty("zeroDateTimeBehavior", "convertToNull");
- defaultJdbcProperties.setProperty("characterEncoding", "UTF-8");
- defaultJdbcProperties.setProperty("characterSetResults", "UTF-8");
- return defaultJdbcProperties;
- }
-
- private static char getQuote(String compatibleMode) {
- return "mysql".equalsIgnoreCase(compatibleMode) ? '`' : '"';
- }
-
- /**
- * Get current timestamp number in seconds.
- *
- * @return current timestamp number.
- * @throws SQLException If a database access error occurs.
- */
- public long getCurrentTimestampS() throws SQLException {
- try {
- long globalTimestamp = getGlobalTimestamp();
- LOG.info("Global timestamp: {}", globalTimestamp);
- return Long.parseLong(String.valueOf(globalTimestamp).substring(0, 10));
- } catch (Exception e) {
- LOG.warn("Failed to get global timestamp, use local timestamp instead");
- }
- return getCurrentTimestamp()
- .orElseThrow(IllegalStateException::new)
- .toInstant()
- .getEpochSecond();
- }
-
- private long getGlobalTimestamp() throws SQLException {
- String schema = "mysql".equalsIgnoreCase(compatibleMode) ? "oceanbase" : "SYS";
- return querySingleValue(
- connection(),
- "SELECT TS_VALUE FROM " + schema + ".V$OB_TIMESTAMP_SERVICE",
- ps -> {},
- rs -> rs.getLong(1));
- }
-
- @Override
- public Optional getCurrentTimestamp() throws SQLException {
- return queryAndMap(
- "mysql".equalsIgnoreCase(compatibleMode)
- ? "SELECT CURRENT_TIMESTAMP"
- : "SELECT CURRENT_TIMESTAMP FROM DUAL",
- rs -> rs.next() ? Optional.of(rs.getTimestamp(1)) : Optional.empty());
- }
-
- /**
- * Get table list by database name pattern and table name pattern.
- *
- * @param dbPattern Database name pattern.
- * @param tbPattern Table name pattern.
- * @return TableId list.
- * @throws SQLException If a database access error occurs.
- */
- public List getTables(String dbPattern, String tbPattern) throws SQLException {
- List result = new ArrayList<>();
- DatabaseMetaData metaData = connection().getMetaData();
- switch (compatibleMode.toLowerCase()) {
- case "mysql":
- List dbNames = getResultList(metaData.getCatalogs(), "TABLE_CAT");
- dbNames =
- dbNames.stream()
- .filter(dbName -> Pattern.matches(dbPattern, dbName))
- .collect(Collectors.toList());
- for (String dbName : dbNames) {
- List tableNames =
- getResultList(
- metaData.getTables(dbName, null, null, supportedTableTypes()),
- "TABLE_NAME");
- tableNames.stream()
- .filter(tbName -> Pattern.matches(tbPattern, tbName))
- .forEach(tbName -> result.add(new TableId(dbName, null, tbName)));
- }
- break;
- case "oracle":
- List schemaNames = getResultList(metaData.getSchemas(), "TABLE_SCHEM");
- schemaNames =
- schemaNames.stream()
- .filter(schemaName -> Pattern.matches(dbPattern, schemaName))
- .collect(Collectors.toList());
- for (String schemaName : schemaNames) {
- List tableNames =
- getResultList(
- metaData.getTables(
- null, schemaName, null, supportedTableTypes()),
- "TABLE_NAME");
- tableNames.stream()
- .filter(tbName -> Pattern.matches(tbPattern, tbName))
- .forEach(tbName -> result.add(new TableId(null, schemaName, tbName)));
- }
- break;
- default:
- throw new FlinkRuntimeException("Unsupported compatible mode: " + compatibleMode);
- }
- return result;
- }
-
- private List getResultList(ResultSet resultSet, String columnName) throws SQLException {
- List result = new ArrayList<>();
- while (resultSet.next()) {
- result.add(resultSet.getString(columnName));
- }
- return result;
- }
-
- @Override
- protected String[] supportedTableTypes() {
- return new String[] {"TABLE"};
- }
-
- @Override
- public String quotedTableIdString(TableId tableId) {
- return tableId.toQuotedString(getQuote(compatibleMode));
- }
-
- public void readSchemaForCapturedTables(
- Tables tables,
- String databaseCatalog,
- String schemaNamePattern,
- Tables.ColumnNameFilter columnFilter,
- boolean removeTablesNotFoundInJdbc,
- Set capturedTables)
- throws SQLException {
-
- Set tableIdsBefore = new HashSet<>(tables.tableIds());
-
- DatabaseMetaData metadata = connection().getMetaData();
- Map> columnsByTable = new HashMap<>();
-
- for (TableId tableId : capturedTables) {
- try (ResultSet columnMetadata =
- metadata.getColumns(
- databaseCatalog, schemaNamePattern, tableId.table(), null)) {
- while (columnMetadata.next()) {
- // add all whitelisted columns
- readTableColumn(columnMetadata, tableId, columnFilter)
- .ifPresent(
- column -> {
- columnsByTable
- .computeIfAbsent(tableId, t -> new ArrayList<>())
- .add(column.create());
- });
- }
- }
- }
-
- // Read the metadata for the primary keys ...
- for (Map.Entry> tableEntry : columnsByTable.entrySet()) {
- // First get the primary key information, which must be done for *each* table ...
- List pkColumnNames = readPrimaryKeyNames(metadata, tableEntry.getKey());
-
- // Then define the table ...
- List columns = tableEntry.getValue();
- Collections.sort(columns);
- tables.overwriteTable(tableEntry.getKey(), columns, pkColumnNames, null);
- }
-
- if (removeTablesNotFoundInJdbc) {
- // Remove any definitions for tables that were not found in the database metadata ...
- tableIdsBefore.removeAll(columnsByTable.keySet());
- tableIdsBefore.forEach(tables::removeTable);
- }
- }
-
- @Override
- protected int resolveNativeType(String typeName) {
- String upperCaseTypeName = typeName.toUpperCase();
- if (upperCaseTypeName.startsWith("JSON")) {
- return Types.VARCHAR;
- }
- if (upperCaseTypeName.startsWith("NCHAR")) {
- return Types.NCHAR;
- }
- if (upperCaseTypeName.startsWith("NVARCHAR2")) {
- return Types.NVARCHAR;
- }
- if (upperCaseTypeName.startsWith("TIMESTAMP")) {
- if (upperCaseTypeName.contains("WITH TIME ZONE")) {
- return TYPE_TIMESTAMP_WITH_TIME_ZONE;
- }
- if (upperCaseTypeName.contains("WITH LOCAL TIME ZONE")) {
- return TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE;
- }
- return Types.TIMESTAMP;
- }
- if (upperCaseTypeName.startsWith("INTERVAL")) {
- if (upperCaseTypeName.contains("TO MONTH")) {
- return TYPE_INTERVAL_YEAR_TO_MONTH;
- }
- if (upperCaseTypeName.contains("TO SECOND")) {
- return TYPE_INTERVAL_DAY_TO_SECOND;
- }
- }
- return Column.UNSET_INT_VALUE;
- }
-
- @Override
- protected int resolveJdbcType(int metadataJdbcType, int nativeType) {
- switch (metadataJdbcType) {
- case TYPE_BINARY_FLOAT:
- return Types.REAL;
- case TYPE_BINARY_DOUBLE:
- return Types.DOUBLE;
- case TYPE_TIMESTAMP_WITH_TIME_ZONE:
- case TYPE_TIMESTAMP_WITH_LOCAL_TIME_ZONE:
- case TYPE_INTERVAL_YEAR_TO_MONTH:
- case TYPE_INTERVAL_DAY_TO_SECOND:
- return Types.OTHER;
- default:
- return nativeType == Column.UNSET_INT_VALUE ? metadataJdbcType : nativeType;
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java
deleted file mode 100644
index 3256b8a777f..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseDeserializationConverterFactory.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.converter;
-
-import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverter;
-import org.apache.flink.cdc.debezium.table.DeserializationRuntimeConverterFactory;
-import org.apache.flink.table.data.GenericArrayData;
-import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.types.logical.LogicalType;
-
-import org.apache.kafka.connect.data.Schema;
-
-import java.time.ZoneId;
-import java.util.Optional;
-
-/** Used to create {@link DeserializationRuntimeConverterFactory} specified to OceanBase. */
-public class OceanBaseDeserializationConverterFactory {
-
- public static DeserializationRuntimeConverterFactory instance() {
- return new DeserializationRuntimeConverterFactory() {
-
- private static final long serialVersionUID = 1L;
-
- @Override
- public Optional createUserDefinedConverter(
- LogicalType logicalType, ZoneId serverTimeZone) {
- switch (logicalType.getTypeRoot()) {
- case ARRAY:
- return createArrayConverter();
- default:
- return Optional.empty();
- }
- }
- };
- }
-
- private static Optional createArrayConverter() {
- return Optional.of(
- new DeserializationRuntimeConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object convert(Object dbzObj, Schema schema) throws Exception {
- if (dbzObj instanceof String) {
- String[] enums = ((String) dbzObj).split(",");
- StringData[] elements = new StringData[enums.length];
- for (int i = 0; i < enums.length; i++) {
- elements[i] = StringData.fromString(enums[i]);
- }
- return new GenericArrayData(elements);
- }
- throw new IllegalArgumentException(
- String.format(
- "Unable convert to Flink ARRAY type from unexpected value '%s'",
- dbzObj));
- }
- });
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java
deleted file mode 100644
index bf2a125db78..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/converter/OceanBaseValueConverters.java
+++ /dev/null
@@ -1,509 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.converter;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig;
-
-import io.debezium.config.CommonConnectorConfig;
-import io.debezium.data.Bits;
-import io.debezium.data.SpecialValueDecimal;
-import io.debezium.jdbc.JdbcValueConverters;
-import io.debezium.relational.Column;
-import io.debezium.relational.ValueConverter;
-import io.debezium.time.MicroTimestamp;
-import io.debezium.time.NanoTimestamp;
-import org.apache.kafka.connect.data.Decimal;
-import org.apache.kafka.connect.data.Field;
-import org.apache.kafka.connect.data.SchemaBuilder;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.nio.ByteBuffer;
-import java.sql.Blob;
-import java.sql.Clob;
-import java.sql.Date;
-import java.sql.SQLException;
-import java.sql.Time;
-import java.sql.Timestamp;
-import java.sql.Types;
-import java.time.Instant;
-import java.time.LocalDateTime;
-import java.time.ZoneId;
-import java.time.ZoneOffset;
-import java.time.format.DateTimeFormatter;
-import java.time.format.DateTimeFormatterBuilder;
-import java.time.temporal.ChronoField;
-import java.util.Locale;
-
-/** JdbcValueConverters for OceanBase. */
-public class OceanBaseValueConverters extends JdbcValueConverters {
-
- public static final String EMPTY_BLOB_FUNCTION = "EMPTY_BLOB()";
- public static final String EMPTY_CLOB_FUNCTION = "EMPTY_CLOB()";
-
- private static final DateTimeFormatter TIMESTAMP_FORMATTER =
- new DateTimeFormatterBuilder()
- .parseCaseInsensitive()
- .appendPattern("yyyy-MM-dd HH:mm:ss")
- .optionalStart()
- .appendPattern(".")
- .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false)
- .optionalEnd()
- .toFormatter();
-
- private static final DateTimeFormatter TIMESTAMP_AM_PM_SHORT_FORMATTER =
- new DateTimeFormatterBuilder()
- .parseCaseInsensitive()
- .appendPattern("dd-MMM-yy hh.mm.ss")
- .optionalStart()
- .appendPattern(".")
- .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, false)
- .optionalEnd()
- .appendPattern(" a")
- .toFormatter(Locale.ENGLISH);
-
- private final String compatibleMode;
- private final String serverTimezone;
-
- public OceanBaseValueConverters(OceanBaseConnectorConfig connectorConfig) {
- super(
- connectorConfig.getDecimalMode(),
- connectorConfig.getTemporalPrecisionMode(),
- ZoneOffset.UTC,
- x -> x,
- BigIntUnsignedMode.PRECISE,
- connectorConfig.binaryHandlingMode());
- this.compatibleMode = connectorConfig.getCompatibleMode();
- this.serverTimezone = connectorConfig.getServerTimeZone();
- }
-
- @Override
- protected int getTimePrecision(Column column) {
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- return super.getTimePrecision(column);
- }
- return column.scale().orElse(0);
- }
-
- protected boolean isUnsignedColumn(Column column) {
- return column.typeName().toUpperCase().contains("UNSIGNED");
- }
-
- @Override
- public SchemaBuilder schemaBuilder(Column column) {
- logger.debug(
- "Building schema for column {} of type {} named {} with constraints ({},{})",
- column.name(),
- column.jdbcType(),
- column.typeName(),
- column.length(),
- column.scale());
-
- switch (column.jdbcType()) {
- case Types.BIT:
- if (column.length() > 1) {
- return Bits.builder(column.length());
- }
- return SchemaBuilder.bool();
- case Types.TINYINT:
- if (column.length() == 1) {
- return SchemaBuilder.bool();
- }
- if (isUnsignedColumn(column)) {
- return SchemaBuilder.int16();
- }
- return SchemaBuilder.int8();
- case Types.SMALLINT:
- if (isUnsignedColumn(column)) {
- return SchemaBuilder.int32();
- }
- return SchemaBuilder.int16();
- case Types.INTEGER:
- if (!column.typeName().toUpperCase().startsWith("MEDIUMINT")
- && isUnsignedColumn(column)) {
- return SchemaBuilder.int64();
- }
- return SchemaBuilder.int32();
- case Types.BIGINT:
- if (isUnsignedColumn(column)) {
- return Decimal.builder(0);
- }
- return SchemaBuilder.int64();
- case Types.FLOAT:
- return getDecimalSchema(column);
- case Types.NUMERIC:
- case Types.DECIMAL:
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- return getDecimalSchema(column);
- }
- return getNumericSchema(column);
- case Types.REAL:
- return SchemaBuilder.float32();
- case Types.DOUBLE:
- return SchemaBuilder.float64();
- case Types.DATE:
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- if (column.typeName().equalsIgnoreCase("YEAR")) {
- return io.debezium.time.Year.builder();
- }
- if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) {
- return io.debezium.time.Date.builder();
- }
- return org.apache.kafka.connect.data.Date.builder();
- }
- return getTimestampSchema(column);
- case Types.TIME:
- if (adaptiveTimeMicrosecondsPrecisionMode) {
- return io.debezium.time.MicroTime.builder();
- }
- if (adaptiveTimePrecisionMode) {
- if (getTimePrecision(column) <= 3) {
- return io.debezium.time.Time.builder();
- }
- if (getTimePrecision(column) <= 6) {
- return io.debezium.time.MicroTime.builder();
- }
- return io.debezium.time.NanoTime.builder();
- }
- return org.apache.kafka.connect.data.Time.builder();
- case Types.TIMESTAMP:
- return getTimestampSchema(column);
- case Types.CHAR:
- case Types.VARCHAR:
- case Types.LONGVARCHAR:
- case Types.NCHAR:
- case Types.NVARCHAR:
- case Types.CLOB:
- return SchemaBuilder.string();
- case Types.BINARY:
- case Types.VARBINARY:
- case Types.LONGVARBINARY:
- case Types.BLOB:
- return binaryMode.getSchema();
- default:
- return super.schemaBuilder(column);
- }
- }
-
- protected SchemaBuilder getNumericSchema(Column column) {
- if (column.scale().isPresent()) {
- int scale = column.scale().get();
- if (scale <= 0) {
- int width = column.length() - scale;
- if (width < 3) {
- return SchemaBuilder.int8();
- } else if (width < 5) {
- return SchemaBuilder.int16();
- } else if (width < 10) {
- return SchemaBuilder.int32();
- } else if (width < 19) {
- return SchemaBuilder.int64();
- }
- }
- }
- return getDecimalSchema(column);
- }
-
- protected SchemaBuilder getDecimalSchema(Column column) {
- return SpecialValueDecimal.builder(decimalMode, column.length(), column.scale().orElse(0));
- }
-
- protected SchemaBuilder getTimestampSchema(Column column) {
- if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) {
- if (getTimePrecision(column) <= 3) {
- return io.debezium.time.Timestamp.builder();
- }
- if (getTimePrecision(column) <= 6) {
- return MicroTimestamp.builder();
- }
- return NanoTimestamp.builder();
- }
- return org.apache.kafka.connect.data.Timestamp.builder();
- }
-
- @Override
- public ValueConverter converter(Column column, Field fieldDefn) {
- switch (column.jdbcType()) {
- case Types.BIT:
- return convertBits(column, fieldDefn);
- case Types.TINYINT:
- if (column.length() == 1) {
- return data -> convertBit(column, fieldDefn, data);
- }
- if (isUnsignedColumn(column)) {
- return data -> convertSmallInt(column, fieldDefn, data);
- }
- return data -> convertTinyInt(column, fieldDefn, data);
- case Types.SMALLINT:
- if (isUnsignedColumn(column)) {
- return data -> convertInteger(column, fieldDefn, data);
- }
- return data -> convertSmallInt(column, fieldDefn, data);
- case Types.INTEGER:
- if (column.typeName().toUpperCase().startsWith("MEDIUMINT")) {
- return data -> convertInteger(column, fieldDefn, data);
- }
- if (isUnsignedColumn(column)) {
- return data -> convertBigInt(column, fieldDefn, data);
- }
- return data -> convertInteger(column, fieldDefn, data);
- case Types.BIGINT:
- if (isUnsignedColumn(column)) {
- switch (bigIntUnsignedMode) {
- case LONG:
- return (data) -> convertBigInt(column, fieldDefn, data);
- case PRECISE:
- return (data) -> convertUnsignedBigint(column, fieldDefn, data);
- }
- }
- return (data) -> convertBigInt(column, fieldDefn, data);
- case Types.FLOAT:
- return data -> convertDecimal(column, fieldDefn, data);
- case Types.NUMERIC:
- case Types.DECIMAL:
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- return data -> convertDecimal(column, fieldDefn, data);
- }
- return data -> convertNumeric(column, fieldDefn, data);
- case Types.REAL:
- return data -> convertReal(column, fieldDefn, data);
- case Types.DOUBLE:
- return data -> convertDouble(column, fieldDefn, data);
- case Types.DATE:
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- if (column.typeName().equalsIgnoreCase("YEAR")) {
- return (data) -> convertYearToInt(column, fieldDefn, data);
- }
- if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) {
- return (data) -> convertDateToEpochDays(column, fieldDefn, data);
- }
- return (data) -> convertDateToEpochDaysAsDate(column, fieldDefn, data);
- }
- return (data) -> convertTimestamp(column, fieldDefn, data);
- case Types.TIME:
- return (data) -> convertTime(column, fieldDefn, data);
- case Types.TIMESTAMP:
- return data -> convertTimestamp(column, fieldDefn, data);
- case Types.CHAR:
- case Types.VARCHAR:
- case Types.LONGVARCHAR:
- case Types.NCHAR:
- case Types.NVARCHAR:
- case Types.CLOB:
- return data -> convertString(column, fieldDefn, data);
- case Types.BINARY:
- case Types.VARBINARY:
- case Types.LONGVARBINARY:
- case Types.BLOB:
- return (data) -> convertBinary(column, fieldDefn, data, binaryMode);
- default:
- return super.converter(column, fieldDefn);
- }
- }
-
- @Override
- protected Object convertBits(Column column, Field fieldDefn, Object data, int numBytes) {
- if (data instanceof String) {
- return ByteBuffer.allocate(numBytes).putLong(Long.parseLong((String) data)).array();
- }
- return super.convertBits(column, fieldDefn, data, numBytes);
- }
-
- @Override
- protected Object convertBit(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return Boolean.parseBoolean((String) data) || "1".equals(data);
- }
- return super.convertBit(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertTinyInt(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return Byte.parseByte((String) data);
- }
- if (data instanceof Number) {
- return ((Number) data).byteValue();
- }
- throw new IllegalArgumentException(
- "Unexpected value for JDBC type "
- + column.jdbcType()
- + " and column "
- + column
- + ": class="
- + data.getClass());
- }
-
- @Override
- protected Object convertBigInt(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return new BigInteger((String) data).longValue();
- }
- return super.convertBigInt(column, fieldDefn, data);
- }
-
- protected Object convertUnsignedBigint(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return new BigDecimal((String) data);
- }
- if (data instanceof BigInteger) {
- return new BigDecimal((BigInteger) data);
- }
- return convertDecimal(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertReal(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return Float.parseFloat((String) data);
- }
- return super.convertReal(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertDouble(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- return Double.parseDouble((String) data);
- }
- return super.convertDouble(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertNumeric(Column column, Field fieldDefn, Object data) {
- if (column.scale().isPresent()) {
- int scale = column.scale().get();
-
- if (scale <= 0) {
- int width = column.length() - scale;
- if (width < 3) {
- return convertTinyInt(column, fieldDefn, data);
- } else if (width < 5) {
- return convertSmallInt(column, fieldDefn, data);
- } else if (width < 10) {
- return convertInteger(column, fieldDefn, data);
- } else if (width < 19) {
- return convertBigInt(column, fieldDefn, data);
- }
- }
- }
- return convertDecimal(column, fieldDefn, data);
- }
-
- protected Object convertYearToInt(Column column, Field fieldDefn, Object data) {
- if (data instanceof Date) {
- return ((Date) data).toLocalDate().getYear();
- }
- return convertInteger(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertDateToEpochDays(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- data = Date.valueOf((String) data);
- }
- return super.convertDateToEpochDays(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertDateToEpochDaysAsDate(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- data = Date.valueOf((String) data);
- }
- return super.convertDateToEpochDaysAsDate(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertTime(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- data = Time.valueOf((String) data);
- }
- return super.convertTime(column, fieldDefn, data);
- }
-
- protected Object convertTimestamp(Column column, Field fieldDefn, Object data) {
- if (data instanceof String) {
- if ("mysql".equalsIgnoreCase(compatibleMode)) {
- data = Timestamp.valueOf(((String) data).trim());
- } else {
- data = resolveTimestampStringAsInstant((String) data);
- }
- }
- if (adaptiveTimePrecisionMode || adaptiveTimeMicrosecondsPrecisionMode) {
- if (getTimePrecision(column) <= 3) {
- return convertTimestampToEpochMillis(column, fieldDefn, data);
- }
- if (getTimePrecision(column) <= 6) {
- return convertTimestampToEpochMicros(column, fieldDefn, data);
- }
- return convertTimestampToEpochNanos(column, fieldDefn, data);
- }
- return convertTimestampToEpochMillisAsDate(column, fieldDefn, data);
- }
-
- protected Instant resolveTimestampStringAsInstant(String dateText) {
- LocalDateTime dateTime;
- if (dateText.indexOf(" AM") > 0 || dateText.indexOf(" PM") > 0) {
- dateTime = LocalDateTime.from(TIMESTAMP_AM_PM_SHORT_FORMATTER.parse(dateText.trim()));
- } else {
- dateTime = LocalDateTime.from(TIMESTAMP_FORMATTER.parse(dateText.trim()));
- }
- return dateTime.atZone(ZoneId.of(serverTimezone)).toInstant();
- }
-
- @Override
- protected Object convertString(Column column, Field fieldDefn, Object data) {
- if (data instanceof Clob) {
- try {
- Clob clob = (Clob) data;
- return clob.getSubString(1, (int) clob.length());
- } catch (SQLException e) {
- throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
- }
- }
- if (data instanceof String) {
- String s = (String) data;
- if (EMPTY_CLOB_FUNCTION.equals(s)) {
- return column.isOptional() ? null : "";
- }
- }
- return super.convertString(column, fieldDefn, data);
- }
-
- @Override
- protected Object convertBinary(
- Column column,
- Field fieldDefn,
- Object data,
- CommonConnectorConfig.BinaryHandlingMode mode) {
- try {
- if (data instanceof Blob) {
- Blob blob = (Blob) data;
- data = blob.getBytes(1, Long.valueOf(blob.length()).intValue());
- }
- if (data instanceof String) {
- String str = (String) data;
- if (EMPTY_BLOB_FUNCTION.equals(str)) {
- data = column.isOptional() ? null : "";
- }
- }
- return super.convertBinary(column, fieldDefn, data, mode);
- } catch (SQLException e) {
- throw new RuntimeException("Couldn't convert value for column " + column.name(), e);
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java
deleted file mode 100644
index 1319b1a22c8..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfo.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.offset;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig;
-
-import io.debezium.connector.common.BaseSourceInfo;
-import io.debezium.relational.TableId;
-
-import java.time.Instant;
-import java.util.Collections;
-import java.util.Objects;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-/** OceanBase source info. */
-public class OceanBaseSourceInfo extends BaseSourceInfo {
-
- public static final String TENANT_KEY = "tenant";
- public static final String TRANSACTION_ID_KEY = "transaction_id";
-
- private final String tenant;
-
- private Instant sourceTime;
- private Set tableIds;
- private String transactionId;
-
- public OceanBaseSourceInfo(OceanBaseConnectorConfig config, String tenant) {
- super(config);
- this.tenant = tenant;
- }
-
- public String tenant() {
- return tenant;
- }
-
- @Override
- protected Instant timestamp() {
- return sourceTime;
- }
-
- public void setSourceTime(Instant sourceTime) {
- this.sourceTime = sourceTime;
- }
-
- public void beginTransaction(String transactionId) {
- this.transactionId = transactionId;
- }
-
- public void commitTransaction() {
- this.transactionId = null;
- }
-
- public String transactionId() {
- return transactionId;
- }
-
- public void tableEvent(TableId tableId) {
- this.tableIds = Collections.singleton(tableId);
- }
-
- @Override
- protected String database() {
- return (tableIds != null) ? tableIds.iterator().next().catalog() : null;
- }
-
- public String tableSchema() {
- return (tableIds == null || tableIds.isEmpty())
- ? null
- : tableIds.stream()
- .filter(Objects::nonNull)
- .map(TableId::schema)
- .filter(Objects::nonNull)
- .distinct()
- .collect(Collectors.joining(","));
- }
-
- public String table() {
- return (tableIds == null || tableIds.isEmpty())
- ? null
- : tableIds.stream()
- .filter(Objects::nonNull)
- .map(TableId::table)
- .collect(Collectors.joining(","));
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java
deleted file mode 100644
index 604ef1e77d5..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/offset/OceanBaseSourceInfoStructMaker.java
+++ /dev/null
@@ -1,74 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.offset;
-
-import io.debezium.connector.SourceInfoStructMaker;
-import org.apache.kafka.connect.data.Schema;
-import org.apache.kafka.connect.data.SchemaBuilder;
-import org.apache.kafka.connect.data.Struct;
-
-import java.time.Instant;
-
-/** The {@link SourceInfoStructMaker} implementation for OceanBase. */
-public class OceanBaseSourceInfoStructMaker implements SourceInfoStructMaker {
- private final Schema schema;
-
- public OceanBaseSourceInfoStructMaker() {
- this.schema =
- SchemaBuilder.struct()
- .field(OceanBaseSourceInfo.TABLE_NAME_KEY, Schema.STRING_SCHEMA)
- .field(OceanBaseSourceInfo.TIMESTAMP_KEY, Schema.INT64_SCHEMA)
- .field(OceanBaseSourceInfo.TENANT_KEY, Schema.OPTIONAL_STRING_SCHEMA)
- .field(OceanBaseSourceInfo.DATABASE_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA)
- .field(OceanBaseSourceInfo.SCHEMA_NAME_KEY, Schema.OPTIONAL_STRING_SCHEMA)
- .field(
- OceanBaseSourceInfo.TRANSACTION_ID_KEY,
- Schema.OPTIONAL_STRING_SCHEMA)
- .build();
- }
-
- @Override
- public Schema schema() {
- return schema;
- }
-
- @Override
- public Struct struct(OceanBaseSourceInfo sourceInfo) {
- Struct source = new Struct(schema);
- source.put(OceanBaseSourceInfo.TABLE_NAME_KEY, sourceInfo.table());
-
- Instant timestamp = sourceInfo.timestamp();
- source.put(
- OceanBaseSourceInfo.TIMESTAMP_KEY,
- timestamp != null ? timestamp.toEpochMilli() : 0);
-
- if (sourceInfo.tenant() != null) {
- source.put(OceanBaseSourceInfo.TENANT_KEY, sourceInfo.tenant());
- }
- if (sourceInfo.database() != null) {
- source.put(OceanBaseSourceInfo.DATABASE_NAME_KEY, sourceInfo.database());
- }
- if (sourceInfo.tableSchema() != null) {
- source.put(OceanBaseSourceInfo.SCHEMA_NAME_KEY, sourceInfo.tableSchema());
- }
- if (sourceInfo.transactionId() != null) {
- source.put(OceanBaseSourceInfo.TRANSACTION_ID_KEY, sourceInfo.transactionId());
- }
- return source;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java
deleted file mode 100644
index 094937bad85..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseDatabaseSchema.java
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.schema;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.config.OceanBaseConnectorConfig;
-import org.apache.flink.cdc.connectors.oceanbase.source.converter.OceanBaseValueConverters;
-
-import io.debezium.relational.RelationalDatabaseSchema;
-import io.debezium.relational.TableSchemaBuilder;
-import io.debezium.relational.Tables;
-import io.debezium.schema.TopicSelector;
-
-/** OceanBase database schema. */
-public class OceanBaseDatabaseSchema extends RelationalDatabaseSchema {
-
- public OceanBaseDatabaseSchema(
- OceanBaseConnectorConfig connectorConfig,
- Tables.TableFilter tableFilter,
- boolean tableIdCaseInsensitive) {
- super(
- connectorConfig,
- TopicSelector.defaultSelector(
- connectorConfig,
- (tableId, prefix, delimiter) ->
- String.join(delimiter, prefix, tableId.identifier())),
- tableFilter,
- connectorConfig.getColumnFilter(),
- new TableSchemaBuilder(
- new OceanBaseValueConverters(connectorConfig),
- connectorConfig.schemaNameAdjustmentMode().createAdjuster(),
- connectorConfig.customConverterRegistry(),
- connectorConfig.getSourceInfoStructMaker().schema(),
- connectorConfig.getSanitizeFieldNames(),
- false),
- tableIdCaseInsensitive,
- connectorConfig.getKeyMapper());
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java
deleted file mode 100644
index b77cc64abd6..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/source/schema/OceanBaseSchema.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.source.schema;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.connection.OceanBaseConnection;
-import org.apache.flink.util.FlinkRuntimeException;
-
-import io.debezium.jdbc.JdbcConnection;
-import io.debezium.relational.Table;
-import io.debezium.relational.TableId;
-import io.debezium.relational.Tables;
-import io.debezium.relational.history.TableChanges;
-
-import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/** A component used to get schema by table path. */
-public class OceanBaseSchema {
-
- private final Map schemasByTableId;
-
- public OceanBaseSchema() {
- this.schemasByTableId = new HashMap<>();
- }
-
- public TableChanges.TableChange getTableSchema(JdbcConnection connection, TableId tableId) {
- TableChanges.TableChange schema = schemasByTableId.get(tableId);
- if (schema == null) {
- schema = readTableSchema(connection, tableId);
- schemasByTableId.put(tableId, schema);
- }
- return schema;
- }
-
- private TableChanges.TableChange readTableSchema(JdbcConnection jdbc, TableId tableId) {
- OceanBaseConnection connection = (OceanBaseConnection) jdbc;
- Set tableIdSet = new HashSet<>();
- tableIdSet.add(tableId);
-
- final Map tableChangeMap = new HashMap<>();
- Tables tables = new Tables();
- tables.overwriteTable(tables.editOrCreateTable(tableId).create());
-
- try {
- connection.readSchemaForCapturedTables(
- tables, tableId.catalog(), tableId.schema(), null, false, tableIdSet);
- Table table = tables.forTable(tableId);
- TableChanges.TableChange tableChange =
- new TableChanges.TableChange(TableChanges.TableChangeType.CREATE, table);
- tableChangeMap.put(tableId, tableChange);
- } catch (SQLException e) {
- throw new FlinkRuntimeException(
- String.format("Failed to read schema for table %s ", tableId), e);
- }
-
- if (!tableChangeMap.containsKey(tableId)) {
- throw new FlinkRuntimeException(
- String.format("Can't obtain schema for table %s ", tableId));
- }
-
- return tableChangeMap.get(tableId);
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java
new file mode 100644
index 00000000000..279eb014bbc
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySqlTableSourceFactory.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.oceanbase.table;
+
+import org.apache.flink.cdc.connectors.mysql.table.MySqlTableSourceFactory;
+
+/**
+ * Factory class for creating table sources that capture data changes from OceanBase databases.
+ *
+ * This factory extends {@link MySqlTableSourceFactory} to reuse MySQL CDC connector.
+ *
+ *
Key characteristics:
+ *
+ *
+ * - Identified by factory identifier "oceanbase-cdc"
+ *
- Compatible with OceanBase Binlog Service
+ *
+ *
+ * @see org.apache.flink.cdc.connectors.mysql.table.MySqlTableSourceFactory Base MySQL
+ * implementation
+ */
+public class OceanBaseMySqlTableSourceFactory extends MySqlTableSourceFactory {
+
+ private static final String IDENTIFIER = "oceanbase-cdc";
+
+ @Override
+ public String factoryIdentifier() {
+ return IDENTIFIER;
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java
deleted file mode 100644
index 12a6282f4fe..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseReadableMetadata.java
+++ /dev/null
@@ -1,140 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.cdc.connectors.oceanbase.source.offset.OceanBaseSourceInfo;
-import org.apache.flink.cdc.debezium.table.MetadataConverter;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.data.StringData;
-import org.apache.flink.table.data.TimestampData;
-import org.apache.flink.table.types.DataType;
-
-import io.debezium.data.Envelope;
-import org.apache.kafka.connect.data.Struct;
-import org.apache.kafka.connect.source.SourceRecord;
-
-/** Defines the supported metadata columns for {@link OceanBaseTableSource}. */
-public enum OceanBaseReadableMetadata {
-
- /** Name of the tenant that contains the row. */
- TENANT(
- "tenant_name",
- DataTypes.STRING().nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(SourceRecord record) {
- Struct messageStruct = (Struct) record.value();
- Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
- return StringData.fromString(
- sourceStruct.getString(OceanBaseSourceInfo.TENANT_KEY));
- }
- }),
-
- /** Name of the database that contains the row. */
- DATABASE(
- "database_name",
- DataTypes.STRING().nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(SourceRecord record) {
- Struct messageStruct = (Struct) record.value();
- Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
- return StringData.fromString(
- sourceStruct.getString(OceanBaseSourceInfo.DATABASE_NAME_KEY));
- }
- }),
-
- /** Name of the schema that contains the row. */
- SCHEMA(
- "schema_name",
- DataTypes.STRING().nullable(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(SourceRecord record) {
- Struct messageStruct = (Struct) record.value();
- Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
- return StringData.fromString(
- sourceStruct.getString(OceanBaseSourceInfo.SCHEMA_NAME_KEY));
- }
- }),
-
- /** Name of the table that contains the row. */
- TABLE(
- "table_name",
- DataTypes.STRING().notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(SourceRecord record) {
- Struct messageStruct = (Struct) record.value();
- Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
- return StringData.fromString(
- sourceStruct.getString(OceanBaseSourceInfo.TABLE_NAME_KEY));
- }
- }),
-
- /**
- * It indicates the time that the change was made in the database. If the record is read from
- * snapshot of the table instead of the change stream, the value is always 0.
- */
- OP_TS(
- "op_ts",
- DataTypes.TIMESTAMP_LTZ(3).notNull(),
- new MetadataConverter() {
- private static final long serialVersionUID = 1L;
-
- @Override
- public Object read(SourceRecord record) {
- Struct messageStruct = (Struct) record.value();
- Struct sourceStruct = messageStruct.getStruct(Envelope.FieldName.SOURCE);
- return TimestampData.fromEpochMillis(
- sourceStruct.getInt64(OceanBaseSourceInfo.TIMESTAMP_KEY));
- }
- });
-
- private final String key;
-
- private final DataType dataType;
-
- private final MetadataConverter converter;
-
- OceanBaseReadableMetadata(String key, DataType dataType, MetadataConverter converter) {
- this.key = key;
- this.dataType = dataType;
- this.converter = converter;
- }
-
- public String getKey() {
- return key;
- }
-
- public DataType getDataType() {
- return dataType;
- }
-
- public MetadataConverter getConverter() {
- return converter;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java
deleted file mode 100644
index e3084b5fb83..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSource.java
+++ /dev/null
@@ -1,333 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.api.common.typeinfo.TypeInformation;
-import org.apache.flink.cdc.connectors.base.options.StartupOptions;
-import org.apache.flink.cdc.connectors.oceanbase.OceanBaseSource;
-import org.apache.flink.cdc.connectors.oceanbase.source.converter.OceanBaseDeserializationConverterFactory;
-import org.apache.flink.cdc.debezium.DebeziumDeserializationSchema;
-import org.apache.flink.cdc.debezium.table.MetadataConverter;
-import org.apache.flink.cdc.debezium.table.RowDataDebeziumDeserializeSchema;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.ChangelogMode;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.connector.source.ScanTableSource;
-import org.apache.flink.table.connector.source.SourceFunctionProvider;
-import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.types.DataType;
-import org.apache.flink.table.types.logical.RowType;
-
-import java.time.Duration;
-import java.time.ZoneId;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.flink.util.Preconditions.checkNotNull;
-
-/** A {@link DynamicTableSource} implementation for OceanBase. */
-public class OceanBaseTableSource implements ScanTableSource, SupportsReadingMetadata {
-
- private final ResolvedSchema physicalSchema;
-
- private final StartupOptions startupOptions;
- private final String username;
- private final String password;
- private final String tenantName;
- private final String databaseName;
- private final String tableName;
- private final String tableList;
- private final Duration connectTimeout;
- private final String serverTimeZone;
-
- private final String hostname;
- private final int port;
- private final String compatibleMode;
- private final String jdbcDriver;
- private final Properties jdbcProperties;
-
- private final String logProxyHost;
- private final Integer logProxyPort;
- private final String logProxyClientId;
- private final Long startupTimestamp;
- private final String rsList;
- private final String configUrl;
- private final String workingMode;
- private final Properties obcdcProperties;
- private final Properties debeziumProperties;
-
- // --------------------------------------------------------------------------------------------
- // Mutable attributes
- // --------------------------------------------------------------------------------------------
-
- /** Data type that describes the final output of the source. */
- protected DataType producedDataType;
-
- /** Metadata that is appended at the end of a physical source row. */
- protected List metadataKeys;
-
- public OceanBaseTableSource(
- ResolvedSchema physicalSchema,
- StartupOptions startupOptions,
- String username,
- String password,
- String tenantName,
- String databaseName,
- String tableName,
- String tableList,
- String serverTimeZone,
- Duration connectTimeout,
- String hostname,
- int port,
- String compatibleMode,
- String jdbcDriver,
- Properties jdbcProperties,
- String logProxyHost,
- Integer logProxyPort,
- String logProxyClientId,
- Long startupTimestamp,
- String rsList,
- String configUrl,
- String workingMode,
- Properties obcdcProperties,
- Properties debeziumProperties) {
- this.physicalSchema = physicalSchema;
- this.startupOptions = checkNotNull(startupOptions);
- this.username = checkNotNull(username);
- this.password = checkNotNull(password);
- this.tenantName = tenantName;
- this.databaseName = databaseName;
- this.tableName = tableName;
- this.tableList = tableList;
- this.serverTimeZone = serverTimeZone;
- this.connectTimeout = connectTimeout;
- this.hostname = checkNotNull(hostname);
- this.port = port;
- this.compatibleMode = compatibleMode;
- this.jdbcDriver = jdbcDriver;
- this.jdbcProperties = jdbcProperties;
- this.logProxyHost = logProxyHost;
- this.logProxyPort = logProxyPort;
- this.logProxyClientId = logProxyClientId;
- this.startupTimestamp = startupTimestamp;
- this.rsList = rsList;
- this.configUrl = configUrl;
- this.workingMode = workingMode;
- this.obcdcProperties = obcdcProperties;
- this.debeziumProperties = debeziumProperties;
-
- this.producedDataType = physicalSchema.toPhysicalRowDataType();
- this.metadataKeys = Collections.emptyList();
- }
-
- @Override
- public ChangelogMode getChangelogMode() {
- return ChangelogMode.all();
- }
-
- @Override
- public ScanRuntimeProvider getScanRuntimeProvider(ScanContext context) {
- RowType physicalDataType =
- (RowType) physicalSchema.toPhysicalRowDataType().getLogicalType();
- MetadataConverter[] metadataConverters = getMetadataConverters();
- TypeInformation resultTypeInfo = context.createTypeInformation(producedDataType);
-
- DebeziumDeserializationSchema deserializer =
- RowDataDebeziumDeserializeSchema.newBuilder()
- .setPhysicalRowType(physicalDataType)
- .setMetadataConverters(metadataConverters)
- .setResultTypeInfo(resultTypeInfo)
- .setServerTimeZone(
- serverTimeZone == null
- ? ZoneId.systemDefault()
- : ZoneId.of(serverTimeZone))
- .setUserDefinedConverterFactory(
- OceanBaseDeserializationConverterFactory.instance())
- .build();
-
- OceanBaseSource.Builder builder =
- OceanBaseSource.builder()
- .startupOptions(startupOptions)
- .username(username)
- .password(password)
- .tenantName(tenantName)
- .databaseName(databaseName)
- .tableName(tableName)
- .tableList(tableList)
- .serverTimeZone(serverTimeZone)
- .connectTimeout(connectTimeout)
- .hostname(hostname)
- .port(port)
- .compatibleMode(compatibleMode)
- .jdbcDriver(jdbcDriver)
- .jdbcProperties(jdbcProperties)
- .logProxyHost(logProxyHost)
- .logProxyPort(logProxyPort)
- .logProxyClientId(logProxyClientId)
- .startupTimestamp(startupTimestamp)
- .rsList(rsList)
- .configUrl(configUrl)
- .workingMode(workingMode)
- .obcdcProperties(obcdcProperties)
- .debeziumProperties(debeziumProperties)
- .deserializer(deserializer);
- return SourceFunctionProvider.of(builder.build(), false);
- }
-
- protected MetadataConverter[] getMetadataConverters() {
- if (metadataKeys.isEmpty()) {
- return new MetadataConverter[0];
- }
- return metadataKeys.stream()
- .map(
- key ->
- Stream.of(OceanBaseReadableMetadata.values())
- .filter(m -> m.getKey().equals(key))
- .findFirst()
- .orElseThrow(IllegalStateException::new))
- .map(OceanBaseReadableMetadata::getConverter)
- .toArray(MetadataConverter[]::new);
- }
-
- @Override
- public Map listReadableMetadata() {
- return Stream.of(OceanBaseReadableMetadata.values())
- .collect(
- Collectors.toMap(
- OceanBaseReadableMetadata::getKey,
- OceanBaseReadableMetadata::getDataType));
- }
-
- @Override
- public void applyReadableMetadata(List metadataKeys, DataType producedDataType) {
- this.metadataKeys = metadataKeys;
- this.producedDataType = producedDataType;
- }
-
- @Override
- public DynamicTableSource copy() {
- OceanBaseTableSource source =
- new OceanBaseTableSource(
- physicalSchema,
- startupOptions,
- username,
- password,
- tenantName,
- databaseName,
- tableName,
- tableList,
- serverTimeZone,
- connectTimeout,
- hostname,
- port,
- compatibleMode,
- jdbcDriver,
- jdbcProperties,
- logProxyHost,
- logProxyPort,
- logProxyClientId,
- startupTimestamp,
- rsList,
- configUrl,
- workingMode,
- obcdcProperties,
- debeziumProperties);
- source.metadataKeys = metadataKeys;
- source.producedDataType = producedDataType;
- return source;
- }
-
- @Override
- public boolean equals(Object o) {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
- OceanBaseTableSource that = (OceanBaseTableSource) o;
- return Objects.equals(this.physicalSchema, that.physicalSchema)
- && Objects.equals(this.startupOptions, that.startupOptions)
- && Objects.equals(this.username, that.username)
- && Objects.equals(this.password, that.password)
- && Objects.equals(this.tenantName, that.tenantName)
- && Objects.equals(this.databaseName, that.databaseName)
- && Objects.equals(this.tableName, that.tableName)
- && Objects.equals(this.tableList, that.tableList)
- && Objects.equals(this.serverTimeZone, that.serverTimeZone)
- && Objects.equals(this.connectTimeout, that.connectTimeout)
- && Objects.equals(this.hostname, that.hostname)
- && Objects.equals(this.port, that.port)
- && Objects.equals(this.compatibleMode, that.compatibleMode)
- && Objects.equals(this.jdbcDriver, that.jdbcDriver)
- && Objects.equals(this.jdbcProperties, that.jdbcProperties)
- && Objects.equals(this.logProxyHost, that.logProxyHost)
- && Objects.equals(this.logProxyPort, that.logProxyPort)
- && Objects.equals(this.logProxyClientId, that.logProxyClientId)
- && Objects.equals(this.startupTimestamp, that.startupTimestamp)
- && Objects.equals(this.rsList, that.rsList)
- && Objects.equals(this.configUrl, that.configUrl)
- && Objects.equals(this.workingMode, that.workingMode)
- && Objects.equals(this.obcdcProperties, that.obcdcProperties)
- && Objects.equals(this.debeziumProperties, that.debeziumProperties)
- && Objects.equals(this.producedDataType, that.producedDataType)
- && Objects.equals(this.metadataKeys, that.metadataKeys);
- }
-
- @Override
- public int hashCode() {
- return Objects.hash(
- physicalSchema,
- startupOptions,
- username,
- password,
- tenantName,
- databaseName,
- tableName,
- tableList,
- serverTimeZone,
- connectTimeout,
- hostname,
- port,
- compatibleMode,
- jdbcDriver,
- jdbcProperties,
- logProxyHost,
- logProxyPort,
- logProxyClientId,
- startupTimestamp,
- rsList,
- configUrl,
- workingMode,
- obcdcProperties,
- debeziumProperties,
- producedDataType,
- metadataKeys);
- }
-
- @Override
- public String asSummaryString() {
- return "OceanBase-CDC";
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java
deleted file mode 100644
index a236c0eb667..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableSourceFactory.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.cdc.connectors.base.options.StartupOptions;
-import org.apache.flink.cdc.connectors.oceanbase.utils.OceanBaseUtils;
-import org.apache.flink.cdc.connectors.oceanbase.utils.OptionUtils;
-import org.apache.flink.cdc.debezium.table.DebeziumOptions;
-import org.apache.flink.cdc.debezium.utils.JdbcUrlUtils;
-import org.apache.flink.configuration.ConfigOption;
-import org.apache.flink.configuration.ConfigOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.configuration.ReadableConfig;
-import org.apache.flink.table.api.ValidationException;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.factories.DynamicTableSourceFactory;
-import org.apache.flink.table.factories.FactoryUtil;
-
-import java.time.Duration;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Properties;
-import java.util.Set;
-
-import static org.apache.flink.cdc.connectors.base.options.SourceOptions.SCAN_STARTUP_MODE;
-
-/** Factory for creating configured instance of {@link OceanBaseTableSource}. */
-public class OceanBaseTableSourceFactory implements DynamicTableSourceFactory {
-
- private static final String IDENTIFIER = "oceanbase-cdc";
-
- public static final ConfigOption USERNAME =
- ConfigOptions.key("username")
- .stringType()
- .noDefaultValue()
- .withDescription("Username to be used when connecting to OceanBase.");
-
- public static final ConfigOption PASSWORD =
- ConfigOptions.key("password")
- .stringType()
- .noDefaultValue()
- .withDescription("Password to be used when connecting to OceanBase.");
-
- public static final ConfigOption TENANT_NAME =
- ConfigOptions.key("tenant-name")
- .stringType()
- .noDefaultValue()
- .withDescription("Tenant name of OceanBase to monitor.");
-
- public static final ConfigOption DATABASE_NAME =
- ConfigOptions.key("database-name")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Database name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode.");
-
- public static final ConfigOption TABLE_NAME =
- ConfigOptions.key("table-name")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Table name of OceanBase to monitor, should be regular expression. Only can be used with 'initial' mode.");
-
- public static final ConfigOption TABLE_LIST =
- ConfigOptions.key("table-list")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "List of full names of tables, separated by commas, e.g. \"db1.table1, db2.table2\".");
-
- public static final ConfigOption SERVER_TIME_ZONE =
- ConfigOptions.key("server-time-zone")
- .stringType()
- .defaultValue("+00:00")
- .withDescription("The session time zone in database server.");
-
- public static final ConfigOption CONNECT_TIMEOUT =
- ConfigOptions.key("connect.timeout")
- .durationType()
- .defaultValue(Duration.ofSeconds(30))
- .withDescription(
- "The maximum time that the connector should wait after trying to connect to the database server or log proxy server before timing out.");
-
- public static final ConfigOption HOSTNAME =
- ConfigOptions.key("hostname")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "IP address or hostname of the OceanBase database server or OceanBase proxy server.");
-
- public static final ConfigOption PORT =
- ConfigOptions.key("port")
- .intType()
- .noDefaultValue()
- .withDescription(
- "Integer port number of OceanBase database server or OceanBase proxy server.");
-
- public static final ConfigOption COMPATIBLE_MODE =
- ConfigOptions.key("compatible-mode")
- .stringType()
- .defaultValue("mysql")
- .withDescription(
- "The compatible mode of OceanBase, can be 'mysql' or 'oracle'.");
-
- public static final ConfigOption JDBC_DRIVER =
- ConfigOptions.key("jdbc.driver")
- .stringType()
- .defaultValue("com.mysql.cj.jdbc.Driver")
- .withDescription(
- "JDBC driver class name, use 'com.mysql.cj.jdbc.Driver' by default.");
-
- public static final ConfigOption LOG_PROXY_HOST =
- ConfigOptions.key("logproxy.host")
- .stringType()
- .noDefaultValue()
- .withDescription("Hostname or IP address of OceanBase log proxy service.");
-
- public static final ConfigOption LOG_PROXY_PORT =
- ConfigOptions.key("logproxy.port")
- .intType()
- .noDefaultValue()
- .withDescription("Port number of OceanBase log proxy service.");
-
- public static final ConfigOption LOG_PROXY_CLIENT_ID =
- ConfigOptions.key("logproxy.client.id")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "Id of log proxy client, used to distinguish different connections.");
-
- public static final ConfigOption SCAN_STARTUP_TIMESTAMP =
- ConfigOptions.key("scan.startup.timestamp")
- .longType()
- .noDefaultValue()
- .withDescription(
- "Optional timestamp in seconds used in case of \"timestamp\" startup mode.");
-
- public static final ConfigOption RS_LIST =
- ConfigOptions.key("rootserver-list")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "The semicolon-separated list of root servers in format `ip:rpc_port:sql_port`, corresponding to the parameter 'rootservice_list' in the database.");
-
- public static final ConfigOption CONFIG_URL =
- ConfigOptions.key("config-url")
- .stringType()
- .noDefaultValue()
- .withDescription(
- "The url used to get root servers list, corresponding to the parameter 'obconfig_url' in the database.");
-
- public static final ConfigOption WORKING_MODE =
- ConfigOptions.key("working-mode")
- .stringType()
- .defaultValue("storage")
- .withDescription(
- "The working mode of 'obcdc', can be `storage` (default value, supported from `obcdc` 3.1.3) or `memory`.");
-
- public static final String OBCDC_PROPERTIES_PREFIX = "obcdc.properties.";
-
- @Override
- public DynamicTableSource createDynamicTableSource(Context context) {
- final FactoryUtil.TableFactoryHelper helper =
- FactoryUtil.createTableFactoryHelper(this, context);
- helper.validateExcept(
- JdbcUrlUtils.PROPERTIES_PREFIX,
- OBCDC_PROPERTIES_PREFIX,
- DebeziumOptions.DEBEZIUM_OPTIONS_PREFIX);
-
- ResolvedSchema physicalSchema = context.getCatalogTable().getResolvedSchema();
-
- ReadableConfig config = helper.getOptions();
-
- StartupOptions startupOptions = getStartupOptions(config);
-
- String username = config.get(USERNAME);
- String password = config.get(PASSWORD);
- String tenantName = config.get(TENANT_NAME);
- String databaseName = config.get(DATABASE_NAME);
- String tableName = config.get(TABLE_NAME);
- String tableList = config.get(TABLE_LIST);
-
- String serverTimeZone = config.get(SERVER_TIME_ZONE);
- Duration connectTimeout = config.get(CONNECT_TIMEOUT);
-
- String hostname = config.get(HOSTNAME);
- Integer port = config.get(PORT);
- String compatibleMode = config.get(COMPATIBLE_MODE);
- String jdbcDriver = config.get(JDBC_DRIVER);
-
- validateJdbcDriver(compatibleMode, jdbcDriver);
-
- String logProxyHost = config.get(LOG_PROXY_HOST);
- Integer logProxyPort = config.get(LOG_PROXY_PORT);
- String logProxyClientId = config.get(LOG_PROXY_CLIENT_ID);
- Long startupTimestamp = config.get(SCAN_STARTUP_TIMESTAMP);
- String rsList = config.get(RS_LIST);
- String configUrl = config.get(CONFIG_URL);
- String workingMode = config.get(WORKING_MODE);
-
- OptionUtils.printOptions(IDENTIFIER, ((Configuration) config).toMap());
-
- return new OceanBaseTableSource(
- physicalSchema,
- startupOptions,
- username,
- password,
- tenantName,
- databaseName,
- tableName,
- tableList,
- serverTimeZone,
- connectTimeout,
- hostname,
- port,
- compatibleMode,
- jdbcDriver,
- JdbcUrlUtils.getJdbcProperties(context.getCatalogTable().getOptions()),
- logProxyHost,
- logProxyPort,
- logProxyClientId,
- startupTimestamp,
- rsList,
- configUrl,
- workingMode,
- getProperties(context.getCatalogTable().getOptions(), OBCDC_PROPERTIES_PREFIX),
- DebeziumOptions.getDebeziumProperties(context.getCatalogTable().getOptions()));
- }
-
- @Override
- public String factoryIdentifier() {
- return IDENTIFIER;
- }
-
- @Override
- public Set> requiredOptions() {
- Set> options = new HashSet<>();
- options.add(USERNAME);
- options.add(PASSWORD);
- options.add(HOSTNAME);
- options.add(PORT);
- return options;
- }
-
- @Override
- public Set> optionalOptions() {
- Set> options = new HashSet<>();
- options.add(SCAN_STARTUP_MODE);
- options.add(SCAN_STARTUP_TIMESTAMP);
- options.add(DATABASE_NAME);
- options.add(TABLE_NAME);
- options.add(TABLE_LIST);
- options.add(COMPATIBLE_MODE);
- options.add(JDBC_DRIVER);
- options.add(CONNECT_TIMEOUT);
- options.add(SERVER_TIME_ZONE);
- options.add(TENANT_NAME);
- options.add(LOG_PROXY_HOST);
- options.add(LOG_PROXY_PORT);
- options.add(LOG_PROXY_CLIENT_ID);
- options.add(RS_LIST);
- options.add(CONFIG_URL);
- options.add(WORKING_MODE);
- return options;
- }
-
- private static final String SCAN_STARTUP_MODE_VALUE_INITIAL = "initial";
- private static final String SCAN_STARTUP_MODE_VALUE_SNAPSHOT = "snapshot";
- private static final String SCAN_STARTUP_MODE_VALUE_LATEST = "latest-offset";
- private static final String SCAN_STARTUP_MODE_VALUE_TIMESTAMP = "timestamp";
-
- private static StartupOptions getStartupOptions(ReadableConfig config) {
- String modeString = config.get(SCAN_STARTUP_MODE);
-
- switch (modeString.toLowerCase()) {
- case SCAN_STARTUP_MODE_VALUE_INITIAL:
- return StartupOptions.initial();
- case SCAN_STARTUP_MODE_VALUE_SNAPSHOT:
- return StartupOptions.snapshot();
- case SCAN_STARTUP_MODE_VALUE_LATEST:
- return StartupOptions.latest();
- case SCAN_STARTUP_MODE_VALUE_TIMESTAMP:
- if (config.get(SCAN_STARTUP_TIMESTAMP) != null) {
- return StartupOptions.timestamp(config.get(SCAN_STARTUP_TIMESTAMP) * 1000);
- }
- throw new ValidationException(
- String.format(
- "Option '%s' should not be empty", SCAN_STARTUP_TIMESTAMP.key()));
-
- default:
- throw new ValidationException(
- String.format(
- "Invalid value for option '%s'. Supported values are [%s, %s, %s, %s], but was: %s",
- SCAN_STARTUP_MODE.key(),
- SCAN_STARTUP_MODE_VALUE_INITIAL,
- SCAN_STARTUP_MODE_VALUE_SNAPSHOT,
- SCAN_STARTUP_MODE_VALUE_LATEST,
- SCAN_STARTUP_MODE_VALUE_TIMESTAMP,
- modeString));
- }
- }
-
- private void validateJdbcDriver(String compatibleMode, String jdbcDriver) {
- Objects.requireNonNull(compatibleMode, "'compatible-mode' is required.");
- Objects.requireNonNull(jdbcDriver, "'jdbc.driver' is required.");
- if ("oracle".equalsIgnoreCase(compatibleMode)
- && !OceanBaseUtils.isOceanBaseDriver(jdbcDriver)) {
- throw new IllegalArgumentException(
- "OceanBase JDBC driver is required for OceanBase Oracle mode.");
- }
- try {
- Class.forName(jdbcDriver);
- } catch (ClassNotFoundException e) {
- throw new RuntimeException("Jdbc driver class not found", e);
- }
- }
-
- private Properties getProperties(Map tableOptions, String prefix) {
- Properties properties = new Properties();
- tableOptions.keySet().stream()
- .filter(key -> key.startsWith(prefix))
- .forEach(
- key -> {
- final String value = tableOptions.get(key);
- final String subKey = key.substring(prefix.length());
- properties.put(subKey, value);
- });
- return properties;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java
deleted file mode 100644
index 7cb738b68b7..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OceanBaseUtils.java
+++ /dev/null
@@ -1,27 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.utils;
-
-/** Utils for OceanBase. */
-public class OceanBaseUtils {
-
- public static boolean isOceanBaseDriver(String driverClass) {
- return "com.oceanbase.jdbc.Driver".equals(driverClass)
- || "com.alipay.oceanbase.jdbc.Driver".equals(driverClass);
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java
deleted file mode 100644
index a71dd26a627..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/java/org/apache/flink/cdc/connectors/oceanbase/utils/OptionUtils.java
+++ /dev/null
@@ -1,42 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.utils;
-
-import org.apache.flink.configuration.ConfigurationUtils;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Map;
-
-/** A utility class to print configuration of connectors. */
-public class OptionUtils {
-
- private static final Logger LOG = LoggerFactory.getLogger(OptionUtils.class);
-
- /** Utility class can not be instantiated. */
- private OptionUtils() {}
-
- public static void printOptions(String identifier, Map config) {
- Map hideMap = ConfigurationUtils.hideSensitiveValues(config);
- LOG.info("Print {} connector configuration:", identifier);
- for (String key : hideMap.keySet()) {
- LOG.info("{} = {}", key, hideMap.get(key));
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
index 9eadc06c76a..abb9b5eaa86 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory
@@ -13,4 +13,4 @@
# See the License for the specific language governing permissions and
# limitations under the License.
-org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseTableSourceFactory
+org.apache.flink.cdc.connectors.oceanbase.table.OceanBaseMySqlTableSourceFactory
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java
new file mode 100644
index 00000000000..5e1e979891e
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseCharsetITCase.java
@@ -0,0 +1,227 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.oceanbase;
+
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.EnvironmentSettings;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.StringUtils;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.stream.Stream;
+
+/** Test supporting different column charsets for OceanBase. */
+@Disabled(
+ "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.")
+public class OceanBaseCharsetITCase extends OceanBaseSourceTestBase {
+
+ private static final String DDL_FILE = "charset_test";
+ private static final String DATABASE_NAME = "cdc_c_" + getRandomSuffix();
+
+ private final StreamExecutionEnvironment env =
+ StreamExecutionEnvironment.getExecutionEnvironment();
+
+ private final StreamTableEnvironment tEnv =
+ StreamTableEnvironment.create(
+ env, EnvironmentSettings.newInstance().inStreamingMode().build());
+
+ @BeforeAll
+ public static void beforeClass() throws InterruptedException {
+ initializeOceanBaseTables(
+ DDL_FILE,
+ DATABASE_NAME,
+ s -> // see:
+ // https://www.oceanbase.com/docs/common-oceanbase-database-cn-1000000002017544
+ !StringUtils.isNullOrWhitespaceOnly(s)
+ && (s.contains("utf8_test")
+ || s.contains("latin1_test")
+ || s.contains("gbk_test")
+ || s.contains("big5_test")
+ || s.contains("ascii_test")
+ || s.contains("sjis_test")));
+ }
+
+ @AfterAll
+ public static void after() {
+ dropDatabase(DATABASE_NAME);
+ }
+
+ @BeforeEach
+ public void before() {
+ TestValuesTableFactory.clearAllData();
+ env.setParallelism(4);
+ env.enableCheckpointing(200);
+ }
+
+ public static Stream parameters() {
+ return Stream.of(
+ Arguments.of(
+ "utf8_test",
+ new String[] {"+I[1, 测试数据]", "+I[2, Craig Marshall]", "+I[3, 另一个测试数据]"},
+ new String[] {
+ "-D[1, 测试数据]",
+ "-D[2, Craig Marshall]",
+ "-D[3, 另一个测试数据]",
+ "+I[11, 测试数据]",
+ "+I[12, Craig Marshall]",
+ "+I[13, 另一个测试数据]"
+ }),
+ Arguments.of(
+ "ascii_test",
+ new String[] {
+ "+I[1, ascii test!?]", "+I[2, Craig Marshall]", "+I[3, {test}]"
+ },
+ new String[] {
+ "-D[1, ascii test!?]",
+ "-D[2, Craig Marshall]",
+ "-D[3, {test}]",
+ "+I[11, ascii test!?]",
+ "+I[12, Craig Marshall]",
+ "+I[13, {test}]"
+ }),
+ Arguments.of(
+ "gbk_test",
+ new String[] {"+I[1, 测试数据]", "+I[2, Craig Marshall]", "+I[3, 另一个测试数据]"},
+ new String[] {
+ "-D[1, 测试数据]",
+ "-D[2, Craig Marshall]",
+ "-D[3, 另一个测试数据]",
+ "+I[11, 测试数据]",
+ "+I[12, Craig Marshall]",
+ "+I[13, 另一个测试数据]"
+ }),
+ Arguments.of(
+ "latin1_test",
+ new String[] {"+I[1, ÀÆÉ]", "+I[2, Craig Marshall]", "+I[3, Üæû]"},
+ new String[] {
+ "-D[1, ÀÆÉ]",
+ "-D[2, Craig Marshall]",
+ "-D[3, Üæû]",
+ "+I[11, ÀÆÉ]",
+ "+I[12, Craig Marshall]",
+ "+I[13, Üæû]"
+ }),
+ Arguments.of(
+ "big5_test",
+ new String[] {"+I[1, 大五]", "+I[2, Craig Marshall]", "+I[3, 丹店]"},
+ new String[] {
+ "-D[1, 大五]",
+ "-D[2, Craig Marshall]",
+ "-D[3, 丹店]",
+ "+I[11, 大五]",
+ "+I[12, Craig Marshall]",
+ "+I[13, 丹店]"
+ }),
+ Arguments.of(
+ "sjis_test",
+ new String[] {"+I[1, ひびぴ]", "+I[2, Craig Marshall]", "+I[3, フブプ]"},
+ new String[] {
+ "-D[1, ひびぴ]",
+ "-D[2, Craig Marshall]",
+ "-D[3, フブプ]",
+ "+I[11, ひびぴ]",
+ "+I[12, Craig Marshall]",
+ "+I[13, フブプ]"
+ }));
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testCharset(String testName, String[] snapshotExpected, String[] binlogExpected)
+ throws Exception {
+ String sourceDDL =
+ String.format(
+ "CREATE TABLE %s (\n"
+ + " table_id BIGINT,\n"
+ + " table_name STRING,\n"
+ + " primary key(table_id) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'oceanbase-cdc',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'scan.incremental.snapshot.enabled' = '%s',"
+ + " 'server-id' = '%s',"
+ + " 'server-time-zone' = 'Asia/Shanghai',"
+ + " 'jdbc.properties.connectTimeout' = '6000000000',"
+ + " 'jdbc.properties.socketTimeout' = '6000000000',"
+ + " 'jdbc.properties.autoReconnect' = 'true',"
+ + " 'jdbc.properties.failOverReadOnly' = 'false',"
+ + " 'scan.incremental.snapshot.chunk.size' = '%s'"
+ + ")",
+ testName,
+ getHost(),
+ PORT,
+ USER_NAME,
+ PASSWORD,
+ DATABASE_NAME,
+ testName,
+ true,
+ getServerId(),
+ 4);
+ tEnv.executeSql(sourceDDL);
+ // async submit job
+ TableResult result =
+ tEnv.executeSql(String.format("SELECT table_id,table_name FROM %s", testName));
+
+ // test snapshot phase
+ CloseableIterator iterator = result.collect();
+ waitForSnapshotStarted(iterator);
+ assertEqualsInAnyOrder(
+ Arrays.asList(snapshotExpected), fetchRows(iterator, snapshotExpected.length));
+
+ // test binlog phase
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute(
+ String.format(
+ "/*TDDL:FORBID_EXECUTE_DML_ALL=FALSE*/UPDATE %s.%s SET table_id = table_id + 10;",
+ DATABASE_NAME, testName));
+ }
+ assertEqualsInAnyOrder(
+ Arrays.asList(binlogExpected), fetchRows(iterator, binlogExpected.length));
+ result.getJobClient().get().cancel().get();
+
+ // Sleep to avoid the issue: The last packet successfully received from the server was 35
+ // milliseconds ago.
+ Thread.sleep(1_000);
+ }
+
+ private static void waitForSnapshotStarted(CloseableIterator iterator) throws Exception {
+ while (!iterator.hasNext()) {
+ Thread.sleep(100);
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java
new file mode 100644
index 00000000000..aaad93e1ca9
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseFailoverITCase.java
@@ -0,0 +1,521 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.oceanbase;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.cdc.connectors.utils.ExternalResourceProxy;
+import org.apache.flink.runtime.minicluster.RpcServiceSharing;
+import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration;
+import org.apache.flink.streaming.api.CheckpointingMode;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.test.util.MiniClusterWithClientResource;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+import org.apache.flink.util.StringUtils;
+
+import io.debezium.jdbc.JdbcConnection;
+import org.assertj.core.api.Assertions;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Timeout;
+import org.junit.jupiter.api.extension.RegisterExtension;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.Arguments;
+import org.junit.jupiter.params.provider.MethodSource;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.FutureTask;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Stream;
+
+import static java.lang.String.format;
+import static org.apache.flink.api.common.JobStatus.RUNNING;
+
+/** failover IT tests for oceanbase. */
+@Disabled(
+ "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.")
+@Timeout(value = 180, unit = TimeUnit.SECONDS)
+public class OceanBaseFailoverITCase extends OceanBaseSourceTestBase {
+
+ private static final String DEFAULT_SCAN_STARTUP_MODE = "initial";
+ private static final String DDL_FILE = "oceanbase_ddl_test";
+ private static final String DEFAULT_TEST_DATABASE = "customer_" + getRandomSuffix();
+ protected static final int DEFAULT_PARALLELISM = 4;
+
+ private final List firstPartBinlogEvents =
+ Arrays.asList(
+ "-U[103, user_3, Shanghai, 123567891234]",
+ "+U[103, user_3, Hangzhou, 123567891234]",
+ "-D[102, user_2, Shanghai, 123567891234]",
+ "+I[102, user_2, Shanghai, 123567891234]",
+ "-U[103, user_3, Hangzhou, 123567891234]",
+ "+U[103, user_3, Shanghai, 123567891234]");
+
+ private final List secondPartBinlogEvents =
+ Arrays.asList(
+ "-U[1010, user_11, Shanghai, 123567891234]",
+ "+I[2001, user_22, Shanghai, 123567891234]",
+ "+I[2002, user_23, Shanghai, 123567891234]",
+ "+I[2003, user_24, Shanghai, 123567891234]",
+ "+U[1010, user_11, Hangzhou, 123567891234]");
+
+ public static Stream parameters() {
+ return Stream.of(
+ Arguments.of("customers", null),
+ Arguments.of("customers", "id"),
+ Arguments.of("customers_no_pk", "id"));
+ }
+
+ @RegisterExtension
+ public final ExternalResourceProxy miniClusterResource =
+ new ExternalResourceProxy<>(
+ new MiniClusterWithClientResource(
+ new MiniClusterResourceConfiguration.Builder()
+ .setNumberTaskManagers(1)
+ .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM)
+ .setRpcServiceSharing(RpcServiceSharing.DEDICATED)
+ .withHaLeadershipControl()
+ .build()));
+
+ @BeforeEach
+ public void setup() throws InterruptedException {
+ initializeOceanBaseTables(
+ DDL_FILE,
+ DEFAULT_TEST_DATABASE,
+ s -> !StringUtils.isNullOrWhitespaceOnly(s) && (s.contains("customers")));
+ }
+
+ @AfterEach
+ public void clean() {
+ dropDatabase(DEFAULT_TEST_DATABASE);
+ }
+
+ // Failover tests
+ @ParameterizedTest
+ @MethodSource("parameters")
+ @Timeout(value = 120, unit = TimeUnit.SECONDS)
+ public void testTaskManagerFailoverInSnapshotPhase(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ FailoverType.TM,
+ FailoverPhase.SNAPSHOT,
+ new String[] {tableName, "customers_1"},
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testTaskManagerFailoverInBinlogPhase(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ FailoverType.TM,
+ FailoverPhase.BINLOG,
+ new String[] {tableName, "customers_1"},
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testTaskManagerFailoverFromLatestOffset(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ DEFAULT_PARALLELISM,
+ "latest-offset",
+ FailoverType.TM,
+ FailoverPhase.BINLOG,
+ new String[] {tableName, "customers_1"},
+ RestartStrategies.fixedDelayRestart(1, 0),
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testJobManagerFailoverInSnapshotPhase(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ FailoverType.JM,
+ FailoverPhase.SNAPSHOT,
+ new String[] {tableName, "customers_1"},
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testJobManagerFailoverInBinlogPhase(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ FailoverType.JM,
+ FailoverPhase.BINLOG,
+ new String[] {tableName, "customers_1"},
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testJobManagerFailoverFromLatestOffset(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ DEFAULT_PARALLELISM,
+ "latest-offset",
+ FailoverType.JM,
+ FailoverPhase.BINLOG,
+ new String[] {tableName, "customers_1"},
+ RestartStrategies.fixedDelayRestart(1, 0),
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testTaskManagerFailoverSingleParallelism(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ 1,
+ FailoverType.TM,
+ FailoverPhase.SNAPSHOT,
+ new String[] {tableName},
+ tableName,
+ chunkColumnName);
+ }
+
+ @ParameterizedTest
+ @MethodSource("parameters")
+ public void testJobManagerFailoverSingleParallelism(String tableName, String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ 1,
+ FailoverType.JM,
+ FailoverPhase.SNAPSHOT,
+ new String[] {tableName},
+ tableName,
+ chunkColumnName);
+ }
+
+ private void testMySqlParallelSource(
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables,
+ String tableName,
+ String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ DEFAULT_PARALLELISM,
+ failoverType,
+ failoverPhase,
+ captureCustomerTables,
+ tableName,
+ chunkColumnName);
+ }
+
+ private void testMySqlParallelSource(
+ int parallelism,
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables,
+ String tableName,
+ String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ parallelism,
+ DEFAULT_SCAN_STARTUP_MODE,
+ failoverType,
+ failoverPhase,
+ captureCustomerTables,
+ RestartStrategies.fixedDelayRestart(1, 0),
+ tableName,
+ chunkColumnName);
+ }
+
+ private void testMySqlParallelSource(
+ int parallelism,
+ String scanStartupMode,
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables,
+ RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration,
+ String tableName,
+ String chunkColumnName)
+ throws Exception {
+ testMySqlParallelSource(
+ parallelism,
+ scanStartupMode,
+ failoverType,
+ failoverPhase,
+ captureCustomerTables,
+ restartStrategyConfiguration,
+ false,
+ tableName,
+ chunkColumnName);
+ }
+
+ private void testMySqlParallelSource(
+ int parallelism,
+ String scanStartupMode,
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables,
+ RestartStrategies.RestartStrategyConfiguration restartStrategyConfiguration,
+ boolean skipSnapshotBackfill,
+ String tableName,
+ String chunkColumnName)
+ throws Exception {
+ captureCustomerTables = new String[] {tableName};
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ env.enableCheckpointing(3000L, CheckpointingMode.EXACTLY_ONCE);
+ StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+ env.setParallelism(parallelism);
+ env.enableCheckpointing(200L);
+ env.setRestartStrategy(restartStrategyConfiguration);
+ String sourceDDL =
+ format(
+ "CREATE TABLE customers ("
+ + " id BIGINT NOT NULL,"
+ + " name STRING,"
+ + " address STRING,"
+ + " phone_number STRING"
+ + ("customers_no_pk".equals(tableName)
+ ? ""
+ : ", primary key (id) not enforced")
+ + ") WITH ("
+ + " 'connector' = 'oceanbase-cdc',"
+ + " 'scan.incremental.snapshot.enabled' = 'true',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'scan.startup.mode' = '%s',"
+ + " 'scan.incremental.snapshot.chunk.size' = '100',"
+ + " 'scan.incremental.snapshot.backfill.skip' = '%s',"
+ + " 'server-time-zone' = 'Asia/Shanghai',"
+ + " 'server-id' = '%s'"
+ + " %s"
+ + ")",
+ getHost(),
+ getPort(),
+ getUserName(),
+ getPassword(),
+ DEFAULT_TEST_DATABASE,
+ getTableNameRegex(captureCustomerTables),
+ scanStartupMode,
+ skipSnapshotBackfill,
+ getServerId(),
+ chunkColumnName == null
+ ? ""
+ : String.format(
+ ", 'scan.incremental.snapshot.chunk.key-column' = '%s'",
+ chunkColumnName));
+ tEnv.executeSql(sourceDDL);
+ TableResult tableResult = tEnv.executeSql("select * from customers");
+
+ // first step: check the snapshot data
+ if (DEFAULT_SCAN_STARTUP_MODE.equals(scanStartupMode)) {
+ checkSnapshotData(tableResult, failoverType, failoverPhase, captureCustomerTables);
+ }
+
+ // second step: check the binlog data
+ checkBinlogData(tableResult, failoverType, failoverPhase, captureCustomerTables);
+
+ // sleepMs(3000);
+ tableResult.getJobClient().get().cancel().get();
+ }
+
+ private void checkSnapshotData(
+ TableResult tableResult,
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables)
+ throws Exception {
+ String[] snapshotForSingleTable =
+ new String[] {
+ "+I[101, user_1, Shanghai, 123567891234]",
+ "+I[102, user_2, Shanghai, 123567891234]",
+ "+I[103, user_3, Shanghai, 123567891234]",
+ "+I[109, user_4, Shanghai, 123567891234]",
+ "+I[110, user_5, Shanghai, 123567891234]",
+ "+I[111, user_6, Shanghai, 123567891234]",
+ "+I[118, user_7, Shanghai, 123567891234]",
+ "+I[121, user_8, Shanghai, 123567891234]",
+ "+I[123, user_9, Shanghai, 123567891234]",
+ "+I[1009, user_10, Shanghai, 123567891234]",
+ "+I[1010, user_11, Shanghai, 123567891234]",
+ "+I[1011, user_12, Shanghai, 123567891234]",
+ "+I[1012, user_13, Shanghai, 123567891234]",
+ "+I[1013, user_14, Shanghai, 123567891234]",
+ "+I[1014, user_15, Shanghai, 123567891234]",
+ "+I[1015, user_16, Shanghai, 123567891234]",
+ "+I[1016, user_17, Shanghai, 123567891234]",
+ "+I[1017, user_18, Shanghai, 123567891234]",
+ "+I[1018, user_19, Shanghai, 123567891234]",
+ "+I[1019, user_20, Shanghai, 123567891234]",
+ "+I[2000, user_21, Shanghai, 123567891234]"
+ };
+
+ List expectedSnapshotData = new ArrayList<>();
+ for (int i = 0; i < captureCustomerTables.length; i++) {
+ expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
+ }
+
+ CloseableIterator iterator = tableResult.collect();
+ JobID jobId = tableResult.getJobClient().get().getJobID();
+
+ // trigger failover after some snapshot splits read finished
+ if (failoverPhase == FailoverPhase.SNAPSHOT && iterator.hasNext()) {
+ triggerFailover(
+ failoverType,
+ jobId,
+ miniClusterResource.get().getMiniCluster(),
+ () -> sleepMs(100));
+ }
+
+ assertEqualsInAnyOrder(
+ expectedSnapshotData, fetchRows(iterator, expectedSnapshotData.size()));
+ }
+
+ private void checkBinlogData(
+ TableResult tableResult,
+ FailoverType failoverType,
+ FailoverPhase failoverPhase,
+ String[] captureCustomerTables)
+ throws Exception {
+ waitUntilJobRunning(tableResult);
+ CloseableIterator iterator = tableResult.collect();
+ JobID jobId = tableResult.getJobClient().get().getJobID();
+
+ for (String tableId : captureCustomerTables) {
+ makeFirstPartBinlogEvents(getConnection(), DEFAULT_TEST_DATABASE + '.' + tableId);
+ }
+
+ // wait for the binlog reading
+ Thread.sleep(3_000L);
+
+ if (failoverPhase == FailoverPhase.BINLOG) {
+ triggerFailover(
+ failoverType,
+ jobId,
+ miniClusterResource.get().getMiniCluster(),
+ () -> sleepMs(200));
+ waitUntilJobRunning(tableResult);
+ }
+ for (String tableId : captureCustomerTables) {
+ makeSecondPartBinlogEvents(getConnection(), DEFAULT_TEST_DATABASE + '.' + tableId);
+ }
+
+ List expectedBinlogData = new ArrayList<>();
+ for (int i = 0; i < captureCustomerTables.length; i++) {
+ expectedBinlogData.addAll(firstPartBinlogEvents);
+ expectedBinlogData.addAll(secondPartBinlogEvents);
+ }
+ sleepMs(3_000);
+ assertEqualsInAnyOrder(expectedBinlogData, fetchRows(iterator, expectedBinlogData.size()));
+ Assertions.assertThat(hasNextData(iterator)).isFalse();
+ }
+
+ private void waitUntilJobRunning(TableResult tableResult)
+ throws InterruptedException, ExecutionException {
+ do {
+ Thread.sleep(5000L);
+ } while (tableResult.getJobClient().get().getJobStatus().get() != RUNNING);
+ }
+
+ private boolean hasNextData(final CloseableIterator> iterator)
+ throws InterruptedException, ExecutionException {
+ ExecutorService executor = Executors.newSingleThreadExecutor();
+ try {
+ FutureTask future = new FutureTask(iterator::hasNext);
+ executor.execute(future);
+ return future.get(3, TimeUnit.SECONDS);
+ } catch (TimeoutException e) {
+ return false;
+ } finally {
+ executor.shutdown();
+ }
+ }
+
+ /**
+ * Make some changes on the specified customer table. Changelog in string could be accessed by
+ * {@link #firstPartBinlogEvents}.
+ */
+ private void makeFirstPartBinlogEvents(JdbcConnection connection, String tableId)
+ throws SQLException {
+ try {
+ connection.setAutoCommit(false);
+
+ // make binlog events for the first split
+ connection.execute(
+ "UPDATE " + tableId + " SET address = 'Hangzhou' where id = 103",
+ "DELETE FROM " + tableId + " where id = 102",
+ "INSERT INTO " + tableId + " VALUES(102, 'user_2','Shanghai','123567891234')",
+ "UPDATE " + tableId + " SET address = 'Shanghai' where id = 103");
+ connection.commit();
+ } finally {
+ connection.close();
+ }
+ }
+
+ /**
+ * Make some other changes on the specified customer table. Changelog in string could be
+ * accessed by {@link #secondPartBinlogEvents}.
+ */
+ private void makeSecondPartBinlogEvents(JdbcConnection connection, String tableId)
+ throws SQLException {
+ try {
+ connection.setAutoCommit(false);
+
+ // make binlog events for split-1
+ connection.execute("UPDATE " + tableId + " SET address = 'Hangzhou' where id = 1010");
+ connection.commit();
+
+ // make binlog events for the last split
+ connection.execute(
+ "INSERT INTO "
+ + tableId
+ + " VALUES(2001, 'user_22','Shanghai','123567891234'),"
+ + " (2002, 'user_23','Shanghai','123567891234'),"
+ + "(2003, 'user_24','Shanghai','123567891234')");
+ connection.commit();
+ } finally {
+ connection.close();
+ }
+ }
+
+ private void sleepMs(long millis) {
+ try {
+ Thread.sleep(millis);
+ } catch (InterruptedException ignored) {
+ }
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java
new file mode 100644
index 00000000000..75b6726ff09
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceITCase.java
@@ -0,0 +1,376 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.oceanbase;
+
+import org.apache.flink.api.common.restartstrategy.RestartStrategies;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableResult;
+import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.types.Row;
+import org.apache.flink.util.CloseableIterator;
+
+import org.junit.jupiter.api.AfterAll;
+import org.junit.jupiter.api.BeforeAll;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import static java.lang.String.format;
+
+/** OceanBase CDC source connector integration test. */
+@Disabled(
+ "Temporarily disabled for GitHub CI due to unavailability of OceanBase Binlog Service docker image. These tests are currently only supported for local execution.")
+public class OceanBaseSourceITCase extends OceanBaseSourceTestBase {
+ private static final String DDL_FILE = "oceanbase_ddl_test";
+ private static final String DATABASE_NAME = "cdc_s_" + getRandomSuffix();
+
+ @BeforeAll
+ public static void beforeClass() throws InterruptedException {
+ initializeOceanBaseTables(DDL_FILE, DATABASE_NAME, null);
+ }
+
+ @AfterAll
+ public static void afterClass() {
+ dropDatabase(DATABASE_NAME);
+ }
+
+ @Test
+ public void testSingleKey() throws Exception {
+ int parallelism = 1;
+ String[] captureCustomerTables = new String[] {"orders"};
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+ env.setParallelism(parallelism);
+ env.enableCheckpointing(200L);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+ String sourceDDL =
+ format(
+ "CREATE TABLE orders_source ("
+ + " id BIGINT NOT NULL,"
+ + " seller_id STRING,"
+ + " order_id STRING,"
+ + " buyer_id STRING,"
+ + " create_time TIMESTAMP,"
+ + " primary key (id) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'oceanbase-cdc',"
+ + " 'scan.incremental.snapshot.enabled' = 'true',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'scan.incremental.snapshot.chunk.size' = '100',"
+ + " 'server-time-zone' = 'Asia/Shanghai',"
+ + " 'server-id' = '%s'"
+ + ")",
+ getHost(),
+ PORT,
+ USER_NAME,
+ PASSWORD,
+ DATABASE_NAME,
+ getTableNameRegex(captureCustomerTables),
+ getServerId());
+
+ // first step: check the snapshot data
+ String[] snapshotForSingleTable =
+ new String[] {
+ "+I[1, 1001, 1, 102, 2022-01-16T00:00]",
+ "+I[2, 1002, 2, 105, 2022-01-16T00:00]",
+ "+I[3, 1004, 3, 109, 2022-01-16T00:00]",
+ "+I[4, 1002, 2, 106, 2022-01-16T00:00]",
+ "+I[5, 1003, 1, 107, 2022-01-16T00:00]",
+ };
+ tEnv.executeSql(sourceDDL);
+ TableResult tableResult = tEnv.executeSql("select * from orders_source");
+ CloseableIterator iterator = tableResult.collect();
+ List expectedSnapshotData = new ArrayList<>();
+ for (int i = 0; i < captureCustomerTables.length; i++) {
+ expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
+ }
+
+ List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size());
+ assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData);
+
+ // second step: check the sink data
+ tEnv.executeSql(
+ "CREATE TABLE sink ("
+ + " id BIGINT NOT NULL,"
+ + " seller_id STRING,"
+ + " order_id STRING,"
+ + " buyer_id STRING,"
+ + " create_time TIMESTAMP,"
+ + " primary key (id) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'values',"
+ + " 'sink-insert-only' = 'false'"
+ + ")");
+ tableResult = tEnv.executeSql("insert into sink select * from orders_source");
+
+ waitForSinkSize("sink", realSnapshotData.size());
+ assertEqualsInAnyOrder(
+ expectedSnapshotData, TestValuesTableFactory.getRawResultsAsStrings("sink"));
+
+ // third step: check dml events
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("use " + DATABASE_NAME);
+ statement.execute("INSERT INTO orders VALUES (6, 1006,1006, 1006,'2022-01-17');");
+ statement.execute("INSERT INTO orders VALUES (7,1007, 1007,1007, '2022-01-17');");
+ statement.execute("UPDATE orders SET seller_id= 9999, order_id=9999 WHERE id=6;");
+ statement.execute("UPDATE orders SET seller_id= 9999, order_id=9999 WHERE id=7;");
+ statement.execute("DELETE FROM orders WHERE id=7;");
+ }
+
+ String[] expectedBinlog =
+ new String[] {
+ "+I[6, 1006, 1006, 1006, 2022-01-17T00:00]",
+ "+I[7, 1007, 1007, 1007, 2022-01-17T00:00]",
+ "-U[6, 1006, 1006, 1006, 2022-01-17T00:00]",
+ "+U[6, 9999, 9999, 1006, 2022-01-17T00:00]",
+ "-U[7, 1007, 1007, 1007, 2022-01-17T00:00]",
+ "+U[7, 9999, 9999, 1007, 2022-01-17T00:00]",
+ "-D[7, 9999, 9999, 1007, 2022-01-17T00:00]"
+ };
+ List expectedBinlogData = new ArrayList<>();
+ for (int i = 0; i < captureCustomerTables.length; i++) {
+ expectedBinlogData.addAll(Arrays.asList(expectedBinlog));
+ }
+ List realBinlog = fetchRows(iterator, expectedBinlog.length);
+ assertEqualsInOrder(expectedBinlogData, realBinlog);
+ Thread.sleep(3_000);
+ tableResult.getJobClient().get().cancel().get();
+ }
+
+ @Test
+ public void testFullTypesDdl() throws Exception {
+ int parallelism = 1;
+ String[] captureCustomerTables = new String[] {"oceanbase_full_types"};
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+ env.setParallelism(parallelism);
+ env.enableCheckpointing(200L);
+ String sourceDDL =
+ format(
+ "CREATE TABLE oceanbase_full_types (\n"
+ + " `id` INT NOT NULL,\n"
+ + " tiny_c TINYINT,\n"
+ + " tiny_un_c SMALLINT ,\n"
+ + " small_c SMALLINT,\n"
+ + " small_un_c INT,\n"
+ + " medium_c INT,\n"
+ + " medium_un_c INT,\n"
+ + " int_c INT ,\n"
+ + " int_un_c BIGINT,\n"
+ + " int11_c BIGINT,\n"
+ + " big_c BIGINT,\n"
+ + " big_un_c DECIMAL(20, 0),\n"
+ + " varchar_c VARCHAR(255),\n"
+ + " char_c CHAR(3),\n"
+ + " real_c FLOAT,\n"
+ + " float_c FLOAT,\n"
+ + " double_c DOUBLE,\n"
+ + " decimal_c DECIMAL(8, 4),\n"
+ + " numeric_c DECIMAL(6, 0),\n"
+ + " big_decimal_c STRING,\n"
+ + " bit1_c BOOLEAN,\n"
+ + " tiny1_c BOOLEAN,\n"
+ + " boolean_c BOOLEAN,\n"
+ + " date_c DATE,\n"
+ + " time_c TIME(0),\n"
+ + " datetime3_c TIMESTAMP(3),\n"
+ + " datetime6_c TIMESTAMP(6),\n"
+ + " timestamp_c TIMESTAMP(0),\n"
+ + " file_uuid BYTES,\n"
+ + " bit_c BINARY(8),\n"
+ + " text_c STRING,\n"
+ + " tiny_blob_c BYTES,\n"
+ + " blob_c BYTES,\n"
+ + " medium_blob_c BYTES,\n"
+ + " long_blob_c BYTES,\n"
+ + " year_c INT,\n"
+ + " enum_c STRING,\n"
+ + " set_c ARRAY,\n"
+ + " json_c STRING,\n"
+ + " point_c STRING,\n"
+ + " geometry_c STRING,\n"
+ + " linestring_c STRING,\n"
+ + " polygon_c STRING,\n"
+ + " multipoint_c STRING,\n"
+ + " multiline_c STRING,\n"
+ + " multipolygon_c STRING,\n"
+ + " geometrycollection_c STRING,\n"
+ + " primary key (`id`) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'oceanbase-cdc',"
+ + " 'scan.incremental.snapshot.enabled' = 'true',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'scan.incremental.snapshot.chunk.size' = '100',"
+ + " 'server-time-zone' = 'Asia/Shanghai',"
+ + " 'server-id' = '%s'"
+ + ")",
+ getHost(),
+ PORT,
+ USER_NAME,
+ PASSWORD,
+ DATABASE_NAME,
+ getTableNameRegex(captureCustomerTables),
+ getServerId());
+ tEnv.executeSql(sourceDDL);
+
+ TableResult tableResult = tEnv.executeSql("select * from oceanbase_full_types");
+ CloseableIterator iterator = tableResult.collect();
+ List realSnapshotData = fetchRows(iterator, 1);
+ String[] expectedSnapshotData =
+ new String[] {
+ "+I[1, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807, 18446744073709551615, Hello World, abc, 123.102, 123.102, 404.4443, 123.4567, 346, 34567892.1, false, true, true, 2020-07-17, 18:00:22, 2020-07-17T18:00:22.123, 2020-07-17T18:00:22.123456, 2020-07-17T18:00:22, [101, 26, -19, 8, 57, 15, 72, -109, -78, -15, 54, -110, 62, 123, 116, 0], [4, 4, 4, 4, 4, 4, 4, 4], text, [16], [16], [16], [16], 2021, red, [a, b], {\"key1\": \"value1\", \"key2\": \"value2\", \"num1\": 16708304.0, \"num2\": 16708305}, {\"coordinates\":[1,1],\"type\":\"Point\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, {\"coordinates\":[[3,0],[3,3],[3,5]],\"type\":\"LineString\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,1],[2,2],[1,2],[1,1]]],\"type\":\"Polygon\",\"srid\":0}, {\"coordinates\":[[1,1],[2,2]],\"type\":\"MultiPoint\",\"srid\":0}, {\"coordinates\":[[[1,1],[2,2],[3,3]],[[4,4],[5,5]]],\"type\":\"MultiLineString\",\"srid\":0}, {\"coordinates\":[[[[0,0],[10,0],[10,10],[0,10],[0,0]]],[[[5,5],[7,5],[7,7],[5,7],[5,5]]]],\"type\":\"MultiPolygon\",\"srid\":0}, {\"geometries\":[{\"type\":\"Point\",\"coordinates\":[10,10]},{\"type\":\"Point\",\"coordinates\":[30,30]},{\"type\":\"LineString\",\"coordinates\":[[15,15],[20,20]]}],\"type\":\"GeometryCollection\",\"srid\":0}]"
+ };
+ assertEqualsInAnyOrder(Arrays.asList(expectedSnapshotData), realSnapshotData);
+ Thread.sleep(3_000);
+ tableResult.getJobClient().get().cancel().get();
+ }
+
+ @Test
+ public void testMultiKeys() throws Exception {
+ int parallelism = 1;
+ String[] captureCustomerTables = new String[] {"orders_with_multi_pks"};
+
+ StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();
+ StreamTableEnvironment tEnv = StreamTableEnvironment.create(env);
+
+ env.setParallelism(parallelism);
+ env.enableCheckpointing(200L);
+ env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0));
+ String sourceDDL =
+ format(
+ "CREATE TABLE orders_with_multi_pks ("
+ + " id BIGINT NOT NULL,"
+ + " seller_id STRING,"
+ + " order_id STRING,"
+ + " buyer_id STRING,"
+ + " create_time TIMESTAMP,"
+ + " primary key (id,order_id) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'oceanbase-cdc',"
+ + " 'scan.incremental.snapshot.enabled' = 'true',"
+ + " 'hostname' = '%s',"
+ + " 'port' = '%s',"
+ + " 'username' = '%s',"
+ + " 'password' = '%s',"
+ + " 'database-name' = '%s',"
+ + " 'table-name' = '%s',"
+ + " 'scan.incremental.snapshot.chunk.size' = '100',"
+ + " 'server-time-zone' = 'Asia/Shanghai',"
+ + " 'server-id' = '%s'"
+ + ")",
+ getHost(),
+ PORT,
+ USER_NAME,
+ PASSWORD,
+ DATABASE_NAME,
+ getTableNameRegex(captureCustomerTables),
+ getServerId());
+
+ // first step: check the snapshot data
+ String[] snapshotForSingleTable =
+ new String[] {
+ "+I[1, 1001, 1, 102, 2022-01-16T00:00]",
+ "+I[2, 1002, 2, 105, 2022-01-16T00:00]",
+ "+I[3, 1004, 3, 109, 2022-01-16T00:00]",
+ "+I[4, 1002, 2, 106, 2022-01-16T00:00]",
+ "+I[5, 1003, 1, 107, 2022-01-16T00:00]",
+ };
+ tEnv.executeSql(sourceDDL);
+ TableResult tableResult = tEnv.executeSql("select * from orders_with_multi_pks");
+ CloseableIterator iterator = tableResult.collect();
+ List expectedSnapshotData = new ArrayList<>();
+ for (int i = 0; i < captureCustomerTables.length; i++) {
+ expectedSnapshotData.addAll(Arrays.asList(snapshotForSingleTable));
+ }
+
+ List realSnapshotData = fetchRows(iterator, expectedSnapshotData.size());
+ assertEqualsInAnyOrder(expectedSnapshotData, realSnapshotData);
+
+ // second step: check the sink data
+ tEnv.executeSql(
+ "CREATE TABLE multi_key_sink ("
+ + " id BIGINT NOT NULL,"
+ + " seller_id STRING,"
+ + " order_id STRING,"
+ + " buyer_id STRING,"
+ + " create_time TIMESTAMP,"
+ + " primary key (id,order_id) not enforced"
+ + ") WITH ("
+ + " 'connector' = 'values',"
+ + " 'sink-insert-only' = 'false'"
+ + ")");
+
+ tEnv.executeSql("insert into multi_key_sink select * from orders_with_multi_pks");
+
+ waitForSinkSize("multi_key_sink", realSnapshotData.size());
+ assertEqualsInAnyOrder(
+ expectedSnapshotData,
+ TestValuesTableFactory.getRawResultsAsStrings("multi_key_sink"));
+
+ // third step: check dml events
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("use " + DATABASE_NAME);
+ statement.execute(
+ "INSERT INTO orders_with_multi_pks VALUES (6, 1006,1006, 1006,'2022-01-17');");
+ statement.execute(
+ "INSERT INTO orders_with_multi_pks VALUES (7,1007, 1007,1007, '2022-01-17');");
+ statement.execute(
+ "UPDATE orders_with_multi_pks SET seller_id= 9999, order_id=9999 WHERE id=6;");
+ statement.execute(
+ "UPDATE orders_with_multi_pks SET seller_id= 9999, order_id=9999 WHERE id=7;");
+ statement.execute("DELETE FROM orders_with_multi_pks WHERE id=7;");
+ }
+
+ String[] expectedBinlog =
+ new String[] {
+ "+I[6, 1006, 1006, 1006, 2022-01-17T00:00]",
+ "+I[7, 1007, 1007, 1007, 2022-01-17T00:00]",
+ "-D[6, 1006, 1006, 1006, 2022-01-17T00:00]",
+ "+I[6, 9999, 9999, 1006, 2022-01-17T00:00]",
+ "-D[7, 1007, 1007, 1007, 2022-01-17T00:00]",
+ "+I[7, 9999, 9999, 1007, 2022-01-17T00:00]",
+ "-D[7, 9999, 9999, 1007, 2022-01-17T00:00]"
+ };
+ List realBinlog = fetchRows(iterator, expectedBinlog.length);
+ assertEqualsInAnyOrder(Arrays.asList(expectedBinlog), realBinlog);
+ Thread.sleep(3_000);
+ tableResult.getJobClient().get().cancel().get();
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java
new file mode 100644
index 00000000000..0ea3cb9662d
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseSourceTestBase.java
@@ -0,0 +1,261 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.cdc.connectors.oceanbase;
+
+import org.apache.flink.api.common.JobID;
+import org.apache.flink.cdc.connectors.mysql.debezium.DebeziumUtils;
+import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl;
+import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.table.planner.factories.TestValuesTableFactory;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+
+import io.debezium.connector.mysql.MySqlConnection;
+import org.apache.commons.lang3.StringUtils;
+import org.assertj.core.api.Assertions;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.time.ZoneId;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.UUID;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+import static java.lang.String.format;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/** Basic class for testing Database OceanBase which supported the mysql protocol. */
+public abstract class OceanBaseSourceTestBase extends AbstractTestBase {
+
+ private static final Logger LOG = LoggerFactory.getLogger(OceanBaseSourceTestBase.class);
+ private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
+ protected static final Integer PORT = 3306;
+ protected static final String USER_NAME = System.getenv("OCEANBASE_USERNAME");
+ protected static final String PASSWORD = System.getenv("OCEANBASE_PASSWORD");
+ protected static final String HOSTNAME = System.getenv("OCEANBASE_HOSTNAME");
+
+ protected static String getHost() {
+ return HOSTNAME;
+ }
+
+ protected static Integer getPort() {
+ return PORT;
+ }
+
+ protected static String getUserName() {
+ return USER_NAME;
+ }
+
+ protected static String getPassword() {
+ return PASSWORD;
+ }
+
+ protected static String getJdbcUrl() {
+ return String.format("jdbc:mysql://%s:%s", HOSTNAME, PORT);
+ }
+
+ protected static Connection getJdbcConnection() throws SQLException {
+ String jdbcUrl = getJdbcUrl();
+ LOG.info("jdbcUrl is :" + jdbcUrl);
+ return DriverManager.getConnection(jdbcUrl, USER_NAME, PASSWORD);
+ }
+
+ /** initialize database and tables with ${databaseName}.sql for testing. */
+ protected static void initializeOceanBaseTables(
+ String ddlName, String dbName, Function filter)
+ throws InterruptedException {
+ final String ddlFile = String.format("ddl/mysql/%s.sql", ddlName);
+ final URL ddlTestFile = OceanBaseSourceTestBase.class.getClassLoader().getResource(ddlFile);
+ Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull();
+ // need to sleep 1s, make sure the jdbc connection can be created
+ Thread.sleep(1000);
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("drop database if exists " + dbName);
+ statement.execute("create database if not exists " + dbName);
+ statement.execute("use " + dbName + ";");
+ final List statements =
+ Arrays.stream(
+ Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
+ .map(String::trim)
+ .filter(x -> !x.startsWith("--") && !x.isEmpty())
+ .map(
+ x -> {
+ final Matcher m =
+ COMMENT_PATTERN.matcher(x);
+ return m.matches() ? m.group(1) : x;
+ })
+ .collect(Collectors.joining("\n"))
+ .split(";"))
+ .filter(sql -> filter == null || filter.apply(sql))
+ .collect(Collectors.toList());
+ for (String stmt : statements) {
+ statement.execute(stmt);
+ }
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected static void dropDatabase(String dbName) {
+ try (Connection connection = getJdbcConnection();
+ Statement statement = connection.createStatement()) {
+ statement.execute("drop database if exists " + dbName);
+ } catch (Exception e) {
+ throw new RuntimeException(e);
+ }
+ }
+
+ protected static List fetchRows(Iterator iter, int size) {
+ List rows = new ArrayList<>(size);
+ for (int i = 0; i < size; i++) {
+ Row row = iter.next();
+ rows.add(row.toString());
+ }
+ return rows;
+ }
+
+ protected String getTableNameRegex(String[] captureCustomerTables) {
+ checkState(captureCustomerTables.length > 0);
+ if (captureCustomerTables.length == 1) {
+ return captureCustomerTables[0];
+ } else {
+ // pattern that matches multiple tables
+ return format("(%s)", StringUtils.join(captureCustomerTables, "|"));
+ }
+ }
+
+ protected String getServerId() {
+ final Random random = new Random();
+ int serverId = random.nextInt(100) + 5400;
+ return serverId + "-" + (serverId + 4);
+ }
+
+ // ------------------------------------------------------------------------
+ // test utilities
+ // ------------------------------------------------------------------------
+
+ protected static void waitForSinkSize(String sinkName, int expectedSize)
+ throws InterruptedException {
+ while (sinkSize(sinkName) < expectedSize) {
+ Thread.sleep(100);
+ }
+ }
+
+ protected static int sinkSize(String sinkName) {
+ synchronized (TestValuesTableFactory.class) {
+ try {
+ return TestValuesTableFactory.getRawResults(sinkName).size();
+ } catch (IllegalArgumentException e) {
+ // job is not started yet
+ return 0;
+ }
+ }
+ }
+
+ protected static void assertEqualsInAnyOrder(List expected, List actual) {
+ Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expected);
+ }
+
+ protected static void assertEqualsInOrder(List expected, List actual) {
+ Assertions.assertThat(actual).containsExactlyElementsOf(expected);
+ }
+
+ protected static String getRandomSuffix() {
+ String base = UUID.randomUUID().toString().replaceAll("-", "");
+ if (base.length() > 10) {
+ return base.substring(0, 11);
+ }
+ return base;
+ }
+
+ /** The type of failover. */
+ protected enum FailoverType {
+ TM,
+ JM,
+ NONE
+ }
+
+ /** The phase of failover. */
+ protected enum FailoverPhase {
+ SNAPSHOT,
+ BINLOG,
+ NEVER
+ }
+
+ protected static void triggerFailover(
+ FailoverType type, JobID jobId, MiniCluster miniCluster, Runnable afterFailAction)
+ throws Exception {
+ switch (type) {
+ case TM:
+ restartTaskManager(miniCluster, afterFailAction);
+ break;
+ case JM:
+ triggerJobManagerFailover(jobId, miniCluster, afterFailAction);
+ break;
+ case NONE:
+ break;
+ default:
+ throw new IllegalStateException("Unexpected value: " + type);
+ }
+ }
+
+ protected static void triggerJobManagerFailover(
+ JobID jobId, MiniCluster miniCluster, Runnable afterFailAction) throws Exception {
+ final HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get();
+ haLeadershipControl.revokeJobMasterLeadership(jobId).get();
+ afterFailAction.run();
+ haLeadershipControl.grantJobMasterLeadership(jobId).get();
+ }
+
+ protected static void restartTaskManager(MiniCluster miniCluster, Runnable afterFailAction)
+ throws Exception {
+ miniCluster.terminateTaskManager(0).get();
+ afterFailAction.run();
+ miniCluster.startTaskManager();
+ }
+
+ protected MySqlConnection getConnection() {
+ Map properties = new HashMap<>();
+ properties.put("database.hostname", getHost());
+ properties.put("database.port", String.valueOf(getPort()));
+ properties.put("database.user", getUserName());
+ properties.put("database.password", getPassword());
+ properties.put("database.serverTimezone", ZoneId.of("UTC").toString());
+ io.debezium.config.Configuration configuration =
+ io.debezium.config.Configuration.from(properties);
+ return DebeziumUtils.createMySqlConnection(configuration, new Properties());
+ }
+}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java
deleted file mode 100644
index d9f12f9e762..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestBase.java
+++ /dev/null
@@ -1,167 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase;
-
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata;
-import org.apache.flink.cdc.connectors.utils.StaticExternalResourceProxy;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-import org.apache.flink.table.utils.LegacyRowResource;
-import org.apache.flink.test.util.AbstractTestBase;
-
-import org.assertj.core.api.Assertions;
-import org.junit.jupiter.api.extension.RegisterExtension;
-
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.time.Duration;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeoutException;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-
-/** Basic class for testing OceanBase source. */
-public abstract class OceanBaseTestBase extends AbstractTestBase {
-
- private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
-
- @RegisterExtension
- public static StaticExternalResourceProxy usesLegacyRows =
- new StaticExternalResourceProxy<>(LegacyRowResource.INSTANCE);
-
- public static final Duration FETCH_TIMEOUT = Duration.ofSeconds(60);
-
- protected abstract OceanBaseCdcMetadata metadata();
-
- protected String commonOptionsString() {
- return String.format(
- " 'connector' = 'oceanbase-cdc', "
- + " 'username' = '%s', "
- + " 'password' = '%s', "
- + " 'hostname' = '%s', "
- + " 'port' = '%s', "
- + " 'compatible-mode' = '%s', "
- + " 'jdbc.driver' = '%s'",
- metadata().getUsername(),
- metadata().getPassword(),
- metadata().getHostname(),
- metadata().getPort(),
- metadata().getCompatibleMode(),
- metadata().getDriverClass());
- }
-
- protected String logProxyOptionsString() {
- return String.format(
- " 'working-mode' = 'memory',"
- + " 'tenant-name' = '%s',"
- + " 'logproxy.host' = '%s',"
- + " 'logproxy.port' = '%s'",
- metadata().getTenantName(),
- metadata().getLogProxyHost(),
- metadata().getLogProxyPort());
- }
-
- protected String initialOptionsString() {
- return " 'scan.startup.mode' = 'initial', "
- + commonOptionsString()
- + ", "
- + logProxyOptionsString();
- }
-
- protected String snapshotOptionsString() {
- return " 'scan.startup.mode' = 'snapshot', " + commonOptionsString();
- }
-
- protected Connection getJdbcConnection() throws SQLException {
- return DriverManager.getConnection(
- metadata().getJdbcUrl(), metadata().getUsername(), metadata().getPassword());
- }
-
- protected void setGlobalTimeZone(String serverTimeZone) throws SQLException {
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(String.format("SET GLOBAL time_zone = '%s';", serverTimeZone));
- }
- }
-
- protected void initializeTable(String sqlFile) {
- final String ddlFile =
- String.format("ddl/%s/%s.sql", metadata().getCompatibleMode(), sqlFile);
- final URL ddlTestFile = getClass().getClassLoader().getResource(ddlFile);
- Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull();
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- final List statements =
- Arrays.stream(
- Files.readAllLines(Paths.get(ddlTestFile.toURI())).stream()
- .map(String::trim)
- .filter(x -> !x.startsWith("--") && !x.isEmpty())
- .map(
- x -> {
- final Matcher m =
- COMMENT_PATTERN.matcher(x);
- return m.matches() ? m.group(1) : x;
- })
- .collect(Collectors.joining("\n"))
- .split(";"))
- .collect(Collectors.toList());
- for (String stmt : statements) {
- statement.execute(stmt);
- }
- } catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
- public static void waitForSinkSize(String sinkName, int expectedSize)
- throws InterruptedException, TimeoutException {
- long deadlineTimestamp = System.currentTimeMillis() + FETCH_TIMEOUT.toMillis();
- while (System.currentTimeMillis() < deadlineTimestamp) {
- if (sinkSize(sinkName) < expectedSize) {
- Thread.sleep(100);
- } else {
- return;
- }
- }
- throw new TimeoutException(
- String.format(
- "Failed to fetch enough records in sink.\nExpected size: %d\nActual values: %s",
- expectedSize, TestValuesTableFactory.getRawResults(sinkName)));
- }
-
- public static int sinkSize(String sinkName) {
- synchronized (TestValuesTableFactory.class) {
- try {
- return TestValuesTableFactory.getRawResultsAsStrings(sinkName).size();
- } catch (IllegalArgumentException e) {
- // job is not started yet
- return 0;
- }
- }
- }
-
- public static void assertContainsInAnyOrder(List expected, List actual) {
- Assertions.assertThat(actual).containsAll(expected);
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java
index 47400f2e59d..fd75c282236 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/OceanBaseTestUtils.java
@@ -17,7 +17,6 @@
package org.apache.flink.cdc.connectors.oceanbase;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.LogProxyContainer;
import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer;
import org.slf4j.Logger;
@@ -33,18 +32,10 @@ public class OceanBaseTestUtils {
private static final Logger LOG = LoggerFactory.getLogger(OceanBaseTestUtils.class);
private static final String OB_4_3_3_VERSION = "4.3.3.0-100000142024101215";
- private static final String OB_LOG_PROXY_2_0_2_VERSION = "2.0.2-101000142024080121";
- private static final String CDC_TEST_OB_VERSION = "4.2.1.6-106000012024042515";
private static final String SYS_PASSWORD = "123456";
private static final String TEST_PASSWORD = "654321";
- public static OceanBaseContainer createOceanBaseContainerForCDC() {
- return createOceanBaseContainer(CDC_TEST_OB_VERSION, "mini")
- .withSysPassword(SYS_PASSWORD)
- .withStartupTimeout(Duration.ofMinutes(4));
- }
-
public static OceanBaseContainer createOceanBaseContainerForJdbc() {
return createOceanBaseContainer(OB_4_3_3_VERSION, "mini")
.withStartupTimeout(Duration.ofMinutes(4));
@@ -58,11 +49,4 @@ public static OceanBaseContainer createOceanBaseContainer(String version, String
.withEnv("OB_LOG_DISK_SIZE", "4G")
.withLogConsumer(new Slf4jLogConsumer(LOG));
}
-
- public static LogProxyContainer createLogProxyContainer() {
- return new LogProxyContainer(OB_LOG_PROXY_2_0_2_VERSION)
- .withSysPassword(SYS_PASSWORD)
- .withStartupTimeout(Duration.ofMinutes(1))
- .withLogConsumer(new Slf4jLogConsumer(LOG));
- }
}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java
deleted file mode 100644
index 644eb997cf6..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseMySQLModeITCase.java
+++ /dev/null
@@ -1,648 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.api.common.JobStatus;
-import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.LogProxyContainer;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseContainer;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseMySQLCdcMetadata;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.UniqueDatabase;
-import org.apache.flink.core.execution.JobClient;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Test;
-import org.testcontainers.containers.Network;
-import org.testcontainers.junit.jupiter.Container;
-import org.testcontainers.junit.jupiter.Testcontainers;
-
-import java.sql.Connection;
-import java.sql.Statement;
-import java.time.ZoneId;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createLogProxyContainer;
-import static org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestUtils.createOceanBaseContainerForCDC;
-
-/** Integration tests for OceanBase MySQL mode table source. */
-@Testcontainers
-class OceanBaseMySQLModeITCase extends OceanBaseTestBase {
-
- private final StreamExecutionEnvironment env =
- StreamExecutionEnvironment.getExecutionEnvironment();
- private final StreamTableEnvironment tEnv =
- StreamTableEnvironment.create(
- env, EnvironmentSettings.newInstance().inStreamingMode().build());
-
- private static final Network NETWORK = Network.newNetwork();
-
- @Container
- private static final OceanBaseContainer OB_SERVER =
- createOceanBaseContainerForCDC().withNetwork(NETWORK);
-
- @Container
- private static final LogProxyContainer LOG_PROXY =
- createLogProxyContainer().withNetwork(NETWORK);
-
- private static final OceanBaseCdcMetadata METADATA =
- new OceanBaseMySQLCdcMetadata(OB_SERVER, LOG_PROXY);
-
- private UniqueDatabase inventoryDatabase;
- private UniqueDatabase columnTypesDatabase;
-
- @Override
- protected OceanBaseCdcMetadata metadata() {
- return METADATA;
- }
-
- @BeforeEach
- public void before() {
- TestValuesTableFactory.clearAllData();
- env.enableCheckpointing(1000);
- env.getCheckpointConfig().setMinPauseBetweenCheckpoints(500);
- }
-
- @AfterEach
- public void after() {
- if (inventoryDatabase != null) {
- inventoryDatabase.dropDatabase();
- inventoryDatabase = null;
- }
-
- if (columnTypesDatabase != null) {
- columnTypesDatabase.dropDatabase();
- columnTypesDatabase = null;
- }
- }
-
- @Override
- protected String logProxyOptionsString() {
- return super.logProxyOptionsString()
- + " , "
- + String.format(" 'rootserver-list' = '%s'", METADATA.getRsList());
- }
-
- /**
- * Current OceanBase connector uses timestamp (in seconds) to mark the offset during the
- * transition from {@code SNAPSHOT} to {@code STREAMING} mode. Thus, if some snapshot inserting
- * events are too close to the transitioning offset, snapshot inserting events might be emitted
- * multiple times.
- * This could be safely removed after switching to incremental snapshot framework which provides
- * Exactly-once guarantee.
- */
- private void waitForTableInitialization() throws InterruptedException {
- Thread.sleep(5000L);
- }
-
- @Test
- void testTableList() throws Exception {
- inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory");
- inventoryDatabase.createAndInitialize("mysql");
- waitForTableInitialization();
-
- String sourceDDL =
- String.format(
- "CREATE TABLE ob_source ("
- + " `id` INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " PRIMARY KEY (`id`) NOT ENFORCED"
- + ") WITH ("
- + initialOptionsString()
- + ", "
- + " 'table-list' = '%s'"
- + ")",
- inventoryDatabase.getDatabaseName() + ".products");
-
- String sinkDDL =
- "CREATE TABLE sink ("
- + " `id` INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " PRIMARY KEY (`id`) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '30'"
- + ")";
-
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
-
- waitForSinkSize("sink", 9);
- int snapshotSize = sinkSize("sink");
-
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(String.format("USE %s;", inventoryDatabase.getDatabaseName()));
- statement.execute(
- "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
- statement.execute("UPDATE products SET weight='5.1' WHERE id=107;");
- statement.execute(
- "INSERT INTO products VALUES (default,'jacket','water resistent white wind breaker',0.2);"); // 110
- statement.execute(
- "INSERT INTO products VALUES (default,'scooter','Big 2-wheel scooter ',5.18);");
- statement.execute(
- "UPDATE products SET description='new water resistent white wind breaker', weight='0.5' WHERE id=110;");
- statement.execute("UPDATE products SET weight='5.17' WHERE id=111;");
- statement.execute("DELETE FROM products WHERE id=111;");
- }
-
- waitForSinkSize("sink", snapshotSize + 7);
-
- /*
- *
- * The final database table looks like this:
- *
- * > SELECT * FROM products;
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | id | name | description | weight |
- * +-----+--------------------+---------------------------------------------------------+--------+
- * | 101 | scooter | Small 2-wheel scooter | 3.14 |
- * | 102 | car battery | 12V car battery | 8.1 |
- * | 103 | 12-pack drill bits | 12-pack of drill bits with sizes ranging from #40 to #3 | 0.8 |
- * | 104 | hammer | 12oz carpenter's hammer | 0.75 |
- * | 105 | hammer | 14oz carpenter's hammer | 0.875 |
- * | 106 | hammer | 18oz carpenter hammer | 1 |
- * | 107 | rocks | box of assorted rocks | 5.1 |
- * | 108 | jacket | water resistent black wind breaker | 0.1 |
- * | 109 | spare tire | 24 inch spare tire | 22.2 |
- * | 110 | jacket | new water resistent white wind breaker | 0.5 |
- * +-----+--------------------+---------------------------------------------------------+--------+
- *
- */
-
- List expected =
- Arrays.asList(
- "+I(101,scooter,Small 2-wheel scooter,3.1400000000)",
- "+I(102,car battery,12V car battery,8.1000000000)",
- "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)",
- "+I(104,hammer,12oz carpenter's hammer,0.7500000000)",
- "+I(105,hammer,14oz carpenter's hammer,0.8750000000)",
- "+I(106,hammer,16oz carpenter's hammer,1.0000000000)",
- "+I(107,rocks,box of assorted rocks,5.3000000000)",
- "+I(108,jacket,water resistent black wind breaker,0.1000000000)",
- "+I(109,spare tire,24 inch spare tire,22.2000000000)",
- "+U(106,hammer,18oz carpenter hammer,1.0000000000)",
- "+U(107,rocks,box of assorted rocks,5.1000000000)",
- "+I(110,jacket,water resistent white wind breaker,0.2000000000)",
- "+I(111,scooter,Big 2-wheel scooter ,5.1800000000)",
- "+U(110,jacket,new water resistent white wind breaker,0.5000000000)",
- "+U(111,scooter,Big 2-wheel scooter ,5.1700000000)",
- "-D(111,scooter,Big 2-wheel scooter ,5.1700000000)");
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
-
- result.getJobClient().get().cancel().get();
- }
-
- @Test
- void testMetadataColumns() throws Exception {
- inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory");
- inventoryDatabase.createAndInitialize("mysql");
- waitForTableInitialization();
-
- String sourceDDL =
- String.format(
- "CREATE TABLE ob_source ("
- + " tenant STRING METADATA FROM 'tenant_name' VIRTUAL,"
- + " database STRING METADATA FROM 'database_name' VIRTUAL,"
- + " `table` STRING METADATA FROM 'table_name' VIRTUAL,"
- + " `id` INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " PRIMARY KEY (`id`) NOT ENFORCED"
- + ") WITH ("
- + initialOptionsString()
- + ","
- + " 'database-name' = '%s',"
- + " 'table-name' = '%s'"
- + ")",
- String.format("^%s$", inventoryDatabase.getDatabaseName()),
- "^products$");
-
- String sinkDDL =
- "CREATE TABLE sink ("
- + " tenant STRING,"
- + " database STRING,"
- + " `table` STRING,"
- + " `id` DECIMAL(20, 0) NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " primary key (tenant, database, `table`, `id`) not enforced"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '20'"
- + ")";
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- // async submit job
- TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
-
- waitForSinkSize("sink", 9);
- int snapshotSize = sinkSize("sink");
-
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(String.format("USE %s;", inventoryDatabase.getDatabaseName()));
- statement.execute(
- "UPDATE products SET description='18oz carpenter hammer' WHERE id=106;");
- }
-
- waitForSinkSize("sink", snapshotSize + 1);
-
- String tenant = metadata().getTenantName();
-
- List expected =
- Stream.of(
- "+I(%s,%s,products,101,scooter,Small 2-wheel scooter,3.1400000000)",
- "+I(%s,%s,products,102,car battery,12V car battery,8.1000000000)",
- "+I(%s,%s,products,103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)",
- "+I(%s,%s,products,104,hammer,12oz carpenter's hammer,0.7500000000)",
- "+I(%s,%s,products,105,hammer,14oz carpenter's hammer,0.8750000000)",
- "+I(%s,%s,products,106,hammer,16oz carpenter's hammer,1.0000000000)",
- "+I(%s,%s,products,107,rocks,box of assorted rocks,5.3000000000)",
- "+I(%s,%s,products,108,jacket,water resistent black wind breaker,0.1000000000)",
- "+I(%s,%s,products,109,spare tire,24 inch spare tire,22.2000000000)",
- "+U(%s,%s,products,106,hammer,18oz carpenter hammer,1.0000000000)")
- .map(
- line ->
- String.format(
- line, tenant, inventoryDatabase.getDatabaseName()))
- .collect(Collectors.toList());
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
- result.getJobClient().get().cancel().get();
- }
-
- @Test
- void testAllDataTypes() throws Exception {
- String serverTimeZone = "+00:00";
- setGlobalTimeZone(serverTimeZone);
- tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone));
-
- columnTypesDatabase = new UniqueDatabase(OB_SERVER, "column_type_test");
- columnTypesDatabase.createAndInitialize("mysql");
- waitForTableInitialization();
-
- String sourceDDL =
- String.format(
- "CREATE TABLE ob_source (\n"
- + " `id` INT NOT NULL,\n"
- + " bit1_c BOOLEAN,\n"
- + " tiny1_c BOOLEAN,\n"
- + " boolean_c BOOLEAN,\n"
- + " tiny_c TINYINT,\n"
- + " tiny_un_c SMALLINT,\n"
- + " small_c SMALLINT ,\n"
- + " small_un_c INT ,\n"
- + " medium_c INT,\n"
- + " medium_un_c INT,\n"
- + " int11_c INT,\n"
- + " int_c INT,\n"
- + " int_un_c BIGINT,\n"
- + " big_c BIGINT,\n"
- + " big_un_c DECIMAL(20, 0),\n"
- + " real_c FLOAT,\n"
- + " float_c FLOAT,\n"
- + " double_c DOUBLE,\n"
- + " decimal_c DECIMAL(8, 4),\n"
- + " numeric_c DECIMAL(6, 0),\n"
- + " big_decimal_c STRING,\n"
- + " date_c DATE,\n"
- + " time_c TIME(0),\n"
- + " datetime3_c TIMESTAMP(3),\n"
- + " datetime6_c TIMESTAMP(6),\n"
- + " timestamp_c TIMESTAMP,\n"
- + " timestamp3_c TIMESTAMP(3),\n"
- + " timestamp6_c TIMESTAMP(6),\n"
- + " char_c CHAR(3),\n"
- + " varchar_c VARCHAR(255),\n"
- + " file_uuid BINARY(16),\n"
- + " bit_c BINARY(8),\n"
- + " text_c STRING,\n"
- + " tiny_blob_c BYTES,\n"
- + " medium_blob_c BYTES,\n"
- + " blob_c BYTES,\n"
- + " long_blob_c BYTES,\n"
- + " year_c INT,\n"
- + " set_c ARRAY,\n"
- + " enum_c STRING,\n"
- + " json_c STRING,\n"
- + " primary key (`id`) not enforced"
- + ") WITH ("
- + initialOptionsString()
- + ","
- + " 'database-name' = '%s',"
- + " 'table-name' = '%s',"
- + " 'server-time-zone' = '%s'"
- + ")",
- String.format("^%s$", columnTypesDatabase.getDatabaseName()),
- "^full_types$",
- serverTimeZone);
- String sinkDDL =
- "CREATE TABLE sink ("
- + " `id` INT NOT NULL,\n"
- + " bit1_c BOOLEAN,\n"
- + " tiny1_c BOOLEAN,\n"
- + " boolean_c BOOLEAN,\n"
- + " tiny_c TINYINT,\n"
- + " tiny_un_c SMALLINT,\n"
- + " small_c SMALLINT ,\n"
- + " small_un_c INT ,\n"
- + " medium_c INT,\n"
- + " medium_un_c INT,\n"
- + " int11_c INT,\n"
- + " int_c INT,\n"
- + " int_un_c BIGINT,\n"
- + " big_c BIGINT,\n"
- + " big_un_c DECIMAL(20, 0),\n"
- + " real_c FLOAT,\n"
- + " float_c FLOAT,\n"
- + " double_c DOUBLE,\n"
- + " decimal_c DECIMAL(8, 4),\n"
- + " numeric_c DECIMAL(6, 0),\n"
- + " big_decimal_c STRING,\n"
- + " date_c DATE,\n"
- + " time_c TIME(0),\n"
- + " datetime3_c TIMESTAMP(3),\n"
- + " datetime6_c TIMESTAMP(6),\n"
- + " timestamp_c TIMESTAMP,\n"
- + " timestamp3_c TIMESTAMP(3),\n"
- + " timestamp6_c TIMESTAMP(6),\n"
- + " char_c CHAR(3),\n"
- + " varchar_c VARCHAR(255),\n"
- + " file_uuid STRING,\n"
- + " bit_c BINARY(8),\n"
- + " text_c STRING,\n"
- + " tiny_blob_c BYTES,\n"
- + " medium_blob_c BYTES,\n"
- + " blob_c BYTES,\n"
- + " long_blob_c BYTES,\n"
- + " year_c INT,\n"
- + " set_c ARRAY,\n"
- + " enum_c STRING,\n"
- + " json_c STRING,\n"
- + " primary key (`id`) not enforced"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '3'"
- + ")";
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- TableResult result =
- tEnv.executeSql(
- "INSERT INTO sink SELECT id,\n"
- + "bit1_c,\n"
- + "tiny1_c,\n"
- + "boolean_c,\n"
- + "tiny_c,\n"
- + "tiny_un_c,\n"
- + "small_c ,\n"
- + "small_un_c,\n"
- + "medium_c,\n"
- + "medium_un_c,\n"
- + "int11_c,\n"
- + "int_c,\n"
- + "int_un_c,\n"
- + "big_c,\n"
- + "big_un_c,\n"
- + "real_c,\n"
- + "float_c,\n"
- + "double_c,\n"
- + "decimal_c,\n"
- + "numeric_c,\n"
- + "big_decimal_c,\n"
- + "date_c,\n"
- + "time_c,\n"
- + "datetime3_c,\n"
- + "datetime6_c,\n"
- + "timestamp_c,\n"
- + "timestamp3_c,\n"
- + "timestamp6_c,\n"
- + "char_c,\n"
- + "varchar_c,\n"
- + "TO_BASE64(DECODE(file_uuid, 'UTF-8')),\n"
- + "bit_c,\n"
- + "text_c,\n"
- + "tiny_blob_c,\n"
- + "medium_blob_c,\n"
- + "blob_c,\n"
- + "long_blob_c,\n"
- + "year_c,\n"
- + "set_c,\n"
- + "enum_c,\n"
- + "json_c\n"
- + " FROM ob_source");
-
- waitForSinkSize("sink", 1);
- int snapshotSize = sinkSize("sink");
-
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(String.format("USE %s;", columnTypesDatabase.getDatabaseName()));
- statement.execute(
- "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
- }
-
- waitForSinkSize("sink", snapshotSize + 1);
-
- List expected =
- Arrays.asList(
- "+I(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})",
- "+U(1,false,true,true,127,255,32767,65535,8388607,16777215,2147483647,2147483647,4294967295,9223372036854775807,18446744073709551615,123.102,123.102,404.4443,123.4567,346,34567892.1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,abc,Hello World,ZRrvv70IOQ9I77+977+977+9Nu+/vT57dAA=,[4, 4, 4, 4, 4, 4, 4, 4],text,[16],[16],[16],[16],2022,[a, b],red,{\"key1\": \"value1\"})");
-
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
- result.getJobClient().get().cancel().get();
- }
-
- @Test
- void testTimezoneBerlin() throws Exception {
- testTimeDataTypes("+02:00");
- }
-
- @Test
- void testTimezoneShanghai() throws Exception {
- testTimeDataTypes("+08:00");
- }
-
- void testTimeDataTypes(String serverTimeZone) throws Exception {
- setGlobalTimeZone(serverTimeZone);
- tEnv.getConfig().setLocalTimeZone(ZoneId.of(serverTimeZone));
-
- columnTypesDatabase = new UniqueDatabase(OB_SERVER, "column_type_test");
- columnTypesDatabase.createAndInitialize("mysql");
- waitForTableInitialization();
-
- String sourceDDL =
- String.format(
- "CREATE TABLE ob_source (\n"
- + " `id` INT NOT NULL,\n"
- + " date_c DATE,\n"
- + " time_c TIME(0),\n"
- + " datetime3_c TIMESTAMP(3),\n"
- + " datetime6_c TIMESTAMP(6),\n"
- + " timestamp_c TIMESTAMP,\n"
- + " primary key (`id`) not enforced"
- + ") WITH ("
- + initialOptionsString()
- + ","
- + " 'database-name' = '%s',"
- + " 'table-name' = '%s',"
- + " 'server-time-zone' = '%s'"
- + ")",
- columnTypesDatabase.getDatabaseName(),
- "full_types",
- serverTimeZone);
-
- String sinkDDL =
- "CREATE TABLE sink ("
- + " `id` INT NOT NULL,\n"
- + " date_c DATE,\n"
- + " time_c TIME(0),\n"
- + " datetime3_c TIMESTAMP(3),\n"
- + " datetime6_c TIMESTAMP(6),\n"
- + " timestamp_c TIMESTAMP,\n"
- + " primary key (`id`) not enforced"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '20'"
- + ")";
-
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- TableResult result =
- tEnv.executeSql(
- "INSERT INTO sink SELECT `id`, date_c, time_c, datetime3_c, datetime6_c, cast(timestamp_c as timestamp) FROM ob_source");
-
- // wait for snapshot finished and begin binlog
- waitForSinkSize("sink", 1);
- int snapshotSize = sinkSize("sink");
-
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(String.format("USE %s;", columnTypesDatabase.getDatabaseName()));
- statement.execute(
- "UPDATE full_types SET timestamp_c = '2020-07-17 18:33:22' WHERE id=1;");
- }
-
- waitForSinkSize("sink", snapshotSize + 1);
-
- List expected =
- Arrays.asList(
- "+I(1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:00:22)",
- "+U(1,2020-07-17,18:00:22,2020-07-17T18:00:22.123,2020-07-17T18:00:22.123456,2020-07-17T18:33:22)");
-
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
- result.getJobClient().get().cancel().get();
- }
-
- @Test
- void testSnapshotOnly() throws Exception {
- inventoryDatabase = new UniqueDatabase(OB_SERVER, "inventory");
- inventoryDatabase.createAndInitialize("mysql");
- waitForTableInitialization();
-
- String sourceDDL =
- String.format(
- "CREATE TABLE ob_source ("
- + " `id` INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " PRIMARY KEY (`id`) NOT ENFORCED"
- + ") WITH ("
- + snapshotOptionsString()
- + ", "
- + " 'table-list' = '%s'"
- + ")",
- inventoryDatabase.getDatabaseName() + ".products");
-
- String sinkDDL =
- "CREATE TABLE sink ("
- + " `id` INT NOT NULL,"
- + " name STRING,"
- + " description STRING,"
- + " weight DECIMAL(20, 10),"
- + " PRIMARY KEY (`id`) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '30'"
- + ")";
-
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- TableResult result = tEnv.executeSql("INSERT INTO sink SELECT * FROM ob_source");
-
- waitForSinkSize("sink", 9);
-
- List expected =
- Arrays.asList(
- "+I(101,scooter,Small 2-wheel scooter,3.1400000000)",
- "+I(102,car battery,12V car battery,8.1000000000)",
- "+I(103,12-pack drill bits,12-pack of drill bits with sizes ranging from #40 to #3,0.8000000000)",
- "+I(104,hammer,12oz carpenter's hammer,0.7500000000)",
- "+I(105,hammer,14oz carpenter's hammer,0.8750000000)",
- "+I(106,hammer,16oz carpenter's hammer,1.0000000000)",
- "+I(107,rocks,box of assorted rocks,5.3000000000)",
- "+I(108,jacket,water resistent black wind breaker,0.1000000000)",
- "+I(109,spare tire,24 inch spare tire,22.2000000000)");
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
-
- try {
- while (result.getJobClient()
- .map(JobClient::getJobStatus)
- .map(CompletableFuture::join)
- .orElse(JobStatus.FINISHED)
- .equals(JobStatus.RUNNING)) {
- Thread.sleep(100);
- // Waiting for job to finish (SNAPSHOT job will end spontaneously)
- }
- } catch (IllegalStateException e) {
- // It's fine if miniCluster has been shut down
- }
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java
deleted file mode 100644
index aad8041fecc..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseOracleModeITCase.java
+++ /dev/null
@@ -1,227 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.cdc.connectors.oceanbase.OceanBaseTestBase;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseCdcMetadata;
-import org.apache.flink.cdc.connectors.oceanbase.testutils.OceanBaseOracleCdcMetadata;
-import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
-import org.apache.flink.table.api.EnvironmentSettings;
-import org.apache.flink.table.api.TableResult;
-import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
-import org.apache.flink.table.planner.factories.TestValuesTableFactory;
-
-import org.junit.jupiter.api.Disabled;
-import org.junit.jupiter.api.Test;
-
-import java.sql.Connection;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.List;
-
-/** Integration tests for OceanBase Oracle mode table source. */
-@Disabled("Test ignored before oceanbase-xe docker image is available")
-class OceanBaseOracleModeITCase extends OceanBaseTestBase {
-
- private final StreamExecutionEnvironment env =
- StreamExecutionEnvironment.getExecutionEnvironment();
- private final StreamTableEnvironment tEnv =
- StreamTableEnvironment.create(
- env, EnvironmentSettings.newInstance().inStreamingMode().build());
-
- private static final OceanBaseCdcMetadata METADATA = new OceanBaseOracleCdcMetadata();
-
- @Override
- protected OceanBaseCdcMetadata metadata() {
- return METADATA;
- }
-
- @Override
- protected String logProxyOptionsString() {
- return super.logProxyOptionsString()
- + " , "
- + String.format(" 'config-url' = '%s'", METADATA.getConfigUrl());
- }
-
- @Test
- void testAllDataTypes() throws Exception {
- initializeTable("column_type_test");
-
- String schema = metadata().getDatabase();
- String sourceDDL =
- String.format(
- "CREATE TABLE full_types ("
- + " ID INT NOT NULL,"
- + " VAL_VARCHAR STRING,"
- + " VAL_VARCHAR2 STRING,"
- + " VAL_NVARCHAR2 STRING,"
- + " VAL_CHAR STRING,"
- + " VAL_NCHAR STRING,"
- + " VAL_BF FLOAT,"
- + " VAL_BD DOUBLE,"
- + " VAL_F FLOAT,"
- + " VAL_F_10 FLOAT,"
- + " VAL_NUM DECIMAL(10, 6),"
- + " VAL_DP DOUBLE,"
- + " VAL_R DECIMAL(38,2),"
- + " VAL_DECIMAL DECIMAL(10, 6),"
- + " VAL_NUMERIC DECIMAL(10, 6),"
- + " VAL_NUM_VS DECIMAL(10, 3),"
- + " VAL_INT DECIMAL(38,0),"
- + " VAL_INTEGER DECIMAL(38,0),"
- + " VAL_SMALLINT DECIMAL(38,0),"
- + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0),"
- + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0),"
- + " VAL_NUMBER_1 BOOLEAN,"
- + " VAL_NUMBER_2 TINYINT,"
- + " VAL_NUMBER_4 SMALLINT,"
- + " VAL_NUMBER_9 INT,"
- + " VAL_NUMBER_18 BIGINT,"
- + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT,"
- + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT,"
- + " VAL_NUMBER_9_NEGATIVE_SCALE INT,"
- + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT,"
- + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0),"
- + " VAL_DATE TIMESTAMP,"
- + " VAL_TS TIMESTAMP,"
- + " VAL_TS_PRECISION2 TIMESTAMP(2),"
- + " VAL_TS_PRECISION4 TIMESTAMP(4),"
- + " VAL_TS_PRECISION9 TIMESTAMP(6),"
- + " VAL_CLOB_INLINE STRING,"
- + " VAL_BLOB_INLINE BYTES,"
- + " PRIMARY KEY (ID) NOT ENFORCED"
- + ") WITH ("
- + initialOptionsString()
- + ", "
- + " 'table-list' = '%s'"
- + ")",
- schema + ".FULL_TYPES");
-
- String sinkDDL =
- "CREATE TABLE sink ("
- + " ID INT,"
- + " VAL_VARCHAR STRING,"
- + " VAL_VARCHAR2 STRING,"
- + " VAL_NVARCHAR2 STRING,"
- + " VAL_CHAR STRING,"
- + " VAL_NCHAR STRING,"
- + " VAL_BF FLOAT,"
- + " VAL_BD DOUBLE,"
- + " VAL_F FLOAT,"
- + " VAL_F_10 FLOAT,"
- + " VAL_NUM DECIMAL(10, 6),"
- + " VAL_DP DOUBLE,"
- + " VAL_R DECIMAL(38,2),"
- + " VAL_DECIMAL DECIMAL(10, 6),"
- + " VAL_NUMERIC DECIMAL(10, 6),"
- + " VAL_NUM_VS DECIMAL(10, 3),"
- + " VAL_INT DECIMAL(38,0),"
- + " VAL_INTEGER DECIMAL(38,0),"
- + " VAL_SMALLINT DECIMAL(38,0),"
- + " VAL_NUMBER_38_NO_SCALE DECIMAL(38,0),"
- + " VAL_NUMBER_38_SCALE_0 DECIMAL(38,0),"
- + " VAL_NUMBER_1 BOOLEAN,"
- + " VAL_NUMBER_2 TINYINT,"
- + " VAL_NUMBER_4 SMALLINT,"
- + " VAL_NUMBER_9 INT,"
- + " VAL_NUMBER_18 BIGINT,"
- + " VAL_NUMBER_2_NEGATIVE_SCALE TINYINT,"
- + " VAL_NUMBER_4_NEGATIVE_SCALE SMALLINT,"
- + " VAL_NUMBER_9_NEGATIVE_SCALE INT,"
- + " VAL_NUMBER_18_NEGATIVE_SCALE BIGINT,"
- + " VAL_NUMBER_36_NEGATIVE_SCALE DECIMAL(38,0),"
- + " VAL_DATE TIMESTAMP,"
- + " VAL_TS TIMESTAMP,"
- + " VAL_TS_PRECISION2 TIMESTAMP(2),"
- + " VAL_TS_PRECISION4 TIMESTAMP(4),"
- + " VAL_TS_PRECISION9 TIMESTAMP(6),"
- + " VAL_CLOB_INLINE STRING,"
- + " VAL_BLOB_INLINE STRING,"
- + " PRIMARY KEY (ID) NOT ENFORCED"
- + ") WITH ("
- + " 'connector' = 'values',"
- + " 'sink-insert-only' = 'false',"
- + " 'sink-expected-messages-num' = '2'"
- + ")";
-
- tEnv.executeSql(sourceDDL);
- tEnv.executeSql(sinkDDL);
-
- TableResult result =
- tEnv.executeSql(
- "INSERT INTO sink SELECT "
- + " ID,"
- + " VAL_VARCHAR,"
- + " VAL_VARCHAR2,"
- + " VAL_NVARCHAR2,"
- + " VAL_CHAR,"
- + " VAL_NCHAR,"
- + " VAL_BF,"
- + " VAL_BD,"
- + " VAL_F,"
- + " VAL_F_10,"
- + " VAL_NUM,"
- + " VAL_DP,"
- + " VAL_R,"
- + " VAL_DECIMAL,"
- + " VAL_NUMERIC,"
- + " VAL_NUM_VS,"
- + " VAL_INT,"
- + " VAL_INTEGER,"
- + " VAL_SMALLINT,"
- + " VAL_NUMBER_38_NO_SCALE,"
- + " VAL_NUMBER_38_SCALE_0,"
- + " VAL_NUMBER_1,"
- + " VAL_NUMBER_2,"
- + " VAL_NUMBER_4,"
- + " VAL_NUMBER_9,"
- + " VAL_NUMBER_18,"
- + " VAL_NUMBER_2_NEGATIVE_SCALE,"
- + " VAL_NUMBER_4_NEGATIVE_SCALE,"
- + " VAL_NUMBER_9_NEGATIVE_SCALE,"
- + " VAL_NUMBER_18_NEGATIVE_SCALE,"
- + " VAL_NUMBER_36_NEGATIVE_SCALE,"
- + " VAL_DATE,"
- + " VAL_TS,"
- + " VAL_TS_PRECISION2,"
- + " VAL_TS_PRECISION4,"
- + " VAL_TS_PRECISION9,"
- + " VAL_CLOB_INLINE,"
- + " DECODE(VAL_BLOB_INLINE, 'UTF-8')"
- + " FROM full_types");
-
- waitForSinkSize("sink", 1);
-
- try (Connection connection = getJdbcConnection();
- Statement statement = connection.createStatement()) {
- statement.execute(
- "UPDATE FULL_TYPES SET VAL_TS = '2022-10-30 12:34:56.12545' WHERE id=1;");
- }
-
- waitForSinkSize("sink", 2);
-
- List expected =
- Arrays.asList(
- "+I(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.007890,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)",
- "+U(1,vc2,vc2,nvc2,c ,nc ,1.1,2.22,3.33,8.888,4.444400,5.555,6.66,1234.567891,1234.567891,77.323,1,22,333,4444,5555,true,99,9999,999999999,999999999999999999,90,9900,999999990,999999999999999900,99999999999999999999999999999999999900,2022-10-30T00:00,2022-10-30T12:34:56.125450,2022-10-30T12:34:56.130,2022-10-30T12:34:56.125500,2022-10-30T12:34:56.125457,col_clob,col_blob)");
-
- List actual = TestValuesTableFactory.getRawResultsAsStrings("sink");
- assertContainsInAnyOrder(expected, actual);
- result.getJobClient().get().cancel().get();
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java
deleted file mode 100644
index 9a4dbfcddf3..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/table/OceanBaseTableFactoryTest.java
+++ /dev/null
@@ -1,261 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.table;
-
-import org.apache.flink.cdc.connectors.base.options.StartupOptions;
-import org.apache.flink.configuration.Configuration;
-import org.apache.flink.table.api.DataTypes;
-import org.apache.flink.table.api.Schema;
-import org.apache.flink.table.catalog.CatalogTable;
-import org.apache.flink.table.catalog.Column;
-import org.apache.flink.table.catalog.ObjectIdentifier;
-import org.apache.flink.table.catalog.ResolvedCatalogTable;
-import org.apache.flink.table.catalog.ResolvedSchema;
-import org.apache.flink.table.catalog.UniqueConstraint;
-import org.apache.flink.table.connector.source.DynamicTableSource;
-import org.apache.flink.table.factories.FactoryUtil;
-
-import org.assertj.core.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.time.Duration;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-/** Test for {@link OceanBaseTableSource} created by {@link OceanBaseTableSourceFactory}. */
-class OceanBaseTableFactoryTest {
-
- private static final ResolvedSchema SCHEMA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical("aaa", DataTypes.INT().notNull()),
- Column.physical("bbb", DataTypes.STRING().notNull()),
- Column.physical("ccc", DataTypes.DOUBLE()),
- Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
- Column.physical("eee", DataTypes.TIMESTAMP(3))),
- Collections.emptyList(),
- UniqueConstraint.primaryKey("pk", Collections.singletonList("aaa")));
-
- private static final ResolvedSchema SCHEMA_WITH_METADATA =
- new ResolvedSchema(
- Arrays.asList(
- Column.physical("aaa", DataTypes.INT().notNull()),
- Column.physical("bbb", DataTypes.STRING().notNull()),
- Column.physical("ccc", DataTypes.DOUBLE()),
- Column.physical("ddd", DataTypes.DECIMAL(31, 18)),
- Column.physical("eee", DataTypes.TIMESTAMP(3)),
- Column.metadata("time", DataTypes.TIMESTAMP_LTZ(3), "op_ts", true),
- Column.metadata("tenant", DataTypes.STRING(), "tenant_name", true),
- Column.metadata("database", DataTypes.STRING(), "database_name", true),
- Column.metadata("table", DataTypes.STRING(), "table_name", true)),
- Collections.emptyList(),
- UniqueConstraint.primaryKey("pk", Collections.singletonList("aaa")));
-
- private static final String STARTUP_MODE = "latest-offset";
- private static final String USERNAME = "user@sys";
- private static final String PASSWORD = "pswd";
- private static final String TENANT_NAME = "sys";
- private static final String DATABASE_NAME = "db[0-9]";
- private static final String TABLE_NAME = "table[0-9]";
- private static final String TABLE_LIST = "db.table";
- private static final String SERVER_TIME_ZONE = "+00:00";
- private static final String CONNECT_TIMEOUT = "30s";
- private static final String HOSTNAME = "127.0.0.1";
- private static final Integer PORT = 2881;
- private static final String COMPATIBLE_MODE = "mysql";
- private static final String DRIVER_CLASS = "com.mysql.cj.jdbc.Driver";
- private static final String LOG_PROXY_HOST = "127.0.0.1";
- private static final Integer LOG_PROXY_PORT = 2983;
- private static final String LOG_PROXY_CLIENT_ID = "clientId";
- private static final String RS_LIST = "127.0.0.1:2882:2881";
- private static final String WORKING_MODE = "storage";
-
- @Test
- void testCommonProperties() {
- Map options = getRequiredOptions();
- options.put("database-name", DATABASE_NAME);
- options.put("table-name", TABLE_NAME);
- options.put("table-list", TABLE_LIST);
- options.put("rootserver-list", RS_LIST);
-
- DynamicTableSource actualSource = createTableSource(SCHEMA, options);
- OceanBaseTableSource expectedSource =
- new OceanBaseTableSource(
- SCHEMA,
- StartupOptions.latest(),
- USERNAME,
- PASSWORD,
- TENANT_NAME,
- DATABASE_NAME,
- TABLE_NAME,
- TABLE_LIST,
- SERVER_TIME_ZONE,
- Duration.parse("PT" + CONNECT_TIMEOUT),
- HOSTNAME,
- PORT,
- COMPATIBLE_MODE,
- DRIVER_CLASS,
- new Properties(),
- LOG_PROXY_HOST,
- LOG_PROXY_PORT,
- null,
- null,
- RS_LIST,
- null,
- WORKING_MODE,
- new Properties(),
- new Properties());
- Assertions.assertThat(actualSource).isEqualTo(expectedSource);
- }
-
- @Test
- void testOptionalProperties() {
- Map options = getRequiredOptions();
- options.put("scan.startup.mode", "initial");
- options.put("database-name", DATABASE_NAME);
- options.put("table-name", TABLE_NAME);
- options.put("table-list", TABLE_LIST);
- options.put("compatible-mode", COMPATIBLE_MODE);
- options.put("jdbc.driver", DRIVER_CLASS);
- options.put("logproxy.client.id", LOG_PROXY_CLIENT_ID);
- options.put("rootserver-list", RS_LIST);
- DynamicTableSource actualSource = createTableSource(SCHEMA, options);
-
- OceanBaseTableSource expectedSource =
- new OceanBaseTableSource(
- SCHEMA,
- StartupOptions.initial(),
- USERNAME,
- PASSWORD,
- TENANT_NAME,
- DATABASE_NAME,
- TABLE_NAME,
- TABLE_LIST,
- SERVER_TIME_ZONE,
- Duration.parse("PT" + CONNECT_TIMEOUT),
- "127.0.0.1",
- 2881,
- COMPATIBLE_MODE,
- DRIVER_CLASS,
- new Properties(),
- LOG_PROXY_HOST,
- LOG_PROXY_PORT,
- LOG_PROXY_CLIENT_ID,
- null,
- RS_LIST,
- null,
- WORKING_MODE,
- new Properties(),
- new Properties());
- Assertions.assertThat(actualSource).isEqualTo(expectedSource);
- }
-
- @Test
- void testMetadataColumns() {
- Map options = getRequiredOptions();
- options.put("database-name", DATABASE_NAME);
- options.put("table-name", TABLE_NAME);
- options.put("table-list", TABLE_LIST);
- options.put("rootserver-list", RS_LIST);
-
- DynamicTableSource actualSource = createTableSource(SCHEMA_WITH_METADATA, options);
- OceanBaseTableSource oceanBaseTableSource = (OceanBaseTableSource) actualSource;
- oceanBaseTableSource.applyReadableMetadata(
- Arrays.asList("op_ts", "tenant_name", "database_name", "table_name"),
- SCHEMA_WITH_METADATA.toSourceRowDataType());
- actualSource = oceanBaseTableSource.copy();
-
- OceanBaseTableSource expectedSource =
- new OceanBaseTableSource(
- SCHEMA_WITH_METADATA,
- StartupOptions.latest(),
- USERNAME,
- PASSWORD,
- TENANT_NAME,
- DATABASE_NAME,
- TABLE_NAME,
- TABLE_LIST,
- SERVER_TIME_ZONE,
- Duration.parse("PT" + CONNECT_TIMEOUT),
- HOSTNAME,
- PORT,
- COMPATIBLE_MODE,
- DRIVER_CLASS,
- new Properties(),
- LOG_PROXY_HOST,
- LOG_PROXY_PORT,
- null,
- null,
- RS_LIST,
- null,
- WORKING_MODE,
- new Properties(),
- new Properties());
- expectedSource.producedDataType = SCHEMA_WITH_METADATA.toSourceRowDataType();
- expectedSource.metadataKeys =
- Arrays.asList("op_ts", "tenant_name", "database_name", "table_name");
-
- Assertions.assertThat(actualSource).isEqualTo(expectedSource);
- }
-
- @Test
- void testValidation() {
- Assertions.assertThatThrownBy(
- () -> {
- Map properties = getRequiredOptions();
- properties.put("unknown", "abc");
- createTableSource(SCHEMA, properties);
- })
- .hasStackTraceContaining("Unsupported options:\n\nunknown");
- }
-
- private Map getRequiredOptions() {
- Map options = new HashMap<>();
- options.put("connector", "oceanbase-cdc");
- options.put("scan.startup.mode", STARTUP_MODE);
- options.put("username", USERNAME);
- options.put("password", PASSWORD);
- options.put("hostname", HOSTNAME);
- options.put("port", String.valueOf(PORT));
- options.put("tenant-name", TENANT_NAME);
- options.put("logproxy.host", LOG_PROXY_HOST);
- options.put("logproxy.port", String.valueOf(LOG_PROXY_PORT));
- return options;
- }
-
- private static DynamicTableSource createTableSource(
- ResolvedSchema schema, Map options) {
- return FactoryUtil.createTableSource(
- null,
- ObjectIdentifier.of("default", "default", "t1"),
- new ResolvedCatalogTable(
- CatalogTable.of(
- Schema.newBuilder().fromResolvedSchema(schema).build(),
- "mock source",
- new ArrayList<>(),
- options),
- schema),
- new Configuration(),
- OceanBaseTableFactoryTest.class.getClassLoader(),
- false);
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java
deleted file mode 100644
index c33eccbb428..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/LogProxyContainer.java
+++ /dev/null
@@ -1,62 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.testutils;
-
-import org.jetbrains.annotations.NotNull;
-import org.testcontainers.containers.GenericContainer;
-import org.testcontainers.containers.wait.strategy.Wait;
-import org.testcontainers.utility.DockerImageName;
-
-import java.util.Collections;
-import java.util.Set;
-
-/** OceanBase Log Proxy container. */
-public class LogProxyContainer extends GenericContainer {
-
- private static final String IMAGE = "oceanbase/oblogproxy-ce";
-
- private static final int PORT = 2983;
- private static final String ROOT_USER = "root";
-
- private String sysPassword;
-
- public LogProxyContainer(String version) {
- super(DockerImageName.parse(IMAGE + ":" + version));
- addExposedPorts(PORT);
- setWaitStrategy(Wait.forLogMessage(".*boot success!.*", 1));
- }
-
- @Override
- protected void configure() {
- addEnv("OB_SYS_USERNAME", ROOT_USER);
- addEnv("OB_SYS_PASSWORD", sysPassword);
- }
-
- public @NotNull Set getLivenessCheckPortNumbers() {
- return Collections.singleton(this.getMappedPort(PORT));
- }
-
- public int getPort() {
- return getMappedPort(PORT);
- }
-
- public LogProxyContainer withSysPassword(String sysPassword) {
- this.sysPassword = sysPassword;
- return this;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java
deleted file mode 100644
index bb2469509d8..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseCdcMetadata.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.testutils;
-
-import java.io.Serializable;
-
-/** OceanBase CDC metadata. */
-public interface OceanBaseCdcMetadata extends Serializable {
-
- String getCompatibleMode();
-
- String getHostname();
-
- int getPort();
-
- String getUsername();
-
- String getPassword();
-
- String getDriverClass();
-
- String getDatabase();
-
- String getJdbcUrl();
-
- String getTenantName();
-
- String getLogProxyHost();
-
- int getLogProxyPort();
-
- default String getConfigUrl() {
- return null;
- }
-
- default String getRsList() {
- return null;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java
deleted file mode 100644
index aede8a2924f..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseMySQLCdcMetadata.java
+++ /dev/null
@@ -1,113 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.testutils;
-
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-/** OceanBase CDC MySQL mode metadata. */
-public class OceanBaseMySQLCdcMetadata implements OceanBaseCdcMetadata {
-
- private final OceanBaseContainer obServerContainer;
- private final LogProxyContainer logProxyContainer;
-
- private String rsList;
-
- public OceanBaseMySQLCdcMetadata(
- OceanBaseContainer obServerContainer, LogProxyContainer logProxyContainer) {
- this.obServerContainer = obServerContainer;
- this.logProxyContainer = logProxyContainer;
- }
-
- @Override
- public String getCompatibleMode() {
- return "mysql";
- }
-
- @Override
- public String getHostname() {
- return obServerContainer.getHost();
- }
-
- @Override
- public int getPort() {
- return obServerContainer.getDatabasePort();
- }
-
- @Override
- public String getUsername() {
- return obServerContainer.getUsername();
- }
-
- @Override
- public String getPassword() {
- return obServerContainer.getPassword();
- }
-
- @Override
- public String getDriverClass() {
- return obServerContainer.getDriverClassName();
- }
-
- @Override
- public String getDatabase() {
- return obServerContainer.getDatabaseName();
- }
-
- @Override
- public String getJdbcUrl() {
- return "jdbc:mysql://" + getHostname() + ":" + getPort() + "/?useSSL=false";
- }
-
- @Override
- public String getTenantName() {
- return obServerContainer.getTenantName();
- }
-
- @Override
- public String getLogProxyHost() {
- return logProxyContainer.getHost();
- }
-
- @Override
- public int getLogProxyPort() {
- return logProxyContainer.getPort();
- }
-
- @Override
- public String getRsList() {
- if (rsList == null) {
- try (Connection connection =
- DriverManager.getConnection(
- getJdbcUrl(), getUsername(), getPassword());
- Statement statement = connection.createStatement()) {
- ResultSet rs = statement.executeQuery("SHOW PARAMETERS LIKE 'rootservice_list'");
- rsList = rs.next() ? rs.getString("VALUE") : null;
- } catch (SQLException e) {
- throw new RuntimeException("Failed to query rs list", e);
- }
- if (rsList == null) {
- throw new RuntimeException("Got empty rs list");
- }
- }
- return rsList;
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java
deleted file mode 100644
index c68fdda037b..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/OceanBaseOracleCdcMetadata.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.testutils;
-
-/** OceanBase CDC Oracle mode metadata. */
-public class OceanBaseOracleCdcMetadata implements OceanBaseCdcMetadata {
-
- @Override
- public String getCompatibleMode() {
- return "oracle";
- }
-
- @Override
- public String getHostname() {
- return System.getenv("host");
- }
-
- @Override
- public int getPort() {
- return Integer.parseInt(System.getenv("port"));
- }
-
- @Override
- public String getUsername() {
- return System.getenv("username");
- }
-
- @Override
- public String getPassword() {
- return System.getenv("password");
- }
-
- @Override
- public String getDatabase() {
- return System.getenv("schema");
- }
-
- @Override
- public String getDriverClass() {
- return "com.oceanbase.jdbc.Driver";
- }
-
- @Override
- public String getJdbcUrl() {
- return "jdbc:oceanbase://" + getHostname() + ":" + getPort() + "/" + getDatabase();
- }
-
- @Override
- public String getTenantName() {
- return System.getenv("tenant");
- }
-
- @Override
- public String getLogProxyHost() {
- return System.getenv("log_proxy_host");
- }
-
- @Override
- public int getLogProxyPort() {
- return Integer.parseInt(System.getenv("log_proxy_port"));
- }
-
- @Override
- public String getConfigUrl() {
- return System.getenv("config_url");
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java
deleted file mode 100644
index bb59d98d6f0..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/java/org/apache/flink/cdc/connectors/oceanbase/testutils/UniqueDatabase.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.flink.cdc.connectors.oceanbase.testutils;
-
-import org.assertj.core.api.Assertions;
-
-import java.net.URL;
-import java.nio.file.Files;
-import java.nio.file.Paths;
-import java.sql.Connection;
-import java.sql.DriverManager;
-import java.sql.SQLException;
-import java.sql.Statement;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-/**
- * Create and populate a unique instance of an OceanBase database for each run of JUnit test. A user
- * of class needs to provide a logical name for Debezium and database name. It is expected that
- * there is an init file in src/test/resources/ddl/<database_name>.sql
. The
- * database name is enriched with a unique suffix that guarantees complete isolation between runs
- *
- * <database_name>_<suffix>
- *
- * This class is inspired from Debezium project.
- */
-public class UniqueDatabase {
-
- private static final String[] CREATE_DATABASE_DDL =
- new String[] {"CREATE DATABASE `$DBNAME$`;", "USE `$DBNAME$`;"};
- private static final String DROP_DATABASE_DDL = "DROP DATABASE IF EXISTS `$DBNAME$`;";
- private static final Pattern COMMENT_PATTERN = Pattern.compile("^(.*)--.*$");
-
- private final OceanBaseContainer container;
- private final String databaseName;
- private final String templateName;
-
- public UniqueDatabase(OceanBaseContainer container, String databaseName) {
- this(container, databaseName, Integer.toUnsignedString(new Random().nextInt(), 36));
- }
-
- private UniqueDatabase(
- OceanBaseContainer container, String databaseName, final String identifier) {
- this.container = container;
- this.databaseName = databaseName + "_" + identifier;
- this.templateName = databaseName;
- }
-
- public String getHost() {
- return container.getHost();
- }
-
- public int getDatabasePort() {
- return container.getDatabasePort();
- }
-
- public String getDatabaseName() {
- return databaseName;
- }
-
- public String getUsername() {
- return container.getUsername();
- }
-
- public String getPassword() {
- return container.getPassword();
- }
-
- /** @return Fully qualified table name <databaseName>.<tableName>
*/
- public String qualifiedTableName(final String tableName) {
- return String.format("%s.%s", databaseName, tableName);
- }
-
- public void createAndInitialize() {
- createAndInitializeWithDdlFile(String.format("ddl/%s.sql", templateName));
- }
-
- public void createAndInitialize(String variant) {
- createAndInitializeWithDdlFile(String.format("ddl/%s/%s.sql", variant, templateName));
- }
-
- /** Creates the database and populates it with initialization SQL script. */
- public void createAndInitializeWithDdlFile(String ddlFile) {
- final URL ddlTestFile = UniqueDatabase.class.getClassLoader().getResource(ddlFile);
- Assertions.assertThat(ddlTestFile).withFailMessage("Cannot locate " + ddlFile).isNotNull();
- try {
- try (Connection connection =
- DriverManager.getConnection(
- container.getJdbcUrl(), getUsername(), getPassword());
- Statement statement = connection.createStatement()) {
- final List statements =
- Arrays.stream(
- Stream.concat(
- Arrays.stream(CREATE_DATABASE_DDL),
- Files.readAllLines(
- Paths.get(ddlTestFile.toURI()))
- .stream())
- .map(String::trim)
- .filter(x -> !x.startsWith("--") && !x.isEmpty())
- .map(
- x -> {
- final Matcher m =
- COMMENT_PATTERN.matcher(x);
- return m.matches() ? m.group(1) : x;
- })
- .map(this::convertSQL)
- .collect(Collectors.joining("\n"))
- .split(";"))
- .map(x -> x.replace("$$", ";"))
- .collect(Collectors.toList());
- for (String stmt : statements) {
- statement.execute(stmt);
- }
- }
- } catch (final Exception e) {
- throw new IllegalStateException(e);
- }
- }
-
- /** Drop the database if it is existing. */
- public void dropDatabase() {
- try {
- try (Connection connection =
- DriverManager.getConnection(
- container.getJdbcUrl(), getUsername(), getPassword());
- Statement statement = connection.createStatement()) {
- final String dropDatabaseStatement = convertSQL(DROP_DATABASE_DDL);
- statement.execute(dropDatabaseStatement);
- }
- } catch (final Exception e) {
- throw new IllegalStateException(e);
- }
- }
-
- public Connection getJdbcConnection() throws SQLException {
- return DriverManager.getConnection(
- container.getJdbcUrl(databaseName), getUsername(), getPassword());
- }
-
- private String convertSQL(final String sql) {
- return sql.replace("$DBNAME$", databaseName);
- }
-}
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql
new file mode 100644
index 00000000000..552270acb6c
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/charset_test.sql
@@ -0,0 +1,172 @@
+-- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements. See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+-- ----------------------------------------------------------------------------------------------------------------
+-- DATABASE: charset_test
+-- ----------------------------------------------------------------------------------------------------------------
+
+CREATE TABLE `ascii_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET ascii DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ascii;
+
+CREATE TABLE `big5_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET big5 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=big5;
+
+CREATE TABLE `gbk_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET gbk DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=gbk;
+
+CREATE TABLE `sjis_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET sjis DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=sjis;
+
+CREATE TABLE `cp932_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET cp932 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp932;
+
+CREATE TABLE `gb2312_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET gb2312 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=gb2312;
+
+CREATE TABLE `ujis_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET ujis DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ujis;
+
+CREATE TABLE `euckr_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET euckr DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=euckr;
+
+CREATE TABLE `latin1_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET latin1 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin1;
+
+
+CREATE TABLE `latin2_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET latin2 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=latin2;
+
+
+CREATE TABLE `greek_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET greek DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=greek;
+
+
+CREATE TABLE `hebrew_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET hebrew DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=hebrew;
+
+
+CREATE TABLE `cp866_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET cp866 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp866;
+
+
+CREATE TABLE `tis620_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET tis620 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=tis620;
+
+
+CREATE TABLE `cp1250_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET cp1250 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1250;
+
+CREATE TABLE `cp1251_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET cp1251 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1251;
+
+CREATE TABLE `cp1257_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET cp1257 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=cp1257;
+
+CREATE TABLE `macroman_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET macroman DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=macroman;
+
+CREATE TABLE `macce_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET macce DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=macce;
+
+CREATE TABLE `utf8_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET utf8 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=utf8;
+
+CREATE TABLE `ucs2_test` (
+ `table_id` bigint NOT NULL AUTO_INCREMENT COMMENT '编号',
+ `table_name` varchar(200) CHARACTER SET ucs2 DEFAULT '' COMMENT '表名称',
+ PRIMARY KEY (`table_id`) USING BTREE
+) ENGINE=InnoDB AUTO_INCREMENT=24 DEFAULT CHARSET=ucs2;
+
+INSERT into `ascii_test` values (1, 'ascii test!?'), (2, 'Craig Marshall'), (3, '{test}');
+INSERT into `big5_test` values (1, '大五'), (2, 'Craig Marshall'), (3, '丹店');
+INSERT into `gbk_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据');
+INSERT into `sjis_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ');
+INSERT into `cp932_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ');
+INSERT into `gb2312_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据');
+INSERT into `ujis_test` values (1, 'ひびぴ'), (2, 'Craig Marshall'), (3, 'フブプ');
+INSERT into `euckr_test` values (1, '죠주쥬'), (2, 'Craig Marshall'), (3, '한국어');
+INSERT into `latin1_test` values (1, 'ÀÆÉ'), (2, 'Craig Marshall'), (3, 'Üæû');
+INSERT into `latin2_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŠŞŤŹ');
+INSERT into `greek_test` values (1, 'αβγδε'), (2, 'Craig Marshall'), (3, 'θικλ');
+INSERT into `hebrew_test` values (1, 'בבקשה'), (2, 'Craig Marshall'), (3, 'שרפה');
+INSERT into `cp866_test` values (1, 'твой'), (2, 'Craig Marshall'), (3, 'любой');
+INSERT into `tis620_test` values (1, 'ภาษาไทย'), (2, 'Craig Marshall'), (3, 'ฆงจฉ');
+INSERT into `cp1250_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŠŞŤŹ');
+INSERT into `cp1251_test` values (1, 'твой'), (2, 'Craig Marshall'), (3, 'любой');
+INSERT into `cp1257_test` values (1, 'piedzimst brīvi'), (2, 'Craig Marshall'), (3, 'apveltīti ar saprātu');
+INSERT into `macroman_test` values (1, 'ÀÆÉ'), (2, 'Craig Marshall'), (3, 'Üæû');
+INSERT into `macce_test` values (1, 'ÓÔŐÖ'), (2, 'Craig Marshall'), (3, 'ŮÚŰÜ');
+INSERT into `utf8_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据');
+INSERT into `ucs2_test` values (1, '测试数据'), (2, 'Craig Marshall'), (3, '另一个测试数据');
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql
deleted file mode 100644
index 1d6ae4d70dc..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/column_type_test.sql
+++ /dev/null
@@ -1,74 +0,0 @@
--- Licensed to the Apache Software Foundation (ASF) under one or more
--- contributor license agreements. See the NOTICE file distributed with
--- this work for additional information regarding copyright ownership.
--- The ASF licenses this file to You under the Apache License, Version 2.0
--- (the "License"); you may not use this file except in compliance with
--- the License. You may obtain a copy of the License at
---
--- http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
-
--- ----------------------------------------------------------------------------------------------------------------
--- DATABASE: column_type_test
--- ----------------------------------------------------------------------------------------------------------------
-
-DROP TABLE IF EXISTS full_types;
-CREATE TABLE full_types
-(
- id INT AUTO_INCREMENT NOT NULL,
- bit1_c BIT,
- tiny1_c TINYINT(1),
- boolean_c BOOLEAN,
- tiny_c TINYINT,
- tiny_un_c TINYINT UNSIGNED,
- small_c SMALLINT,
- small_un_c SMALLINT UNSIGNED,
- medium_c MEDIUMINT,
- medium_un_c MEDIUMINT UNSIGNED,
- int11_c INT(11),
- int_c INTEGER,
- int_un_c INTEGER UNSIGNED,
- big_c BIGINT,
- big_un_c BIGINT UNSIGNED,
- real_c REAL,
- float_c FLOAT,
- double_c DOUBLE,
- decimal_c DECIMAL(8, 4),
- numeric_c NUMERIC(6, 0),
- big_decimal_c DECIMAL(65, 1),
- date_c DATE,
- time_c TIME(0),
- datetime3_c DATETIME(3),
- datetime6_c DATETIME(6),
- timestamp_c TIMESTAMP,
- timestamp3_c TIMESTAMP(3),
- timestamp6_c TIMESTAMP(6),
- char_c CHAR(3),
- varchar_c VARCHAR(255),
- file_uuid BINARY(16),
- bit_c BIT(64),
- text_c TEXT,
- tiny_blob_c TINYBLOB,
- medium_blob_c MEDIUMBLOB,
- blob_c BLOB,
- long_blob_c LONGBLOB,
- year_c YEAR,
- set_c SET ('a', 'b'),
- enum_c ENUM ('red', 'green', 'blue'),
- json_c JSON,
- PRIMARY KEY (id)
-) DEFAULT CHARSET = utf8mb4;
-
-INSERT INTO full_types
-VALUES (DEFAULT, 0, 1, true, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 2147483647, 4294967295,
- 9223372036854775807, 18446744073709551615, 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1,
- '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',
- '2020-07-17 18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456',
- 'abc', 'Hello World', unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400', '-', '')),
- b'0000010000000100000001000000010000000100000001000000010000000100', 'text', UNHEX(HEX(16)), UNHEX(HEX(16)),
- UNHEX(HEX(16)), UNHEX(HEX(16)), 2022, 'a,b,a', 'red', '{"key1":"value1"}');
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql
deleted file mode 100644
index e0c56ce1e7a..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/inventory.sql
+++ /dev/null
@@ -1,39 +0,0 @@
--- Licensed to the Apache Software Foundation (ASF) under one or more
--- contributor license agreements. See the NOTICE file distributed with
--- this work for additional information regarding copyright ownership.
--- The ASF licenses this file to You under the Apache License, Version 2.0
--- (the "License"); you may not use this file except in compliance with
--- the License. You may obtain a copy of the License at
---
--- http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
-
--- ----------------------------------------------------------------------------------------------------------------
--- DATABASE: inventory
--- ----------------------------------------------------------------------------------------------------------------
-
-DROP TABLE IF EXISTS products;
-CREATE TABLE products
-(
- id INTEGER NOT NULL AUTO_INCREMENT PRIMARY KEY,
- name VARCHAR(255) NOT NULL DEFAULT 'flink',
- description VARCHAR(512),
- weight DECIMAL(20, 10)
-);
-ALTER TABLE products AUTO_INCREMENT = 101;
-
-INSERT INTO products
-VALUES (default, "scooter", "Small 2-wheel scooter", 3.14),
- (default, "car battery", "12V car battery", 8.1),
- (default, "12-pack drill bits", "12-pack of drill bits with sizes ranging from #40 to #3", 0.8),
- (default, "hammer", "12oz carpenter's hammer", 0.75),
- (default, "hammer", "14oz carpenter's hammer", 0.875),
- (default, "hammer", "16oz carpenter's hammer", 1.0),
- (default, "rocks", "box of assorted rocks", 5.3),
- (default, "jacket", "water resistent black wind breaker", 0.1),
- (default, "spare tire", "24 inch spare tire", 22.2);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql
new file mode 100644
index 00000000000..cc5c30b2fda
--- /dev/null
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/mysql/oceanbase_ddl_test.sql
@@ -0,0 +1,232 @@
+-- Licensed to the Apache Software Foundation (ASF) under one or more
+-- contributor license agreements. See the NOTICE file distributed with
+-- this work for additional information regarding copyright ownership.
+-- The ASF licenses this file to You under the Apache License, Version 2.0
+-- (the "License"); you may not use this file except in compliance with
+-- the License. You may obtain a copy of the License at
+--
+-- http://www.apache.org/licenses/LICENSE-2.0
+--
+-- Unless required by applicable law or agreed to in writing, software
+-- distributed under the License is distributed on an "AS IS" BASIS,
+-- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+-- See the License for the specific language governing permissions and
+-- limitations under the License.
+
+-- ----------------------------------------------------------------------------------------------------------------
+-- DATABASE: oceanbase_ddl_test
+-- ----------------------------------------------------------------------------------------------------------------
+
+-- Create orders table with single primary key
+create table orders (
+ id bigint not null auto_increment,
+ seller_id varchar(30) DEFAULT NULL,
+ order_id varchar(30) DEFAULT NULL,
+ buyer_id varchar(30) DEFAULT NULL,
+ create_time datetime DEFAULT NULL,
+ primary key(id),
+ INDEX `i_seller`(`seller_id`)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by key(id) partitions 3;
+
+-- insert some orders for testing
+INSERT INTO orders
+VALUES (1, 1001, 1, 102, '2022-01-16'),
+ (2, 1002, 2, 105, '2022-01-16'),
+ (3, 1004, 3, 109, '2022-01-16'),
+ (4, 1002, 2, 106, '2022-01-16'),
+ (5, 1003, 1, 107, '2022-01-16');
+
+-- Create orders with multi primary keys
+create table orders_with_multi_pks (
+ id bigint not null auto_increment,
+ seller_id varchar(30) DEFAULT NULL,
+ order_id varchar(30) NOT NULL,
+ buyer_id varchar(30) DEFAULT NULL,
+ create_time datetime DEFAULT NULL,
+ primary key(id, order_id),
+ INDEX `g_mi_seller`(`seller_id`)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by key(id, order_id) partitions 3;
+
+-- insert some orders for testing
+INSERT INTO orders_with_multi_pks
+VALUES (1, 1001, 1, 102, '2022-01-16'),
+ (2, 1002, 2, 105, '2022-01-16'),
+ (3, 1004, 3, 109, '2022-01-16'),
+ (4, 1002, 2, 106, '2022-01-16'),
+ (5, 1003, 1, 107, '2022-01-16');
+
+
+-- create table with full types
+CREATE TABLE oceanbase_full_types (
+ id INT AUTO_INCREMENT,
+ tiny_c TINYINT,
+ tiny_un_c TINYINT UNSIGNED,
+ small_c SMALLINT,
+ small_un_c SMALLINT UNSIGNED,
+ medium_c MEDIUMINT,
+ medium_un_c MEDIUMINT UNSIGNED,
+ int_c INTEGER ,
+ int_un_c INTEGER UNSIGNED,
+ int11_c INT(11) DEFAULT 0,
+ big_c BIGINT,
+ big_un_c BIGINT UNSIGNED,
+ varchar_c VARCHAR(255) DEFAULT '1',
+ char_c CHAR(3) DEFAULT '',
+ real_c REAL,
+ float_c FLOAT,
+ double_c DOUBLE,
+ decimal_c DECIMAL(8, 4),
+ numeric_c NUMERIC(6, 0),
+ big_decimal_c DECIMAL(65, 1),
+ bit1_c BIT,
+ tiny1_c TINYINT(1),
+ boolean_c BOOLEAN,
+ date_c DATE,
+ time_c TIME(0),
+ datetime3_c DATETIME(3),
+ datetime6_c DATETIME(6),
+ timestamp_c TIMESTAMP,
+ file_uuid BINARY(16),
+ bit_c BIT(64),
+ text_c TEXT,
+ tiny_blob_c TINYBLOB,
+ blob_c BLOB,
+ medium_blob_c MEDIUMBLOB,
+ long_blob_c LONGBLOB,
+ year_c YEAR,
+ enum_c enum('red', 'white') default 'red',
+ set_c SET('a', 'b'),
+ json_c JSON,
+ point_c POINT,
+ geometry_c GEOMETRY,
+ linestring_c LINESTRING,
+ polygon_c POLYGON,
+ multipoint_c MULTIPOINT,
+ multiline_c MULTILINESTRING,
+ multipolygon_c MULTIPOLYGON,
+ geometrycollection_c GEOMETRYCOLLECTION,
+ PRIMARY KEY (id),
+ INDEX `g_mit_seller`(`int_c`)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by Hash(id) partitions 3;
+
+INSERT INTO oceanbase_full_types VALUES (
+ DEFAULT, 127, 255, 32767, 65535, 8388607, 16777215, 2147483647, 4294967295, 2147483647, 9223372036854775807,
+ 18446744073709551615,
+ 'Hello World', 'abc', 123.102, 123.102, 404.4443, 123.4567, 345.6, 34567892.1, 0, 1, true,
+ '2020-07-17', '18:00:22', '2020-07-17 18:00:22.123', '2020-07-17 18:00:22.123456', '2020-07-17 18:00:22',
+ unhex(replace('651aed08-390f-4893-b2f1-36923e7b7400','-','')), b'0000010000000100000001000000010000000100000001000000010000000100',
+ 'text',UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)),UNHEX(HEX(16)), 2021,
+ 'red', 'a,b,a', '{"key1": "value1", "key2": "value2", "num1": 1.6708304E7, "num2": 16708305}',
+ ST_GeomFromText('POINT(1 1)'),
+ ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'),
+ ST_GeomFromText('LINESTRING(3 0, 3 3, 3 5)'),
+ ST_GeomFromText('POLYGON((1 1, 2 1, 2 2, 1 2, 1 1))'),
+ ST_GeomFromText('MULTIPOINT((1 1),(2 2))'),
+ ST_GeomFromText('MultiLineString((1 1,2 2,3 3),(4 4,5 5))'),
+ ST_GeomFromText('MULTIPOLYGON(((0 0, 10 0, 10 10, 0 10, 0 0)), ((5 5, 7 5, 7 7, 5 7, 5 5)))'),
+ ST_GeomFromText('GEOMETRYCOLLECTION(POINT(10 10), POINT(30 30), LINESTRING(15 15, 20 20))')
+);
+
+-- Create orders_sink for testing the sink of flink-jdbc-connector
+create table orders_sink (
+ id bigint not null auto_increment,
+ seller_id varchar(30) DEFAULT NULL,
+ order_id varchar(30) DEFAULT NULL,
+ buyer_id varchar(30) DEFAULT NULL,
+ create_time datetime DEFAULT NULL,
+ primary key(id)
+) ENGINE=InnoDB DEFAULT CHARSET=utf8 partition by hash(id) partitions 3;
+
+
+-- Create and populate our users using a single insert with many rows
+CREATE TABLE customers (
+ id INTEGER NOT NULL PRIMARY KEY,
+ name VARCHAR(255) NOT NULL DEFAULT 'flink',
+ address VARCHAR(1024),
+ phone_number VARCHAR(512)
+);
+
+INSERT INTO customers
+VALUES (101,"user_1","Shanghai","123567891234"),
+ (102,"user_2","Shanghai","123567891234"),
+ (103,"user_3","Shanghai","123567891234"),
+ (109,"user_4","Shanghai","123567891234"),
+ (110,"user_5","Shanghai","123567891234"),
+ (111,"user_6","Shanghai","123567891234"),
+ (118,"user_7","Shanghai","123567891234"),
+ (121,"user_8","Shanghai","123567891234"),
+ (123,"user_9","Shanghai","123567891234"),
+ (1009,"user_10","Shanghai","123567891234"),
+ (1010,"user_11","Shanghai","123567891234"),
+ (1011,"user_12","Shanghai","123567891234"),
+ (1012,"user_13","Shanghai","123567891234"),
+ (1013,"user_14","Shanghai","123567891234"),
+ (1014,"user_15","Shanghai","123567891234"),
+ (1015,"user_16","Shanghai","123567891234"),
+ (1016,"user_17","Shanghai","123567891234"),
+ (1017,"user_18","Shanghai","123567891234"),
+ (1018,"user_19","Shanghai","123567891234"),
+ (1019,"user_20","Shanghai","123567891234"),
+ (2000,"user_21","Shanghai","123567891234");
+
+-- table has same name prefix with 'customers.*'
+CREATE TABLE customers_1 (
+ id INTEGER NOT NULL PRIMARY KEY,
+ name VARCHAR(255) NOT NULL DEFAULT 'flink',
+ address VARCHAR(1024),
+ phone_number VARCHAR(512)
+);
+
+INSERT INTO customers_1
+VALUES (101,"user_1","Shanghai","123567891234"),
+ (102,"user_2","Shanghai","123567891234"),
+ (103,"user_3","Shanghai","123567891234"),
+ (109,"user_4","Shanghai","123567891234"),
+ (110,"user_5","Shanghai","123567891234"),
+ (111,"user_6","Shanghai","123567891234"),
+ (118,"user_7","Shanghai","123567891234"),
+ (121,"user_8","Shanghai","123567891234"),
+ (123,"user_9","Shanghai","123567891234"),
+ (1009,"user_10","Shanghai","123567891234"),
+ (1010,"user_11","Shanghai","123567891234"),
+ (1011,"user_12","Shanghai","123567891234"),
+ (1012,"user_13","Shanghai","123567891234"),
+ (1013,"user_14","Shanghai","123567891234"),
+ (1014,"user_15","Shanghai","123567891234"),
+ (1015,"user_16","Shanghai","123567891234"),
+ (1016,"user_17","Shanghai","123567891234"),
+ (1017,"user_18","Shanghai","123567891234"),
+ (1018,"user_19","Shanghai","123567891234"),
+ (1019,"user_20","Shanghai","123567891234"),
+ (2000,"user_21","Shanghai","123567891234");
+
+
+CREATE TABLE customers_no_pk (
+ id INTEGER,
+ name VARCHAR(255) DEFAULT 'flink',
+ address VARCHAR(1024),
+ phone_number VARCHAR(512)
+);
+
+INSERT INTO customers_no_pk
+VALUES (101,"user_1","Shanghai","123567891234"),
+ (102,"user_2","Shanghai","123567891234"),
+ (103,"user_3","Shanghai","123567891234"),
+ (109,"user_4","Shanghai","123567891234"),
+ (110,"user_5","Shanghai","123567891234"),
+ (111,"user_6","Shanghai","123567891234"),
+ (118,"user_7","Shanghai","123567891234"),
+ (121,"user_8","Shanghai","123567891234"),
+ (123,"user_9","Shanghai","123567891234"),
+ (1009,"user_10","Shanghai","123567891234"),
+ (1010,"user_11","Shanghai","123567891234"),
+ (1011,"user_12","Shanghai","123567891234"),
+ (1012,"user_13","Shanghai","123567891234"),
+ (1013,"user_14","Shanghai","123567891234"),
+ (1014,"user_15","Shanghai","123567891234"),
+ (1015,"user_16","Shanghai","123567891234"),
+ (1016,"user_17","Shanghai","123567891234"),
+ (1017,"user_18","Shanghai","123567891234"),
+ (1018,"user_19","Shanghai","123567891234"),
+ (1019,"user_20","Shanghai","123567891234"),
+ (2000,"user_21","Shanghai","123567891234");
\ No newline at end of file
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql b/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql
deleted file mode 100644
index bfe622cfb10..00000000000
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-connector-oceanbase-cdc/src/test/resources/ddl/oracle/column_type_test.sql
+++ /dev/null
@@ -1,70 +0,0 @@
--- Licensed to the Apache Software Foundation (ASF) under one or more
--- contributor license agreements. See the NOTICE file distributed with
--- this work for additional information regarding copyright ownership.
--- The ASF licenses this file to You under the Apache License, Version 2.0
--- (the "License"); you may not use this file except in compliance with
--- the License. You may obtain a copy of the License at
---
--- http://www.apache.org/licenses/LICENSE-2.0
---
--- Unless required by applicable law or agreed to in writing, software
--- distributed under the License is distributed on an "AS IS" BASIS,
--- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
--- See the License for the specific language governing permissions and
--- limitations under the License.
-
-CREATE TABLE FULL_TYPES (
- ID NUMBER(9) NOT NULL,
- VAL_VARCHAR VARCHAR2(1000),
- VAL_VARCHAR2 VARCHAR2(1000),
- VAL_NVARCHAR2 NVARCHAR2(1000),
- VAL_CHAR CHAR(3),
- VAL_NCHAR NCHAR(3),
- VAL_BF BINARY_FLOAT,
- VAL_BD BINARY_DOUBLE,
- VAL_F FLOAT,
- VAL_F_10 FLOAT(10),
- VAL_NUM NUMBER(10, 6),
- VAL_DP FLOAT,
- VAL_R FLOAT(63),
- VAL_DECIMAL NUMBER(10, 6),
- VAL_NUMERIC NUMBER(10, 6),
- VAL_NUM_VS NUMBER,
- VAL_INT NUMBER,
- VAL_INTEGER NUMBER,
- VAL_SMALLINT NUMBER,
- VAL_NUMBER_38_NO_SCALE NUMBER(38),
- VAL_NUMBER_38_SCALE_0 NUMBER(38),
- VAL_NUMBER_1 NUMBER(1),
- VAL_NUMBER_2 NUMBER(2),
- VAL_NUMBER_4 NUMBER(4),
- VAL_NUMBER_9 NUMBER(9),
- VAL_NUMBER_18 NUMBER(18),
- VAL_NUMBER_2_NEGATIVE_SCALE NUMBER(1, -1),
- VAL_NUMBER_4_NEGATIVE_SCALE NUMBER(2, -2),
- VAL_NUMBER_9_NEGATIVE_SCALE NUMBER(8, -1),
- VAL_NUMBER_18_NEGATIVE_SCALE NUMBER(16, -2),
- VAL_NUMBER_36_NEGATIVE_SCALE NUMBER(36, -2),
- VAL_DATE DATE,
- VAL_TS TIMESTAMP(6),
- VAL_TS_PRECISION2 TIMESTAMP(2),
- VAL_TS_PRECISION4 TIMESTAMP(4),
- VAL_TS_PRECISION9 TIMESTAMP(6),
- VAL_CLOB_INLINE CLOB,
- VAL_BLOB_INLINE BLOB,
- primary key (ID)
-);
-
-INSERT INTO FULL_TYPES VALUES (
- 1, 'vc2', 'vc2', 'nvc2', 'c', 'nc',
- 1.1, 2.22, 3.33, 8.888, 4.4444, 5.555, 6.66, 1234.567891, 1234.567891, 77.323,
- 1, 22, 333, 4444, 5555, 1, 99, 9999, 999999999, 999999999999999999,
- 94, 9949, 999999994, 999999999999999949, 99999999999999999999999999999999999949,
- TO_DATE('2022-10-30', 'yyyy-mm-dd'),
- TO_TIMESTAMP('2022-10-30 12:34:56.00789', 'yyyy-mm-dd HH24:MI:SS.FF5'),
- TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),
- TO_TIMESTAMP('2022-10-30 12:34:56.12545', 'yyyy-mm-dd HH24:MI:SS.FF5'),
- TO_TIMESTAMP('2022-10-30 12:34:56.125456789', 'yyyy-mm-dd HH24:MI:SS.FF9'),
- TO_CLOB ('col_clob'),
- utl_raw.cast_to_raw ('col_blob')
-);
diff --git a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml
index 6bc425a71b1..3774acf5f92 100644
--- a/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml
+++ b/flink-cdc-connect/flink-cdc-source-connectors/flink-sql-connector-oceanbase-cdc/pom.xml
@@ -33,6 +33,12 @@ limitations under the License.
flink-connector-oceanbase-cdc
${project.version}
+
+