From b6b4c551766d13588167cb41899c25caae3f4ba4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Luis=20Garc=C3=83=C2=83=C3=82=C2=A9s-Erice?= Date: Fri, 4 Oct 2019 17:21:52 +0200 Subject: [PATCH] DBZ-695 Initial import of DB2 Co-authored-by: zrlurb <38785901+zrlurb@users.noreply.github.com> Co-authored-by: Sean Rooney --- debezium-connector-db2/README.md | 93 +++ debezium-connector-db2/pom.xml | 279 +++++++ .../debezium/connector/db2/ChangeTable.java | 101 +++ .../db2/Db2ChangeEventSourceFactory.java | 56 ++ .../connector/db2/Db2ChangeRecordEmitter.java | 74 ++ .../debezium/connector/db2/Db2Connection.java | 416 +++++++++++ .../debezium/connector/db2/Db2Connector.java | 63 ++ .../connector/db2/Db2ConnectorConfig.java | 375 ++++++++++ .../connector/db2/Db2ConnectorTask.java | 241 ++++++ .../connector/db2/Db2DatabaseSchema.java | 67 ++ .../db2/Db2EventMetadataProvider.java | 59 ++ .../connector/db2/Db2OffsetContext.java | 186 +++++ .../db2/Db2SchemaChangeEventEmitter.java | 51 ++ .../db2/Db2SnapshotChangeEventSource.java | 228 ++++++ .../db2/Db2SourceInfoStructMaker.java | 48 ++ .../db2/Db2StreamingChangeEventSource.java | 412 ++++++++++ .../connector/db2/Db2TaskContext.java | 21 + .../connector/db2/Db2TopicSelector.java | 23 + .../connector/db2/Db2ValueConverters.java | 85 +++ .../java/io/debezium/connector/db2/Lsn.java | 181 +++++ .../io/debezium/connector/db2/Module.java | 38 + .../io/debezium/connector/db2/Nullable.java | 20 + .../io/debezium/connector/db2/SourceInfo.java | 99 +++ .../debezium/connector/db2/TxLogPosition.java | 102 +++ .../io/debezium/connector/db2/build.version | 1 + .../src/test/docker/db2-cdc-docker/Dockerfile | 22 + .../src/test/docker/db2-cdc-docker/asncdc.c | 162 ++++ .../test/docker/db2-cdc-docker/asncdc_UDF.sql | 17 + .../docker/db2-cdc-docker/asncdcaddremove.sql | 194 +++++ .../docker/db2-cdc-docker/asncdctables.sql | 544 ++++++++++++++ .../test/docker/db2-cdc-docker/cdcsetup.sh | 18 + .../src/test/docker/db2-cdc-docker/dbsetup.sh | 45 ++ .../connector/db2/Db2ConnectionIT.java | 28 + .../connector/db2/Db2ConnectorIT.java | 702 ++++++++++++++++++ .../connector/db2/SourceInfoTest.java | 101 +++ .../connector/db2/util/TestHelper.java | 213 ++++++ .../src/test/resources/inventory.sql | 86 +++ .../src/test/resources/log4j.properties | 15 + .../src/test/resources/startup-agent.sql | 1 + .../src/test/resources/startup-cdc-demo.sql | 9 + pom.xml | 6 + 41 files changed, 5482 insertions(+) create mode 100644 debezium-connector-db2/README.md create mode 100644 debezium-connector-db2/pom.xml create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/ChangeTable.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeEventSourceFactory.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeRecordEmitter.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connection.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connector.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorConfig.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorTask.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2DatabaseSchema.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2EventMetadataProvider.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2OffsetContext.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SchemaChangeEventEmitter.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SnapshotChangeEventSource.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SourceInfoStructMaker.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2StreamingChangeEventSource.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TaskContext.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TopicSelector.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ValueConverters.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Lsn.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Module.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/Nullable.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/SourceInfo.java create mode 100644 debezium-connector-db2/src/main/java/io/debezium/connector/db2/TxLogPosition.java create mode 100644 debezium-connector-db2/src/main/resources/io/debezium/connector/db2/build.version create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/Dockerfile create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc.c create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc_UDF.sql create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdcaddremove.sql create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdctables.sql create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/cdcsetup.sh create mode 100644 debezium-connector-db2/src/test/docker/db2-cdc-docker/dbsetup.sh create mode 100644 debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectionIT.java create mode 100644 debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectorIT.java create mode 100644 debezium-connector-db2/src/test/java/io/debezium/connector/db2/SourceInfoTest.java create mode 100644 debezium-connector-db2/src/test/java/io/debezium/connector/db2/util/TestHelper.java create mode 100644 debezium-connector-db2/src/test/resources/inventory.sql create mode 100644 debezium-connector-db2/src/test/resources/log4j.properties create mode 100644 debezium-connector-db2/src/test/resources/startup-agent.sql create mode 100644 debezium-connector-db2/src/test/resources/startup-cdc-demo.sql diff --git a/debezium-connector-db2/README.md b/debezium-connector-db2/README.md new file mode 100644 index 000000000..22913204e --- /dev/null +++ b/debezium-connector-db2/README.md @@ -0,0 +1,93 @@ +# Ingesting DB2 change events + +This module defines the connector that ingests change events from DB2 databases. + +## Using the DB2 connector with Kafka Connect + +The DB2 connector is designed to work with [Kafka Connect](http://kafka.apache.org/documentation.html#connect) and to be deployed to a Kafka Connect runtime service. The deployed connector will monitor one or more databases and write all change events to Kafka topics, which can be independently consumed by one or more clients. Kafka Connect can be distributed to provide fault tolerance to ensure the connectors are running and continually keeping up with changes in the database. + +Kafka Connect can also be run standalone as a single process, although doing so is not tolerant of failures. + +## Embedding the DB2 connector + +The DB2 connector can also be used as a library without Kafka or Kafka Connect, enabling applications and services to directly connect to a DB2 database and obtain the ordered change events. This approach requires the application to record the progress of the connector so that upon restart the connect can continue where it left off. Therefore, this may be a useful approach for less critical use cases. For production use cases, we highly recommend using this connector with Kafka and Kafka Connect. + + + +## Testing + +This module contains both unit tests and integration tests. + +A *unit test* is a JUnit test class named `*Test.java` or `Test*.java` that never requires or uses external services, though it can use the file system and can run any components within the same JVM process. They should run very quickly, be independent of each other, and clean up after itself. + +An *integration test* is a JUnit test class named `*IT.java` or `IT*.java` that uses a DB2 database server running in a custom Docker container based upon the [ibmcom/db2](https://hub.docker.com/r/ibmcom/db2) Docker image maintained by the DB2 team. The build will automatically start the DB2 container before the integration tests are run and automatically stop and remove it after all of the integration tests complete (regardless of whether they suceed or fail). All databases used in the integration tests are defined and populated using `*.sql` files and `*.sh` scripts in the `src/test/docker/db2-cdc-docker` directory, which are copied into the Docker image and run by DB2 upon startup. Multiple test methods within a single integration test class can reuse the same database, but generally each integration test class should use its own dedicated database(s). + +Running `mvn install` will compile all code and run the unit and integration tests. If there are any compile problems or any of the unit tests fail, the build will stop immediately. Otherwise, the command will continue to create the module's artifacts, create the Docker image with DB2 and custom scripts, start the Docker container, run the integration tests, stop the container (even if there are integration test failures), and run checkstyle on the code. If there are still no problems, the build will then install the module's artifacts into the local Maven repository. + +You should always default to using `mvn install`, especially prior to committing changes to Git. However, there are a few situations where you may want to run a different Maven command. + +### Running some tests + +If you are trying to get the test methods in a single integration test class to pass and would rather not run *all* of the integration tests, you can instruct Maven to just run that one integration test class and to skip all of the others. For example, use the following command to run the tests in the `ConnectionIT.java` class: + + $ mvn -Dit.test=ConnectionIT install + +Of course, wildcards also work: + + $ mvn -Dit.test=Connect*IT install + +These commands will automatically manage the DB2 Docker container. + +### Debugging tests + +If you want to debug integration tests by stepping through them in your IDE, using the `mvn install` command will be problematic since it will not wait for your IDE's breakpoints. There are ways of doing this, but it is typically far easier to simply start the Docker container and leave it running so that it is available when you run the integration test(s). The following command: + + $ mvn docker:start + +will start the default DB2 container and run the database server. Now you can use your IDE to run/debug one or more integration tests. Just be sure that the integration tests clean up their database before (and after) each test, and that you run the tests with VM arguments that define the required system properties, including: + +* `database.dbname` - the name of the database that your integration test will use; there is no default +* `database.hostname` - the IP address or name of the host where the Docker container is running; defaults to `localhost` which is likely for Linux, but on OS X and Windows Docker it will have to be set to the IP address of the VM that runs Docker (which you can find by looking at the `DOCKER_HOST` environment variable). +* `database.port` - the port on which DB2 is listening; defaults to `50000` and is what this module's Docker container uses +* `database.user` - the name of the database user; defaults to `db2inst1` and is correct unless your database script uses something different +* `database.password` - the password of the database user; defaults to `admin` and is correct unless your database script uses something different + +For example, you can define these properties by passing these arguments to the JVM: + + -Ddatabase.dbname= -Ddatabase.hostname= -Ddatabase.port=50000 -Ddatabase.user=db2inst1 -Ddatabase.password=admin + +When you are finished running the integration tests from your IDE, you have to stop and remove the Docker container before you can run the next build: + + $ mvn docker:stop + + +Please note that when running the DB2 database Docker container, the output is written to the Maven build output and includes several lines with `[Warning] Using a password on the command line interface can be insecure.` You can ignore these warnings, since we don't need a secure database server for our transient database testing. + +### Analyzing the database + +Sometimes you may want to inspect the state of the database(s) after one or more integration tests are run. The `mvn install` command runs the tests but shuts down and removes the container after the integration tests complete. To keep the container running after the integration tests complete, use this Maven command: + + $ mvn integration-test + +### Stopping the Docker container + +This instructs Maven to run the normal Maven lifecycle through `integration-test`, and to stop before the `post-integration-test` phase when the Docker container is normally shut down and removed. Be aware that you will need to manually stop and remove the container before running the build again: + + $ mvn docker:stop + +### Testing all DB2 configurations + +In Debezium builds, the `assembly` profile is used when issuing a release or in our continuous integration builds. In addition to the normal steps, it also creates several additional artifacts (including the connector plugin's ZIP and TAR archives) and runs the whole +integration test suite once for _each_ of the DB2 configurations. If you want to make sure that your changes work on all DB2 configurations, add `-Passembly` to your Maven commands. + + +name=db2-connector +connector.class=io.debezium.connector.db2.Db2Connector +database.hostname=localhost +database.port=50000 +database.user=db2inst1 +database.password=admin +database.dbname=TESTDB +database.cdcschema=ASNCDC +database.history.kafka.bootstrap.servers=localhost:9092 +database.history.kafka.topic=CDCTESTDB \ No newline at end of file diff --git a/debezium-connector-db2/pom.xml b/debezium-connector-db2/pom.xml new file mode 100644 index 000000000..218b2420a --- /dev/null +++ b/debezium-connector-db2/pom.xml @@ -0,0 +1,279 @@ + + + + io.debezium + debezium-parent + 1.1.0-SNAPSHOT + ../pom.xml + + 4.0.0 + debezium-connector-db2 + Debezium Connector for IBM DB2 Server + jar + + + + 50000 + db2inst1 + admin + testdb + debezium/db2-cdc-it + false + true + + + + + + + + io.debezium + debezium-core + + + + + com.ibm.db2 + jcc + 11.5.0.0 + + + org.apache.kafka + connect-api + provided + + + org.slf4j + slf4j-api + provided + + + + + io.debezium + debezium-embedded + test-jar + test + + + io.debezium + debezium-core + test-jar + test + + + io.debezium + debezium-embedded + test + + + org.slf4j + slf4j-log4j12 + test + + + log4j + log4j + test + + + junit + junit + test + + + org.easytesting + fest-assert + test + + + io.confluent + kafka-connect-avro-converter + test + + + + + + + io.fabric8 + docker-maven-plugin + + 500 + default + true + + + + ${docker.filter} + + ${project.basedir}/src/test/docker/db2-cdc-docker + + + none + true + + accept + ${db2.password} + ${db2.dbname} + + + ${db2.port}:50000 + + + DB2 >>> + true + yellow + + + + STOP : DATABASE: TESTDB : DEACTIVATED: NO + + + + + + + + + start + pre-integration-test + + build + start + + + + stop + post-integration-test + + stop + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + integration-test + + integration-test + + + + verify + + verify + + + + + ${skipITs} + true + + + ${docker.host.address} + ${db2.dbname} + ${db2.port} + ${db2.user} + ${db2.password} + ${skipLongRunningTests} + + + + + + + + true + src/main/resources + + * + **/* + + + + + + src/test/resources + true + + * + **/* + + + + + + + + assembly + + false + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + io.debezium + debezium-assembly-descriptors + ${project.version} + + + + + default + package + + single + + + ${project.artifactId}-${project.version} + true + + ${assembly.descriptor} + + + + + + + + + + + skip-integration-tests + + false + + skipITs + + + + true + + + + diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/ChangeTable.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/ChangeTable.java new file mode 100644 index 000000000..1d50dd372 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/ChangeTable.java @@ -0,0 +1,101 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +import io.debezium.relational.TableId; + +/** + * A logical representation of change table containing changes for a given source table. + * There is usually one change table for each source table. When the schema of the source table + * is changed then two change tables could be present. + * + * @author Jiri Pechanec, Peter Urbanetz + * + */ +public class ChangeTable { + + private static final String CDC_SCHEMA = "ASNCDC"; + + /** + * The logical name of the change capture process + */ + private final String captureInstance; + + /** + * The table from which the changes are captured + */ + private final TableId sourceTableId; + + /** + * The table that contains the changes for the source table + */ + private final TableId changeTableId; + + /** + * A LSN from which the data in the change table are relevant + */ + private final Lsn startLsn; + + /** + * A LSN to which the data in the change table are relevant + */ + private Lsn stopLsn; + + /** + * Numeric identifier of change table in DB2 schema + */ + private final int changeTableObjectId; + + public ChangeTable(TableId sourceTableId, String captureInstance, int changeTableObjectId, Lsn startLsn, Lsn stopLsn) { + super(); + this.sourceTableId = sourceTableId; + this.captureInstance = captureInstance; + this.changeTableObjectId = changeTableObjectId; + this.startLsn = startLsn; + this.stopLsn = stopLsn; + this.changeTableId = sourceTableId != null ? new TableId(sourceTableId.catalog(), CDC_SCHEMA, captureInstance) : null; + // this.changeTableId = sourceTableId != null ? new TableId(sourceTableId.catalog(), CDC_SCHEMA, captureInstance + "_CT") : null; + } + + public ChangeTable(String captureInstance, int changeTableObjectId, Lsn startLsn, Lsn stopLsn) { + this(null, captureInstance, changeTableObjectId, startLsn, stopLsn); + } + + public String getCaptureInstance() { + return captureInstance; + } + + public Lsn getStartLsn() { + return startLsn; + } + + public Lsn getStopLsn() { + return stopLsn; + } + + public void setStopLsn(Lsn stopLsn) { + this.stopLsn = stopLsn; + } + + public TableId getSourceTableId() { + return sourceTableId; + } + + public TableId getChangeTableId() { + return changeTableId; + } + + public int getChangeTableObjectId() { + return changeTableObjectId; + } + + @Override + public String toString() { + return "Capture instance \"" + captureInstance + "\" [sourceTableId=" + sourceTableId + + ", changeTableId=" + changeTableId + ", startLsn=" + startLsn + ", changeTableObjectId=" + + changeTableObjectId + ", stopLsn=" + stopLsn + "]"; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeEventSourceFactory.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeEventSourceFactory.java new file mode 100644 index 000000000..4b798811b --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeEventSourceFactory.java @@ -0,0 +1,56 @@ +/* + * 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.db2; + +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.spi.ChangeEventSourceFactory; +import io.debezium.pipeline.source.spi.SnapshotChangeEventSource; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.pipeline.source.spi.StreamingChangeEventSource; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.TableId; +import io.debezium.util.Clock; + +public class Db2ChangeEventSourceFactory implements ChangeEventSourceFactory { + + private final Db2ConnectorConfig configuration; + private final Db2Connection dataConnection; + private final Db2Connection metadataConnection; + private final ErrorHandler errorHandler; + private final EventDispatcher dispatcher; + private final Clock clock; + private final Db2DatabaseSchema schema; + + public Db2ChangeEventSourceFactory(Db2ConnectorConfig configuration, Db2Connection dataConnection, Db2Connection metadataConnection, + ErrorHandler errorHandler, EventDispatcher dispatcher, Clock clock, Db2DatabaseSchema schema) { + this.configuration = configuration; + this.dataConnection = dataConnection; + this.metadataConnection = metadataConnection; + this.errorHandler = errorHandler; + this.dispatcher = dispatcher; + this.clock = clock; + this.schema = schema; + } + + @Override + public SnapshotChangeEventSource getSnapshotChangeEventSource(OffsetContext offsetContext, SnapshotProgressListener snapshotProgressListener) { + return new Db2SnapshotChangeEventSource(configuration, (Db2OffsetContext) offsetContext, dataConnection, schema, dispatcher, clock, snapshotProgressListener); + } + + @Override + public StreamingChangeEventSource getStreamingChangeEventSource(OffsetContext offsetContext) { + return new Db2StreamingChangeEventSource( + configuration, + (Db2OffsetContext) offsetContext, + dataConnection, + metadataConnection, + dispatcher, + errorHandler, + clock, + schema); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeRecordEmitter.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeRecordEmitter.java new file mode 100644 index 000000000..008250e3f --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ChangeRecordEmitter.java @@ -0,0 +1,74 @@ +/* + * 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.db2; + +import io.debezium.data.Envelope.Operation; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.RelationalChangeRecordEmitter; +import io.debezium.util.Clock; + +/** + * Emits change data based on a single (or two in case of updates) CDC data row(s). + * + * @author Jiri Pechanec + */ +public class Db2ChangeRecordEmitter extends RelationalChangeRecordEmitter { + + public static final int OP_DELETE = 1; + public static final int OP_INSERT = 2; + public static final int OP_UPDATE_BEFORE = 3; + public static final int OP_UPDATE_AFTER = 4; + + private final int operation; + private final Object[] data; + private final Object[] dataNext; + + public Db2ChangeRecordEmitter(OffsetContext offset, int operation, Object[] data, Object[] dataNext, Clock clock) { + super(offset, clock); + + this.operation = operation; + this.data = data; + this.dataNext = dataNext; + } + + @Override + protected Operation getOperation() { + if (operation == OP_DELETE) { + return Operation.DELETE; + } + else if (operation == OP_INSERT) { + return Operation.CREATE; + } + else if (operation == OP_UPDATE_BEFORE) { + return Operation.UPDATE; + } + throw new IllegalArgumentException("Received event of unexpected command type: " + operation); + } + + @Override + protected Object[] getOldColumnValues() { + switch (getOperation()) { + case CREATE: + case READ: + return null; + default: + return data; + } + } + + @Override + protected Object[] getNewColumnValues() { + switch (getOperation()) { + case CREATE: + case READ: + return data; + case UPDATE: + return dataNext; + default: + return null; + } + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connection.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connection.java new file mode 100644 index 000000000..daf10808e --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connection.java @@ -0,0 +1,416 @@ +/* + * 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.db2; + +import java.sql.DatabaseMetaData; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Timestamp; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.ibm.db2.jcc.DB2Driver; + +import io.debezium.config.Configuration; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.jdbc.JdbcConnection; +import io.debezium.relational.Column; +import io.debezium.relational.ColumnEditor; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.util.BoundedConcurrentHashMap; + +/** + * {@link JdbcConnection} extension to be used with Microsoft DB2 + * + * @author Horia Chiorean (hchiorea@redhat.com), Jiri Pechanec, Peter Urbanetz + * + */ +public class Db2Connection extends JdbcConnection { + + private static final String GET_DATABASE_NAME = "SELECT CURRENT_SERVER FROM SYSIBM.SYSDUMMY1"; // DB2 + + private static Logger LOGGER = LoggerFactory.getLogger(Db2Connection.class); + + private static final String CDC_SCHEMA = "ASNCDC"; + + private static final String STATEMENTS_PLACEHOLDER = "#"; + private static final String GET_MAX_LSN = "SELECT max(t.SYNCHPOINT) FROM ( SELECT CD_NEW_SYNCHPOINT AS SYNCHPOINT FROM " + CDC_SCHEMA + + ".IBMSNAP_REGISTER UNION ALL SELECT SYNCHPOINT AS SYNCHPOINT FROM " + CDC_SCHEMA + ".IBMSNAP_REGISTER) t"; + + private static final String LOCK_TABLE = "SELECT * FROM # WITH CS"; // DB2 + + private static final String LSN_TO_TIMESTAMP = "SELECT CURRENT TIMEstamp FROM sysibm.sysdummy1 WHERE ? > X'00000000000000000000000000000000'"; + + private static final String INCREMENT_LSN = " SELECT *" + + " FROM (select CAST(CAST( ? AS VARCHAR(16) FOR BIT DATA) AS VARCHAR(32)) AS LSN from sysibm.sysdummy1) FETCH FIRST ROW ONLY;"; + + private static final String GET_ALL_CHANGES_FOR_TABLE = "SELECT " + + "CASE " + + "WHEN IBMSNAP_OPERATION = 'D' AND (LEAD(cdc.IBMSNAP_OPERATION,1,'X') OVER (PARTITION BY cdc.IBMSNAP_COMMITSEQ ORDER BY cdc.IBMSNAP_INTENTSEQ)) ='I' THEN 3 " + + "WHEN IBMSNAP_OPERATION = 'I' AND (LAG(cdc.IBMSNAP_OPERATION,1,'X') OVER (PARTITION BY cdc.IBMSNAP_COMMITSEQ ORDER BY cdc.IBMSNAP_INTENTSEQ)) ='D' THEN 4 " + + "WHEN IBMSNAP_OPERATION = 'D' THEN 1 " + + "WHEN IBMSNAP_OPERATION = 'I' THEN 2 " + + "END " + + "OPCODE," + + "cdc.* " + + "FROM ASNCDC.# cdc WHERE IBMSNAP_COMMITSEQ >= ? AND IBMSNAP_COMMITSEQ <= ? " + + "order by IBMSNAP_COMMITSEQ, IBMSNAP_INTENTSEQ"; + + private static final String GET_LIST_OF_CDC_ENABLED_TABLES = "select r.SOURCE_OWNER, r.SOURCE_TABLE, r.CD_OWNER, r.CD_TABLE, r.CD_NEW_SYNCHPOINT, r.CD_OLD_SYNCHPOINT, t.TBSPACEID, t.TABLEID , CAST((t.TBSPACEID * 65536 + t.TABLEID )AS INTEGER )from " + + CDC_SCHEMA + ".IBMSNAP_REGISTER r left JOIN SYSCAT.TABLES t ON r.SOURCE_OWNER = t.TABSCHEMA AND r.SOURCE_TABLE = t.TABNAME WHERE r.SOURCE_OWNER <> ''"; + + // No new Tabels 1=0 + private static final String GET_LIST_OF_NEW_CDC_ENABLED_TABLES = "select CAST((t.TBSPACEID * 65536 + t.TABLEID )AS INTEGER ) AS OBJECTID, " + + " CD_OWNER CONCAT '.' CONCAT CD_TABLE, " + + " CD_NEW_SYNCHPOINT, " + + " CD_OLD_SYNCHPOINT " + + "from ASNCDC.IBMSNAP_REGISTER r left JOIN SYSCAT.TABLES t ON r.SOURCE_OWNER = t.TABSCHEMA AND r.SOURCE_TABLE = t.TABNAME " + + "WHERE r.SOURCE_OWNER <> '' AND 1=0 AND CD_NEW_SYNCHPOINT > ? AND CD_OLD_SYNCHPOINT < ? "; + + private static final String GET_LIST_OF_KEY_COLUMNS = "SELECT " + + "CAST((t.TBSPACEID * 65536 + t.TABLEID )AS INTEGER ) as objectid, " + + "c.colname,c.colno,c.keyseq " + + "FROM syscat.tables as t " + + "inner join syscat.columns as c on t.tabname = c.tabname and t.tabschema = c.tabschema and c.KEYSEQ > 0 AND " + + "t.tbspaceid = CAST(BITAND( ? , 4294901760) / 65536 AS SMALLINT) AND t.tableid= CAST(BITAND( ? , 65535) AS SMALLINT)"; + + private static final int CHANGE_TABLE_DATA_COLUMN_OFFSET = 4; + + private static final String URL_PATTERN = "jdbc:db2://${" + JdbcConfiguration.HOSTNAME + "}:${" + JdbcConfiguration.PORT + "}/${" + JdbcConfiguration.DATABASE + "}"; + + private static final ConnectionFactory FACTORY = JdbcConnection.patternBasedFactory(URL_PATTERN, + DB2Driver.class.getName(), + Db2Connection.class.getClassLoader()); + + /** + * actual name of the database, which could differ in casing from the database name given in the connector config. + */ + private final String realDatabaseName; + + private static interface ResultSetExtractor { + T apply(ResultSet rs) throws SQLException; + } + + private final BoundedConcurrentHashMap lsnToInstantCache; + + /** + * Creates a new connection using the supplied configuration. + * + * @param config {@link Configuration} instance, may not be null. + */ + public Db2Connection(Configuration config) { + super(config, FACTORY); + lsnToInstantCache = new BoundedConcurrentHashMap<>(100); + realDatabaseName = retrieveRealDatabaseName(); + } + + /** + * @return the current largest log sequence number + */ + public Lsn getMaxLsn() throws SQLException { + return queryAndMap(GET_MAX_LSN, singleResultMapper(rs -> { + final Lsn ret = Lsn.valueOf(rs.getBytes(1)); + LOGGER.trace("Current maximum lsn is {}", ret); + return ret; + }, "Maximum LSN query must return exactly one value")); + } + + /** + * Provides all changes recorded by the DB2 CDC capture process for a given table. + * + * @param tableId - the requested table changes + * @param fromLsn - closed lower bound of interval of changes to be provided + * @param toLsn - closed upper bound of interval of changes to be provided + * @param consumer - the change processor + * @throws SQLException + */ + public void getChangesForTable(TableId tableId, Lsn fromLsn, Lsn toLsn, ResultSetConsumer consumer) throws SQLException { + final String query = GET_ALL_CHANGES_FOR_TABLE.replace(STATEMENTS_PLACEHOLDER, cdcNameForTable(tableId)); + prepareQuery(query, statement -> { + statement.setBytes(1, fromLsn.getBinary()); + statement.setBytes(2, toLsn.getBinary()); + + }, consumer); + } + + /** + * Provides all changes recorder by the DB2 CDC capture process for a set of tables. + * + * @param changeTables - the requested tables to obtain changes for + * @param intervalFromLsn - closed lower bound of interval of changes to be provided + * @param intervalToLsn - closed upper bound of interval of changes to be provided + * @param consumer - the change processor + * @throws SQLException + */ + public void getChangesForTables(ChangeTable[] changeTables, Lsn intervalFromLsn, Lsn intervalToLsn, BlockingMultiResultSetConsumer consumer) + throws SQLException, InterruptedException { + final String[] queries = new String[changeTables.length]; + final StatementPreparer[] preparers = new StatementPreparer[changeTables.length]; + + int idx = 0; + for (ChangeTable changeTable : changeTables) { + final String query = GET_ALL_CHANGES_FOR_TABLE.replace(STATEMENTS_PLACEHOLDER, changeTable.getCaptureInstance()); + queries[idx] = query; + // If the table was added in the middle of queried buffer we need + // to adjust from to the first LSN available + final Lsn fromLsn = changeTable.getStartLsn().compareTo(intervalFromLsn) > 0 ? changeTable.getStartLsn() : intervalFromLsn; + LOGGER.trace("Getting changes for table {} in range[{}, {}]", changeTable, fromLsn, intervalToLsn); + preparers[idx] = statement -> { + statement.setBytes(1, fromLsn.getBinary()); + statement.setBytes(2, intervalToLsn.getBinary()); + + }; + + idx++; + } + prepareQuery(queries, preparers, consumer); + } + + /** + * Obtain the next available position in the database log. + * + * @param lsn - LSN of the current position + * @return LSN of the next position in the database + * @throws SQLException + */ + public Lsn incrementLsn(Lsn lsn) throws SQLException { + return lsn.increment(); + } + + /** + * Map a commit LSN to a point in time when the commit happened. + * + * @param lsn - LSN of the commit + * @return time when the commit was recorded into the database log + * @throws SQLException + */ + public Instant timestampOfLsn(Lsn lsn) throws SQLException { + final String query = LSN_TO_TIMESTAMP; + + if (lsn.getBinary() == null) { + return null; + } + + Instant cachedInstant = lsnToInstantCache.get(lsn); + if (cachedInstant != null) { + return cachedInstant; + } + + return prepareQueryAndMap(query, statement -> { + statement.setBytes(1, lsn.getBinary()); + }, singleResultMapper(rs -> { + final Timestamp ts = rs.getTimestamp(1); + final Instant ret = (ts == null) ? null : ts.toInstant(); + LOGGER.trace("Timestamp of lsn {} is {}", lsn, ret); + if (ret != null) { + lsnToInstantCache.put(lsn, ret); + } + return ret; + }, "LSN to timestamp query must return exactly one value")); + } + + /** + * Creates an exclusive lock for a given table. + * + * @param tableId to be locked + * @throws SQLException + */ + public void lockTable(TableId tableId) throws SQLException { + final String lockTableStmt = LOCK_TABLE.replace(STATEMENTS_PLACEHOLDER, tableId.table()); + execute(lockTableStmt); + } + + private String cdcNameForTable(TableId tableId) { + return tableId.schema() + '_' + tableId.table(); + } + + private ResultSetMapper singleResultMapper(ResultSetExtractor extractor, String error) throws SQLException { + return (rs) -> { + if (rs.next()) { + final T ret = extractor.apply(rs); + if (!rs.next()) { + return ret; + } + } + throw new IllegalStateException(error); + }; + } + + public static class CdcEnabledTable { + private final String tableId; + private final String captureName; + private final Lsn fromLsn; + + private CdcEnabledTable(String tableId, String captureName, Lsn fromLsn) { + this.tableId = tableId; + this.captureName = captureName; + this.fromLsn = fromLsn; + } + + public String getTableId() { + return tableId; + } + + public String getCaptureName() { + return captureName; + } + + public Lsn getFromLsn() { + return fromLsn; + } + } + + public Set listOfChangeTables() throws SQLException { + final String query = GET_LIST_OF_CDC_ENABLED_TABLES; + + return queryAndMap(query, rs -> { + final Set changeTables = new HashSet<>(); + while (rs.next()) { + /** + changeTables.add( + new ChangeTable( + new TableId(realDatabaseName, rs.getString(1), rs.getString(2)), + rs.getString(3), + rs.getInt(4), + Lsn.valueOf(rs.getBytes(6)), + Lsn.valueOf(rs.getBytes(7)) + + ) + **/ + changeTables.add( + new ChangeTable( + new TableId("", rs.getString(1), rs.getString(2)), + rs.getString(4), + rs.getInt(9), + Lsn.valueOf(rs.getBytes(5)), + Lsn.valueOf(rs.getBytes(6)) + + )); + } + return changeTables; + }); + } + + public Set listOfNewChangeTables(Lsn fromLsn, Lsn toLsn) throws SQLException { + final String query = GET_LIST_OF_NEW_CDC_ENABLED_TABLES; + + return prepareQueryAndMap(query, + ps -> { + ps.setBytes(1, fromLsn.getBinary()); + ps.setBytes(2, toLsn.getBinary()); + }, + rs -> { + final Set changeTables = new HashSet<>(); + while (rs.next()) { + changeTables.add(new ChangeTable( + rs.getString(2), + rs.getInt(1), + Lsn.valueOf(rs.getBytes(3)), + Lsn.valueOf(rs.getBytes(4)))); + } + return changeTables; + }); + } + + public Table getTableSchemaFromTable(ChangeTable changeTable) throws SQLException { + final DatabaseMetaData metadata = connection().getMetaData(); + + List columns = new ArrayList<>(); + try (ResultSet rs = metadata.getColumns( + null, + changeTable.getSourceTableId().schema(), + changeTable.getSourceTableId().table(), + null)) { + while (rs.next()) { + readTableColumn(rs, changeTable.getSourceTableId(), null).ifPresent(ce -> columns.add(ce.create())); + } + } + + final List pkColumnNames = readPrimaryKeyNames(metadata, changeTable.getSourceTableId()); + Collections.sort(columns); + return Table.editor() + .tableId(changeTable.getSourceTableId()) + .addColumns(columns) + .setPrimaryKeyNames(pkColumnNames) + .create(); + } + + public Table getTableSchemaFromChangeTable(ChangeTable changeTable) throws SQLException { + final DatabaseMetaData metadata = connection().getMetaData(); + final TableId changeTableId = changeTable.getChangeTableId(); + + List columnEditors = new ArrayList<>(); + try (ResultSet rs = metadata.getColumns(null, changeTableId.schema(), changeTableId.table(), null)) { + while (rs.next()) { + readTableColumn(rs, changeTableId, null).ifPresent(columnEditors::add); + } + } + + // The first 5 columns and the last column of the change table are CDC metadata + // final List columns = columnEditors.subList(CHANGE_TABLE_DATA_COLUMN_OFFSET, columnEditors.size() - 1).stream() + final List columns = columnEditors.subList(CHANGE_TABLE_DATA_COLUMN_OFFSET, columnEditors.size()).stream() + .map(c -> c.position(c.position() - CHANGE_TABLE_DATA_COLUMN_OFFSET).create()) + .collect(Collectors.toList()); + + final List pkColumnNames = new ArrayList<>(); + /** URB + prepareQuery(GET_LIST_OF_KEY_COLUMNS, ps -> ps.setInt(1, changeTable.getChangeTableObjectId()), rs -> { + while (rs.next()) { + pkColumnNames.add(rs.getString(2)); + } + }); + **/ + prepareQuery(GET_LIST_OF_KEY_COLUMNS, ps -> { + ps.setInt(1, changeTable.getChangeTableObjectId()); + ps.setInt(1, changeTable.getChangeTableObjectId()); + }, rs -> { + while (rs.next()) { + pkColumnNames.add(rs.getString(2)); + } + }); + Collections.sort(columns); + return Table.editor() + .tableId(changeTable.getSourceTableId()) + .addColumns(columns) + .setPrimaryKeyNames(pkColumnNames) + .create(); + } + + public synchronized void rollback() throws SQLException { + if (isConnected()) { + connection().rollback(); + } + } + + public String getNameOfChangeTable(String captureName) { + return captureName + "_CT"; + } + + public String getRealDatabaseName() { + return realDatabaseName; + } + + private String retrieveRealDatabaseName() { + try { + return queryAndMap( + GET_DATABASE_NAME, + singleResultMapper(rs -> rs.getString(1), "Could not retrieve database name")); + } + catch (SQLException e) { + throw new RuntimeException("Couldn't obtain database name", e); + } + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connector.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connector.java new file mode 100644 index 000000000..1bf36c6c1 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2Connector.java @@ -0,0 +1,63 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.source.SourceConnector; + +import io.debezium.annotation.ThreadSafe; + +/** + * The main connector class used to instantiate configuration and execution classes + * + * @author Jiri Pechanec, Luis Garcés-Erice + * + */ + +@ThreadSafe +public class Db2Connector extends SourceConnector { + + private Map properties; + + @Override + public String version() { + return Module.version(); + } + + @Override + public void start(Map props) { + this.properties = Collections.unmodifiableMap(new HashMap<>(props)); + } + + @Override + public Class taskClass() { + return Db2ConnectorTask.class; + } + + @Override + public List> taskConfigs(int maxTasks) { + if (maxTasks > 1) { + throw new IllegalArgumentException("Only a single connector task may be started"); + } + + return Collections.singletonList(properties); + } + + @Override + public void stop() { + } + + @Override + public ConfigDef config() { + return Db2ConnectorConfig.configDef(); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorConfig.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorConfig.java new file mode 100644 index 000000000..69afe5f23 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorConfig.java @@ -0,0 +1,375 @@ +/* + * 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.db2; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.function.Predicate; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Width; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.config.Configuration; +import io.debezium.config.EnumeratedValue; +import io.debezium.config.Field; +import io.debezium.connector.AbstractSourceInfo; +import io.debezium.connector.SourceInfoStructMaker; +import io.debezium.document.Document; +import io.debezium.function.Predicates; +import io.debezium.heartbeat.Heartbeat; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.relational.ColumnId; +import io.debezium.relational.HistorizedRelationalDatabaseConnectorConfig; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.Tables.ColumnNameFilter; +import io.debezium.relational.Tables.TableFilter; +import io.debezium.relational.history.HistoryRecordComparator; +import io.debezium.relational.history.KafkaDatabaseHistory; + +/** + * The list of configuration options for DB2 connector + * + * @author Jiri Pechanec + */ +public class Db2ConnectorConfig extends HistorizedRelationalDatabaseConnectorConfig { + + /** + * The set of predefined SnapshotMode options or aliases. + */ + public static enum SnapshotMode implements EnumeratedValue { + + /** + * Perform a snapshot of data and schema upon initial startup of a connector. + */ + INITIAL("initial", true), + + /** + * Perform a snapshot of the schema but no data upon initial startup of a connector. + */ + INITIAL_SCHEMA_ONLY("initial_schema_only", false); + + private final String value; + private final boolean includeData; + + private SnapshotMode(String value, boolean includeData) { + this.value = value; + this.includeData = includeData; + } + + @Override + public String getValue() { + return value; + } + + /** + * Whether this snapshotting mode should include the actual data or just the + * schema of captured tables. + */ + public boolean includeData() { + return includeData; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static SnapshotMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + + for (SnapshotMode option : SnapshotMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is invalid + */ + public static SnapshotMode parse(String value, String defaultValue) { + SnapshotMode mode = parse(value); + + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + + return mode; + } + } + + /** + * The set of predefined snapshot isolation mode options. + */ + public static enum SnapshotIsolationMode implements EnumeratedValue { + + /** + * This mode will block all reads and writes for the entire duration of the snapshot. + * + * The connector will execute {@code SELECT * FROM .. WITH (TABLOCKX)} + */ + EXCLUSIVE("exclusive"), + + /** + * This mode uses SNAPSHOT isolation level. This way reads and writes are not blocked for the entire duration + * of the snapshot. Snapshot consistency is guaranteed as long as DDL statements are not executed at the time. + */ + SNAPSHOT("snapshot"), + + /** + * This mode uses REPEATABLE READ isolation level. This mode will avoid taking any table + * locks during the snapshot process, except schema snapshot phase where exclusive table + * locks are acquired for a short period. Since phantom reads can occur, it does not fully + * guarantee consistency. + */ + REPEATABLE_READ("repeatable_read"), + + /** + * This mode uses READ UNCOMMITTED isolation level. This mode takes neither table locks nor row-level locks + * during the snapshot process. This way other transactions are not affected by initial snapshot process. + * However, snapshot consistency is not guaranteed. + */ + READ_UNCOMMITTED("read_uncommitted"); + + private final String value; + + private SnapshotIsolationMode(String value) { + this.value = value; + } + + @Override + public String getValue() { + return value; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @return the matching option, or null if no match is found + */ + public static SnapshotIsolationMode parse(String value) { + if (value == null) { + return null; + } + value = value.trim(); + for (SnapshotIsolationMode option : SnapshotIsolationMode.values()) { + if (option.getValue().equalsIgnoreCase(value)) { + return option; + } + } + return null; + } + + /** + * Determine if the supplied value is one of the predefined options. + * + * @param value the configuration property value; may not be null + * @param defaultValue the default value; may be null + * @return the matching option, or null if no match is found and the non-null default is invalid + */ + public static SnapshotIsolationMode parse(String value, String defaultValue) { + SnapshotIsolationMode mode = parse(value); + if (mode == null && defaultValue != null) { + mode = parse(defaultValue); + } + return mode; + } + } + + public static final Field SERVER_NAME = RelationalDatabaseConnectorConfig.SERVER_NAME + .withValidation(CommonConnectorConfig::validateServerNameIsDifferentFromHistoryTopicName); + + public static final Field DATABASE_NAME = Field.create(DATABASE_CONFIG_PREFIX + JdbcConfiguration.DATABASE) + .withDisplayName("Database name") + .withType(Type.STRING) + .withWidth(Width.MEDIUM) + .withImportance(Importance.HIGH) + .withValidation(Field::isRequired) + .withDescription("The name of the database the connector should be monitoring."); + + public static final Field SNAPSHOT_MODE = Field.create("snapshot.mode") + .withDisplayName("Snapshot mode") + .withEnum(SnapshotMode.class, SnapshotMode.INITIAL) + .withWidth(Width.SHORT) + .withImportance(Importance.LOW) + .withDescription("The criteria for running a snapshot upon startup of the connector. " + + "Options include: " + + "'initial' (the default) to specify the connector should run a snapshot only when no offsets are available for the logical server name; " + + "'initial_schema_only' to specify the connector should run a snapshot of the schema when no offsets are available for the logical server name. "); + + public static final Field SNAPSHOT_ISOLATION_MODE = Field.create("snapshot.isolation.mode") + .withDisplayName("Snapshot isolation mode") + .withEnum(SnapshotIsolationMode.class, SnapshotIsolationMode.REPEATABLE_READ) + .withWidth(Width.SHORT) + .withImportance(Importance.LOW) + .withDescription("Controls which transaction isolation level is used and how long the connector locks the monitored tables. " + + "The default is '" + SnapshotIsolationMode.REPEATABLE_READ.getValue() + + "', which means that repeatable read isolation level is used. In addition, exclusive locks are taken only during schema snapshot. " + + "Using a value of '" + SnapshotIsolationMode.EXCLUSIVE.getValue() + + "' ensures that the connector holds the exclusive lock (and thus prevents any reads and updates) for all monitored tables during the entire snapshot duration. " + + "When '" + SnapshotIsolationMode.SNAPSHOT.getValue() + + "' is specified, connector runs the initial snapshot in SNAPSHOT isolation level, which guarantees snapshot consistency. In addition, neither table nor row-level locks are held. " + + "In '" + SnapshotIsolationMode.READ_UNCOMMITTED.getValue() + + "' mode neither table nor row-level locks are acquired, but connector does not guarantee snapshot consistency."); + + /** + * The set of {@link Field}s defined as part of this configuration. + */ + public static Field.Set ALL_FIELDS = Field.setOf( + SERVER_NAME, + DATABASE_NAME, + SNAPSHOT_MODE, + RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, + HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY, + RelationalDatabaseConnectorConfig.TABLE_WHITELIST, + RelationalDatabaseConnectorConfig.TABLE_BLACKLIST, + RelationalDatabaseConnectorConfig.TABLE_IGNORE_BUILTIN, + RelationalDatabaseConnectorConfig.COLUMN_BLACKLIST, + RelationalDatabaseConnectorConfig.DECIMAL_HANDLING_MODE, + RelationalDatabaseConnectorConfig.TIME_PRECISION_MODE, + CommonConnectorConfig.POLL_INTERVAL_MS, + CommonConnectorConfig.MAX_BATCH_SIZE, + CommonConnectorConfig.MAX_QUEUE_SIZE, + CommonConnectorConfig.SNAPSHOT_DELAY_MS, + CommonConnectorConfig.SNAPSHOT_FETCH_SIZE, + Heartbeat.HEARTBEAT_INTERVAL, Heartbeat.HEARTBEAT_TOPICS_PREFIX, + CommonConnectorConfig.SOURCE_STRUCT_MAKER_VERSION); + + public static ConfigDef configDef() { + ConfigDef config = new ConfigDef(); + + Field.group(config, "DB2 Server", SERVER_NAME, DATABASE_NAME, SNAPSHOT_MODE); + Field.group(config, "History Storage", KafkaDatabaseHistory.BOOTSTRAP_SERVERS, + KafkaDatabaseHistory.TOPIC, KafkaDatabaseHistory.RECOVERY_POLL_ATTEMPTS, + KafkaDatabaseHistory.RECOVERY_POLL_INTERVAL_MS, HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY); + Field.group(config, "Events", RelationalDatabaseConnectorConfig.TABLE_WHITELIST, + RelationalDatabaseConnectorConfig.TABLE_BLACKLIST, + RelationalDatabaseConnectorConfig.COLUMN_BLACKLIST, + RelationalDatabaseConnectorConfig.SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE, + RelationalDatabaseConnectorConfig.TABLE_IGNORE_BUILTIN, + Heartbeat.HEARTBEAT_INTERVAL, Heartbeat.HEARTBEAT_TOPICS_PREFIX, + CommonConnectorConfig.SOURCE_STRUCT_MAKER_VERSION); + Field.group(config, "Connector", CommonConnectorConfig.POLL_INTERVAL_MS, CommonConnectorConfig.MAX_BATCH_SIZE, + CommonConnectorConfig.MAX_QUEUE_SIZE, CommonConnectorConfig.SNAPSHOT_DELAY_MS, CommonConnectorConfig.SNAPSHOT_FETCH_SIZE, + RelationalDatabaseConnectorConfig.DECIMAL_HANDLING_MODE, RelationalDatabaseConnectorConfig.TIME_PRECISION_MODE); + + return config; + } + + private final String databaseName; + private final SnapshotMode snapshotMode; + private final SnapshotIsolationMode snapshotIsolationMode; + private final ColumnNameFilter columnFilter; + + public Db2ConnectorConfig(Configuration config) { + super(config, config.getString(SERVER_NAME), new SystemTablesPredicate(), x -> x.schema() + "." + x.table(), false); + + this.databaseName = config.getString(DATABASE_NAME); + this.snapshotMode = SnapshotMode.parse(config.getString(SNAPSHOT_MODE), SNAPSHOT_MODE.defaultValueAsString()); + this.snapshotIsolationMode = SnapshotIsolationMode.parse(config.getString(SNAPSHOT_ISOLATION_MODE), SNAPSHOT_ISOLATION_MODE.defaultValueAsString()); + this.columnFilter = getColumnNameFilter(config.getString(RelationalDatabaseConnectorConfig.COLUMN_BLACKLIST)); + } + + private static ColumnNameFilter getColumnNameFilter(String excludedColumnPatterns) { + return new ColumnNameFilter() { + + Predicate delegate = Predicates.excludes(excludedColumnPatterns, ColumnId::toString); + + @Override + public boolean matches(String catalogName, String schemaName, String tableName, String columnName) { + // ignore database name as it's not relevant here + return delegate.test(new ColumnId(new TableId(null, schemaName, tableName), columnName)); + } + }; + } + + public String getDatabaseName() { + return databaseName; + } + + public SnapshotIsolationMode getSnapshotIsolationMode() { + return this.snapshotIsolationMode; + } + + public SnapshotMode getSnapshotMode() { + return snapshotMode; + } + + public ColumnNameFilter getColumnFilter() { + return columnFilter; + } + + @Override + protected SourceInfoStructMaker getSourceInfoStructMaker(Version version) { + return new Db2SourceInfoStructMaker(Module.name(), Module.version(), this); + } + + private static class SystemTablesPredicate implements TableFilter { + + @Override + public boolean isIncluded(TableId t) { + return !(t.table().toLowerCase().startsWith("ibmsnap_") || + t.schema().toUpperCase().startsWith("ASNCDC") || + t.schema().toUpperCase().startsWith("SYSTOOLS") || + t.table().toLowerCase().startsWith("ibmqrep_")); + + } + } + + @Override + protected HistoryRecordComparator getHistoryRecordComparator() { + return new HistoryRecordComparator() { + @Override + protected boolean isPositionAtOrBefore(Document recorded, Document desired) { + return Lsn.valueOf(recorded.getString(SourceInfo.CHANGE_LSN_KEY)) + .compareTo(Lsn.valueOf(desired.getString(SourceInfo.CHANGE_LSN_KEY))) < 1; + } + }; + } + + @Override + public String getContextName() { + return Module.contextName(); + } + + /** + * Returns any SELECT overrides, if present. + */ + @Override + public Map getSnapshotSelectOverridesByTable() { + String tableList = getConfig().getString(SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE); + + if (tableList == null) { + return Collections.emptyMap(); + } + + Map snapshotSelectOverridesByTable = new HashMap<>(); + + for (String table : tableList.split(",")) { + snapshotSelectOverridesByTable.put( + TableId.parse(table, false), + getConfig().getString(SNAPSHOT_SELECT_STATEMENT_OVERRIDES_BY_TABLE + "." + table)); + } + + return Collections.unmodifiableMap(snapshotSelectOverridesByTable); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorTask.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorTask.java new file mode 100644 index 000000000..2e3a17f15 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ConnectorTask.java @@ -0,0 +1,241 @@ +/* + * 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.db2; + +import java.sql.SQLException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; + +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.config.Configuration; +import io.debezium.config.Field; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.common.BaseSourceTask; +import io.debezium.pipeline.ChangeEventSourceCoordinator; +import io.debezium.pipeline.DataChangeEvent; +import io.debezium.pipeline.ErrorHandler; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.HistorizedRelationalDatabaseConnectorConfig; +import io.debezium.relational.TableId; +import io.debezium.relational.history.DatabaseHistory; +import io.debezium.schema.TopicSelector; +import io.debezium.util.Clock; +import io.debezium.util.SchemaNameAdjuster; + +/** + * The main task executing streaming from DB2. + * Responsible for lifecycle management the streaming code. + * + * @author Jiri Pechanec + * + */ +public class Db2ConnectorTask extends BaseSourceTask { + + private static final Logger LOGGER = LoggerFactory.getLogger(Db2ConnectorTask.class); + private static final String CONTEXT_NAME = "db2-server-connector-task"; + + private static enum State { + RUNNING, + STOPPED; + } + + private final AtomicReference state = new AtomicReference(State.STOPPED); + + private volatile Db2TaskContext taskContext; + private volatile ChangeEventQueue queue; + private volatile Db2Connection dataConnection; + private volatile Db2Connection metadataConnection; + private volatile ChangeEventSourceCoordinator coordinator; + private volatile ErrorHandler errorHandler; + private volatile Db2DatabaseSchema schema; + private volatile Map lastOffset; + + @Override + public String version() { + return Module.version(); + } + + @Override + public void start(Configuration config) { + if (!state.compareAndSet(State.STOPPED, State.RUNNING)) { + LOGGER.info("Connector has already been started"); + return; + } + + final Db2ConnectorConfig connectorConfig = new Db2ConnectorConfig(config); + final TopicSelector topicSelector = Db2TopicSelector.defaultSelector(connectorConfig); + final SchemaNameAdjuster schemaNameAdjuster = SchemaNameAdjuster.create(LOGGER); + + // By default do not load whole result sets into memory + config = config.edit() + .withDefault("database.responseBuffering", "adaptive") + .withDefault("database.fetchSize", 10_000) + .build(); + + final Configuration jdbcConfig = config.filter( + x -> !(x.startsWith(DatabaseHistory.CONFIGURATION_FIELD_PREFIX_STRING) || x.equals(HistorizedRelationalDatabaseConnectorConfig.DATABASE_HISTORY.name()))) + .subset("database.", true); + dataConnection = new Db2Connection(jdbcConfig); + metadataConnection = new Db2Connection(jdbcConfig); + try { + dataConnection.setAutoCommit(false); + } + catch (SQLException e) { + throw new ConnectException(e); + } + this.schema = new Db2DatabaseSchema(connectorConfig, schemaNameAdjuster, topicSelector, dataConnection); + this.schema.initializeStorage(); + + final OffsetContext previousOffset = getPreviousOffset(new Db2OffsetContext.Loader(connectorConfig)); + if (previousOffset != null) { + schema.recover(previousOffset); + } + + taskContext = new Db2TaskContext(connectorConfig, schema); + + final Clock clock = Clock.system(); + + // Set up the task record queue ... + this.queue = new ChangeEventQueue.Builder() + .pollInterval(connectorConfig.getPollInterval()) + .maxBatchSize(connectorConfig.getMaxBatchSize()) + .maxQueueSize(connectorConfig.getMaxQueueSize()) + .loggingContextSupplier(() -> taskContext.configureLoggingContext(CONTEXT_NAME)) + .build(); + + errorHandler = new ErrorHandler(Db2Connector.class, connectorConfig.getLogicalName(), queue, this::cleanupResources); + + final EventDispatcher dispatcher = new EventDispatcher<>( + connectorConfig, + topicSelector, + schema, + queue, + connectorConfig.getTableFilters().dataCollectionFilter(), + DataChangeEvent::new); + + coordinator = new ChangeEventSourceCoordinator( + previousOffset, + errorHandler, + Db2Connector.class, + connectorConfig.getLogicalName(), + new Db2ChangeEventSourceFactory(connectorConfig, dataConnection, metadataConnection, errorHandler, dispatcher, clock, schema), + dispatcher, + schema); + + coordinator.start(taskContext, this.queue, new Db2EventMetadataProvider()); + } + + /** + * Loads the connector's persistent offset (if present) via the given loader. + */ + @Override + protected OffsetContext getPreviousOffset(OffsetContext.Loader loader) { + Map partition = loader.getPartition(); + + Map previousOffset = context.offsetStorageReader() + .offsets(Collections.singleton(partition)) + .get(partition); + + if (previousOffset != null) { + OffsetContext offsetContext = loader.load(previousOffset); + LOGGER.info("Found previous offset {}", offsetContext); + return offsetContext; + } + else { + return null; + } + } + + @Override + public List poll() throws InterruptedException { + final List records = queue.poll(); + + final List sourceRecords = records.stream() + .map(DataChangeEvent::getRecord) + .collect(Collectors.toList()); + + if (!sourceRecords.isEmpty()) { + this.lastOffset = sourceRecords.get(sourceRecords.size() - 1).sourceOffset(); + } + + return sourceRecords; + } + + @Override + public void commit() throws InterruptedException { + if (coordinator != null) { + coordinator.commitOffset(lastOffset); + } + } + + @Override + public void stop() { + cleanupResources(); + } + + private void cleanupResources() { + if (!state.compareAndSet(State.RUNNING, State.STOPPED)) { + LOGGER.info("Connector has already been stopped"); + return; + } + + try { + if (coordinator != null) { + coordinator.stop(); + } + } + catch (InterruptedException e) { + Thread.interrupted(); + LOGGER.error("Interrupted while stopping coordinator", e); + throw new ConnectException("Interrupted while stopping coordinator, failing the task"); + } + + try { + if (errorHandler != null) { + errorHandler.stop(); + } + } + catch (InterruptedException e) { + Thread.interrupted(); + LOGGER.error("Interrupted while stopping", e); + } + + try { + if (dataConnection != null) { + dataConnection.close(); + } + } + catch (SQLException e) { + LOGGER.error("Exception while closing JDBC connection", e); + } + + try { + if (metadataConnection != null) { + metadataConnection.close(); + } + } + catch (SQLException e) { + LOGGER.error("Exception while closing JDBC metadata connection", e); + } + + if (schema != null) { + schema.close(); + } + } + + @Override + protected Iterable getAllConfigurationFields() { + return Db2ConnectorConfig.ALL_FIELDS; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2DatabaseSchema.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2DatabaseSchema.java new file mode 100644 index 000000000..0e630e93c --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2DatabaseSchema.java @@ -0,0 +1,67 @@ +/* + * 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.db2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.relational.HistorizedRelationalDatabaseSchema; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.relational.TableSchemaBuilder; +import io.debezium.relational.ddl.DdlParser; +import io.debezium.relational.history.TableChanges; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType; +import io.debezium.schema.TopicSelector; +import io.debezium.util.SchemaNameAdjuster; + +/** + * Logical representation of DB2 schema. + * + * @author Jiri Pechanec + */ +public class Db2DatabaseSchema extends HistorizedRelationalDatabaseSchema { + + private static final Logger LOGGER = LoggerFactory.getLogger(Db2DatabaseSchema.class); + + public Db2DatabaseSchema(Db2ConnectorConfig connectorConfig, SchemaNameAdjuster schemaNameAdjuster, TopicSelector topicSelector, Db2Connection connection) { + super(connectorConfig, topicSelector, connectorConfig.getTableFilters().dataCollectionFilter(), connectorConfig.getColumnFilter(), + new TableSchemaBuilder( + new Db2ValueConverters(connectorConfig.getDecimalMode(), connectorConfig.getTemporalPrecisionMode()), + schemaNameAdjuster, + connectorConfig.getSourceInfoStructMaker().schema(), + connectorConfig.getSanitizeFieldNames()), + false, connectorConfig.getKeyMapper()); + } + + @Override + public void applySchemaChange(SchemaChangeEvent schemaChange) { + LOGGER.debug("Applying schema change event {}", schemaChange); + + // just a single table per DDL event for DB2 + Table table = schemaChange.getTables().iterator().next(); + buildAndRegisterSchema(table); + tables().overwriteTable(table); + + TableChanges tableChanges = null; + if (schemaChange.getType() == SchemaChangeEventType.CREATE) { + tableChanges = new TableChanges(); + tableChanges.create(table); + } + else if (schemaChange.getType() == SchemaChangeEventType.ALTER) { + tableChanges = new TableChanges(); + tableChanges.alter(table); + } + + record(schemaChange, tableChanges); + } + + @Override + protected DdlParser getDdlParser() { + return null; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2EventMetadataProvider.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2EventMetadataProvider.java new file mode 100644 index 000000000..83a2a7867 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2EventMetadataProvider.java @@ -0,0 +1,59 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +import java.time.Instant; +import java.util.Map; + +import org.apache.kafka.connect.data.Struct; + +import io.debezium.data.Envelope; +import io.debezium.pipeline.source.spi.EventMetadataProvider; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.schema.DataCollectionId; +import io.debezium.util.Collect; + +class Db2EventMetadataProvider implements EventMetadataProvider { + + @Override + public Instant getEventTimestamp(DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + final Long timestamp = sourceInfo.getInt64(SourceInfo.TIMESTAMP_KEY); + return timestamp == null ? null : Instant.ofEpochMilli(timestamp); + } + + @Override + public Map getEventSourcePosition(DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + return Collect.hashMapOf( + SourceInfo.COMMIT_LSN_KEY, sourceInfo.getString(SourceInfo.COMMIT_LSN_KEY), + SourceInfo.CHANGE_LSN_KEY, sourceInfo.getString(SourceInfo.CHANGE_LSN_KEY)); + } + + @Override + public String getTransactionId(DataCollectionId source, OffsetContext offset, Object key, Struct value) { + if (value == null) { + return null; + } + final Struct sourceInfo = value.getStruct(Envelope.FieldName.SOURCE); + if (source == null) { + return null; + } + return sourceInfo.getString(SourceInfo.COMMIT_LSN_KEY); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2OffsetContext.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2OffsetContext.java new file mode 100644 index 000000000..40acfad4d --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2OffsetContext.java @@ -0,0 +1,186 @@ +/* + * 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.db2; + +import java.time.Instant; +import java.util.Collections; +import java.util.Map; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import io.debezium.connector.SnapshotRecord; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.TableId; +import io.debezium.schema.DataCollectionId; +import io.debezium.util.Collect; + +public class Db2OffsetContext implements OffsetContext { + + private static final String SERVER_PARTITION_KEY = "server"; + private static final String SNAPSHOT_COMPLETED_KEY = "snapshot_completed"; + private static final String EVENT_SERIAL_NO_KEY = "event_serial_no"; + + private final Schema sourceInfoSchema; + private final SourceInfo sourceInfo; + private final Map partition; + private boolean snapshotCompleted; + + /** + * The index of the current event within the current transaction. + */ + private long eventSerialNo; + + public Db2OffsetContext(Db2ConnectorConfig connectorConfig, TxLogPosition position, boolean snapshot, boolean snapshotCompleted, long eventSerialNo) { + partition = Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName()); + sourceInfo = new SourceInfo(connectorConfig); + + sourceInfo.setCommitLsn(position.getCommitLsn()); + sourceInfo.setChangeLsn(position.getInTxLsn()); + sourceInfoSchema = sourceInfo.schema(); + + this.snapshotCompleted = snapshotCompleted; + if (this.snapshotCompleted) { + postSnapshotCompletion(); + } + else { + sourceInfo.setSnapshot(snapshot ? SnapshotRecord.TRUE : SnapshotRecord.FALSE); + } + this.eventSerialNo = eventSerialNo; + } + + public Db2OffsetContext(Db2ConnectorConfig connectorConfig, TxLogPosition position, boolean snapshot, boolean snapshotCompleted) { + this(connectorConfig, position, snapshot, snapshotCompleted, 1); + } + + @Override + public Map getPartition() { + return partition; + } + + @Override + public Map getOffset() { + if (sourceInfo.isSnapshot()) { + return Collect.hashMapOf( + SourceInfo.SNAPSHOT_KEY, true, + SNAPSHOT_COMPLETED_KEY, snapshotCompleted, + SourceInfo.COMMIT_LSN_KEY, sourceInfo.getCommitLsn().toString()); + } + else { + return Collect.hashMapOf( + SourceInfo.COMMIT_LSN_KEY, sourceInfo.getCommitLsn().toString(), + SourceInfo.CHANGE_LSN_KEY, + sourceInfo.getChangeLsn() == null ? null : sourceInfo.getChangeLsn().toString(), + EVENT_SERIAL_NO_KEY, eventSerialNo); + } + } + + @Override + public Schema getSourceInfoSchema() { + return sourceInfoSchema; + } + + @Override + public Struct getSourceInfo() { + return sourceInfo.struct(); + } + + public TxLogPosition getChangePosition() { + return TxLogPosition.valueOf(sourceInfo.getCommitLsn(), sourceInfo.getChangeLsn()); + } + + public long getEventSerialNo() { + return eventSerialNo; + } + + public void setChangePosition(TxLogPosition position, int eventCount) { + if (getChangePosition().equals(position)) { + eventSerialNo += eventCount; + } + else { + eventSerialNo = eventCount; + } + sourceInfo.setCommitLsn(position.getCommitLsn()); + sourceInfo.setChangeLsn(position.getInTxLsn()); + } + + @Override + public boolean isSnapshotRunning() { + return sourceInfo.isSnapshot() && !snapshotCompleted; + } + + public boolean isSnapshotCompleted() { + return snapshotCompleted; + } + + @Override + public void preSnapshotStart() { + sourceInfo.setSnapshot(SnapshotRecord.TRUE); + snapshotCompleted = false; + } + + @Override + public void preSnapshotCompletion() { + snapshotCompleted = true; + } + + @Override + public void postSnapshotCompletion() { + sourceInfo.setSnapshot(SnapshotRecord.FALSE); + } + + public static class Loader implements OffsetContext.Loader { + + private final Db2ConnectorConfig connectorConfig; + + public Loader(Db2ConnectorConfig connectorConfig) { + this.connectorConfig = connectorConfig; + } + + @Override + public Map getPartition() { + return Collections.singletonMap(SERVER_PARTITION_KEY, connectorConfig.getLogicalName()); + } + + @Override + public OffsetContext load(Map offset) { + final Lsn changeLsn = Lsn.valueOf((String) offset.get(SourceInfo.CHANGE_LSN_KEY)); + final Lsn commitLsn = Lsn.valueOf((String) offset.get(SourceInfo.COMMIT_LSN_KEY)); + boolean snapshot = Boolean.TRUE.equals(offset.get(SourceInfo.SNAPSHOT_KEY)); + boolean snapshotCompleted = Boolean.TRUE.equals(offset.get(SNAPSHOT_COMPLETED_KEY)); + + // only introduced in 0.10.Beta1, so it might be not present when upgrading from earlier versions + Long eventSerialNo = ((Long) offset.get(EVENT_SERIAL_NO_KEY)); + if (eventSerialNo == null) { + eventSerialNo = Long.valueOf(0); + } + + return new Db2OffsetContext(connectorConfig, TxLogPosition.valueOf(commitLsn, changeLsn), snapshot, snapshotCompleted, eventSerialNo); + } + } + + @Override + public String toString() { + return "Db2OffsetContext [" + + "sourceInfoSchema=" + sourceInfoSchema + + ", sourceInfo=" + sourceInfo + + ", partition=" + partition + + ", snapshotCompleted=" + snapshotCompleted + + ", eventSerialNo=" + eventSerialNo + + "]"; + } + + @Override + public void markLastSnapshotRecord() { + sourceInfo.setSnapshot(SnapshotRecord.LAST); + } + + @Override + public void event(DataCollectionId tableId, Instant timestamp) { + sourceInfo.setSourceTime(timestamp); + sourceInfo.setTableId((TableId) tableId); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SchemaChangeEventEmitter.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SchemaChangeEventEmitter.java new file mode 100644 index 000000000..0eb30929b --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SchemaChangeEventEmitter.java @@ -0,0 +1,51 @@ +/* + * 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.db2; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.pipeline.spi.SchemaChangeEventEmitter; +import io.debezium.relational.Table; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType; + +/** + * {@link SchemaChangeEventEmitter} implementation based on DB2. + * + * @author Jiri Pechanec + */ +public class Db2SchemaChangeEventEmitter implements SchemaChangeEventEmitter { + + private static final Logger LOGGER = LoggerFactory.getLogger(Db2SchemaChangeEventEmitter.class); + + private final Db2OffsetContext offsetContext; + private final ChangeTable changeTable; + private final Table tableSchema; + private final SchemaChangeEventType eventType; + + public Db2SchemaChangeEventEmitter(Db2OffsetContext offsetContext, ChangeTable changeTable, Table tableSchema, SchemaChangeEventType eventType) { + this.offsetContext = offsetContext; + this.changeTable = changeTable; + this.tableSchema = tableSchema; + this.eventType = eventType; + } + + @Override + public void emitSchemaChangeEvent(Receiver receiver) throws InterruptedException { + final SchemaChangeEvent event = new SchemaChangeEvent( + offsetContext.getPartition(), + offsetContext.getOffset(), + changeTable.getSourceTableId().catalog(), + changeTable.getSourceTableId().schema(), + "N/A", + tableSchema, + eventType, + false); + + receiver.schemaChangeEvent(event); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SnapshotChangeEventSource.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SnapshotChangeEventSource.java new file mode 100644 index 000000000..76d66eb70 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SnapshotChangeEventSource.java @@ -0,0 +1,228 @@ +/* + * 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.db2; + +import java.sql.Connection; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Savepoint; +import java.sql.Statement; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.connector.db2.Db2ConnectorConfig.SnapshotIsolationMode; +import io.debezium.pipeline.EventDispatcher; +import io.debezium.pipeline.source.spi.SnapshotProgressListener; +import io.debezium.pipeline.spi.OffsetContext; +import io.debezium.relational.RelationalSnapshotChangeEventSource; +import io.debezium.relational.Table; +import io.debezium.relational.TableId; +import io.debezium.schema.SchemaChangeEvent; +import io.debezium.schema.SchemaChangeEvent.SchemaChangeEventType; +import io.debezium.util.Clock; + +public class Db2SnapshotChangeEventSource extends RelationalSnapshotChangeEventSource { + + private static final Logger LOGGER = LoggerFactory.getLogger(Db2SnapshotChangeEventSource.class); + + /** + * Code 4096 corresponds to SNAPSHOT isolation level, which is not a part of the standard but SQL Server specific. + * Need to port this to DB2 (lga-zurich). + */ + private static final int TRANSACTION_SNAPSHOT = 4096; + + private final Db2ConnectorConfig connectorConfig; + private final Db2Connection jdbcConnection; + + public Db2SnapshotChangeEventSource(Db2ConnectorConfig connectorConfig, Db2OffsetContext previousOffset, Db2Connection jdbcConnection, Db2DatabaseSchema schema, + EventDispatcher dispatcher, Clock clock, SnapshotProgressListener snapshotProgressListener) { + super(connectorConfig, previousOffset, jdbcConnection, schema, dispatcher, clock, snapshotProgressListener); + this.connectorConfig = connectorConfig; + this.jdbcConnection = jdbcConnection; + } + + @Override + protected SnapshottingTask getSnapshottingTask(OffsetContext previousOffset) { + boolean snapshotSchema = true; + boolean snapshotData = true; + + // found a previous offset and the earlier snapshot has completed + if (previousOffset != null && !previousOffset.isSnapshotRunning()) { + LOGGER.info("A previous offset indicating a completed snapshot has been found. Neither schema nor data will be snapshotted."); + snapshotSchema = false; + snapshotData = false; + } + else { + LOGGER.info("No previous offset has been found"); + if (connectorConfig.getSnapshotMode().includeData()) { + LOGGER.info("According to the connector configuration both schema and data will be snapshotted"); + } + else { + LOGGER.info("According to the connector configuration only schema will be snapshotted"); + } + snapshotData = connectorConfig.getSnapshotMode().includeData(); + } + + return new SnapshottingTask(snapshotSchema, snapshotData); + } + + @Override + protected SnapshotContext prepare(ChangeEventSourceContext context) throws Exception { + return new Db2SnapshotContext(jdbcConnection.getRealDatabaseName()); + } + + @Override + protected void connectionCreated(SnapshotContext snapshotContext) throws Exception { + ((Db2SnapshotContext) snapshotContext).isolationLevelBeforeStart = jdbcConnection.connection().getTransactionIsolation(); + + if (connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.SNAPSHOT) { + // Terminate any transaction in progress so we can change the isolation level + jdbcConnection.connection().rollback(); + // With one exception, you can switch from one isolation level to another at any time during a transaction. + // The exception occurs when changing from any isolation level to SNAPSHOT isolation. + // That is why SNAPSHOT isolation level has to be set at the very beginning of the transaction. + jdbcConnection.connection().setTransactionIsolation(TRANSACTION_SNAPSHOT); + } + } + + @Override + protected Set getAllTableIds(SnapshotContext ctx) throws Exception { + return jdbcConnection.readTableNames(null, null, null, new String[]{ "TABLE" }); + } + + @Override + protected void lockTablesForSchemaSnapshot(ChangeEventSourceContext sourceContext, SnapshotContext snapshotContext) throws SQLException, InterruptedException { + if (connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.READ_UNCOMMITTED) { + jdbcConnection.connection().setTransactionIsolation(Connection.TRANSACTION_READ_UNCOMMITTED); + LOGGER.info("Schema locking was disabled in connector configuration"); + } + else if (connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.SNAPSHOT) { + // Snapshot transaction isolation level has already been set. + LOGGER.info("Schema locking was disabled in connector configuration"); + } + else if (connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.EXCLUSIVE + || connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.REPEATABLE_READ) { + jdbcConnection.connection().setTransactionIsolation(Connection.TRANSACTION_REPEATABLE_READ); + ((Db2SnapshotContext) snapshotContext).preSchemaSnapshotSavepoint = jdbcConnection.connection().setSavepoint("db2_schema_snapshot"); + + LOGGER.info("Executing schema locking"); + try (Statement statement = jdbcConnection.connection().createStatement(ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY)) { + for (TableId tableId : snapshotContext.capturedTables) { + if (!sourceContext.isRunning()) { + throw new InterruptedException("Interrupted while locking table " + tableId); + } + + LOGGER.info("Locking table {}", tableId); + + String query = String.format("SELECT * FROM %s.%s WHERE 0=1 WITH CS", tableId.schema(), tableId.table()); + statement.executeQuery(query).close(); + } + } + } + else { + throw new IllegalStateException("Unknown locking mode specified."); + } + } + + @Override + protected void releaseSchemaSnapshotLocks(SnapshotContext snapshotContext) throws SQLException { + // Exclusive mode: locks should be kept until the end of transaction. + // read_uncommitted mode; snapshot mode: no locks have been acquired. + if (connectorConfig.getSnapshotIsolationMode() == SnapshotIsolationMode.REPEATABLE_READ) { + jdbcConnection.connection().rollback(((Db2SnapshotContext) snapshotContext).preSchemaSnapshotSavepoint); + LOGGER.info("Schema locks released."); + } + } + + @Override + protected void determineSnapshotOffset(SnapshotContext ctx) throws Exception { + ctx.offset = new Db2OffsetContext( + connectorConfig, + TxLogPosition.valueOf(jdbcConnection.getMaxLsn()), + false, + false); + } + + @Override + protected void readTableStructure(ChangeEventSourceContext sourceContext, SnapshotContext snapshotContext) throws SQLException, InterruptedException { + Set schemas = snapshotContext.capturedTables.stream() + .map(TableId::schema) + .collect(Collectors.toSet()); + + // reading info only for the schemas we're interested in as per the set of captured tables; + // while the passed table name filter alone would skip all non-included tables, reading the schema + // would take much longer that way + for (String schema : schemas) { + if (!sourceContext.isRunning()) { + throw new InterruptedException("Interrupted while reading structure of schema " + schema); + } + + LOGGER.info("Reading structure of schema '{}'", schema); + /** + jdbcConnection.readSchema( + snapshotContext.tables, + snapshotContext.catalogName, + schema, + connectorConfig.getTableFilters().dataCollectionFilter(), + null, + false + ); + **/ + jdbcConnection.readSchema( + snapshotContext.tables, + null, + schema, + connectorConfig.getTableFilters().dataCollectionFilter(), + null, + false); + } + } + + @Override + protected SchemaChangeEvent getCreateTableEvent(SnapshotContext snapshotContext, Table table) throws SQLException { + return new SchemaChangeEvent(snapshotContext.offset.getPartition(), snapshotContext.offset.getOffset(), snapshotContext.catalogName, + table.id().schema(), null, table, SchemaChangeEventType.CREATE, true); + } + + @Override + protected void complete(SnapshotContext snapshotContext) { + try { + jdbcConnection.connection().setTransactionIsolation(((Db2SnapshotContext) snapshotContext).isolationLevelBeforeStart); + } + catch (SQLException e) { + throw new RuntimeException("Failed to set transaction isolation level.", e); + } + } + + /** + * Generate a valid db2 query string for the specified table + * + * @param tableId the table to generate a query for + * @return a valid query string + */ + @Override + protected Optional getSnapshotSelect(SnapshotContext snapshotContext, TableId tableId) { + return Optional.of(String.format("SELECT * FROM %s.%s", tableId.schema(), tableId.table())); + } + + /** + * Mutable context which is populated in the course of snapshotting. + */ + private static class Db2SnapshotContext extends SnapshotContext { + + private int isolationLevelBeforeStart; + private Savepoint preSchemaSnapshotSavepoint; + + public Db2SnapshotContext(String catalogName) throws SQLException { + super(catalogName); + } + } + +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SourceInfoStructMaker.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SourceInfoStructMaker.java new file mode 100644 index 000000000..46826d6b6 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2SourceInfoStructMaker.java @@ -0,0 +1,48 @@ +/* + * 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.db2; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; + +import io.debezium.config.CommonConnectorConfig; +import io.debezium.connector.AbstractSourceInfoStructMaker; + +public class Db2SourceInfoStructMaker extends AbstractSourceInfoStructMaker { + + private final Schema schema; + + public Db2SourceInfoStructMaker(String connector, String version, CommonConnectorConfig connectorConfig) { + super(connector, version, connectorConfig); + schema = commonSchemaBuilder() + .name("io.debezium.connector.db2.Source") + .field(SourceInfo.SCHEMA_NAME_KEY, Schema.STRING_SCHEMA) + .field(SourceInfo.TABLE_NAME_KEY, Schema.STRING_SCHEMA) + .field(SourceInfo.CHANGE_LSN_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .field(SourceInfo.COMMIT_LSN_KEY, Schema.OPTIONAL_STRING_SCHEMA) + .build(); + } + + @Override + public Schema schema() { + return schema; + } + + @Override + public Struct struct(SourceInfo sourceInfo) { + final Struct ret = super.commonStruct(sourceInfo) + .put(SourceInfo.SCHEMA_NAME_KEY, sourceInfo.getTableId().schema()) + .put(SourceInfo.TABLE_NAME_KEY, sourceInfo.getTableId().table()); + + if (sourceInfo.getChangeLsn() != null && sourceInfo.getChangeLsn().isAvailable()) { + ret.put(SourceInfo.CHANGE_LSN_KEY, sourceInfo.getChangeLsn().toString()); + } + if (sourceInfo.getCommitLsn() != null && sourceInfo.getCommitLsn().isAvailable()) { + ret.put(SourceInfo.COMMIT_LSN_KEY, sourceInfo.getCommitLsn().toString()); + } + return ret; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2StreamingChangeEventSource.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2StreamingChangeEventSource.java new file mode 100644 index 000000000..89604a21a --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2StreamingChangeEventSource.java @@ -0,0 +1,412 @@ +/* + * 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.db2; + +import java.sql.ResultSet; +import java.sql.SQLException; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +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.SchemaChangeEventType; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; + +/** + *

A {@link StreamingChangeEventSource} based on DB2 change data capture functionality. + * A main loop polls database DDL change and change data tables and turns them into change events.

+ * + *

The connector uses CDC functionality of DB2 that is implemented as as a process that monitors + * source table and write changes from the table into the change table.

+ * + *

The main loop keeps a pointer to the LSN of changes that were already processed. It queries all change + * tables and get result set of changes. It always finds the smallest LSN across all tables and the change + * is converted into the event message and sent downstream. The process repeats until all result sets are + * empty. The LSN is marked and the procedure repeats.

+ * + *

The schema changes detection follows the procedure recommended by DB2 CDC documentation. + * The database operator should create one more capture process (and table) when a table schema is updated. + * The code detects presence of two change tables for a single source table. It decides which table is the new one + * depending on LSNs stored in them. The loop streams changes from the older table till there are events in new + * table with the LSN larger than in the old one. Then the change table is switched and streaming is executed + * from the new one.

+ * + * @author Jiri Pechanec, Peter Urbanetz + */ +public class Db2StreamingChangeEventSource implements StreamingChangeEventSource { + + private static final int COL_COMMIT_LSN = 2; + private static final int COL_ROW_LSN = 3; + private static final int COL_OPERATION = 1; + private static final int COL_DATA = 5; + + private static final Pattern MISSING_CDC_FUNCTION_CHANGES_ERROR = Pattern.compile("Invalid object name 'cdc.fn_cdc_get_all_changes_(.*)'\\."); + + private static final Logger LOGGER = LoggerFactory.getLogger(Db2StreamingChangeEventSource.class); + + /** + * Connection used for reading CDC tables. + */ + private final Db2Connection dataConnection; + + /** + * A separate connection for retrieving timestamps; without it, adaptive + * buffering will not work. + * + * @see https://docs.microsoft.com/en-us/sql/connect/jdbc/using-adaptive-buffering?view=sql-server-2017#guidelines-for-using-adaptive-buffering + */ + private final Db2Connection metadataConnection; + + private final EventDispatcher dispatcher; + private final ErrorHandler errorHandler; + private final Clock clock; + private final Db2DatabaseSchema schema; + private final Db2OffsetContext offsetContext; + private final Duration pollInterval; + private final Db2ConnectorConfig connectorConfig; + + public Db2StreamingChangeEventSource(Db2ConnectorConfig connectorConfig, Db2OffsetContext offsetContext, Db2Connection dataConnection, + Db2Connection metadataConnection, EventDispatcher dispatcher, ErrorHandler errorHandler, Clock clock, + Db2DatabaseSchema schema) { + this.connectorConfig = connectorConfig; + this.dataConnection = dataConnection; + this.metadataConnection = metadataConnection; + this.dispatcher = dispatcher; + this.errorHandler = errorHandler; + this.clock = clock; + this.schema = schema; + this.offsetContext = offsetContext; + this.pollInterval = connectorConfig.getPollInterval(); + } + + @Override + public void execute(ChangeEventSourceContext context) throws InterruptedException { + final Metronome metronome = Metronome.sleeper(pollInterval, clock); + final Queue schemaChangeCheckpoints = new PriorityQueue<>((x, y) -> x.getStopLsn().compareTo(y.getStopLsn())); + try { + final AtomicReference tablesSlot = new AtomicReference(getCdcTablesToQuery()); + + final TxLogPosition lastProcessedPositionOnStart = offsetContext.getChangePosition(); + final long lastProcessedEventSerialNoOnStart = offsetContext.getEventSerialNo(); + LOGGER.info("Last position recorded in offsets is {}[{}]", lastProcessedPositionOnStart, lastProcessedEventSerialNoOnStart); + + TxLogPosition lastProcessedPosition = lastProcessedPositionOnStart; + + // LSN should be increased for the first run only immediately after snapshot completion + // otherwise we might skip an incomplete transaction after restart + boolean shouldIncreaseFromLsn = offsetContext.isSnapshotCompleted(); + while (context.isRunning()) { + final Lsn currentMaxLsn = dataConnection.getMaxLsn(); + + // Shouldn't happen if the agent is running, but it is better to guard against such situation + if (!currentMaxLsn.isAvailable()) { + LOGGER.warn("No maximum LSN recorded in the database; please ensure that the DB2 Agent is running"); + metronome.pause(); + continue; + } + // There is no change in the database + if (currentMaxLsn.equals(lastProcessedPosition.getCommitLsn()) && shouldIncreaseFromLsn) { + LOGGER.debug("No change in the database"); + metronome.pause(); + continue; + } + + // Reading interval is inclusive so we need to move LSN forward but not for first + // run as TX might not be streamed completely + final Lsn fromLsn = lastProcessedPosition.getCommitLsn().isAvailable() && shouldIncreaseFromLsn + ? dataConnection.incrementLsn(lastProcessedPosition.getCommitLsn()) + : lastProcessedPosition.getCommitLsn(); + shouldIncreaseFromLsn = true; + + while (!schemaChangeCheckpoints.isEmpty()) { + migrateTable(schemaChangeCheckpoints); + } + if (!dataConnection.listOfNewChangeTables(fromLsn, currentMaxLsn).isEmpty()) { + final ChangeTable[] tables = getCdcTablesToQuery(); + tablesSlot.set(tables); + for (ChangeTable table : tables) { + if (table.getStartLsn().isBetween(fromLsn, currentMaxLsn)) { + LOGGER.info("Schema will be changed for {}", table); + schemaChangeCheckpoints.add(table); + } + } + } + try { + dataConnection.getChangesForTables(tablesSlot.get(), fromLsn, currentMaxLsn, resultSets -> { + + long eventSerialNoInInitialTx = 1; + final int tableCount = resultSets.length; + final ChangeTablePointer[] changeTables = new ChangeTablePointer[tableCount]; + final ChangeTable[] tables = tablesSlot.get(); + + for (int i = 0; i < tableCount; i++) { + changeTables[i] = new ChangeTablePointer(tables[i], resultSets[i]); + changeTables[i].next(); + } + + for (;;) { + ChangeTablePointer tableWithSmallestLsn = null; + for (ChangeTablePointer changeTable : changeTables) { + if (changeTable.isCompleted()) { + continue; + } + if (tableWithSmallestLsn == null || changeTable.compareTo(tableWithSmallestLsn) < 0) { + tableWithSmallestLsn = changeTable; + } + } + if (tableWithSmallestLsn == null) { + // No more LSNs available + break; + } + + if (!(tableWithSmallestLsn.getChangePosition().isAvailable() && tableWithSmallestLsn.getChangePosition().getInTxLsn().isAvailable())) { + LOGGER.error("Skipping change {} as its LSN is NULL which is not expected", tableWithSmallestLsn); + tableWithSmallestLsn.next(); + continue; + } + // After restart for changes that were executed before the last committed offset + if (tableWithSmallestLsn.getChangePosition().compareTo(lastProcessedPositionOnStart) < 0) { + LOGGER.info("Skipping change {} as its position is smaller than the last recorded position {}", tableWithSmallestLsn, + lastProcessedPositionOnStart); + tableWithSmallestLsn.next(); + continue; + } + // After restart for change that was the last committed and operations in it before the last committed offset + if (tableWithSmallestLsn.getChangePosition().compareTo(lastProcessedPositionOnStart) == 0 + && eventSerialNoInInitialTx <= lastProcessedEventSerialNoOnStart) { + LOGGER.info("Skipping change {} as its order in the transaction {} is smaller than or equal to the last recorded operation {}[{}]", + tableWithSmallestLsn, eventSerialNoInInitialTx, lastProcessedPositionOnStart, lastProcessedEventSerialNoOnStart); + eventSerialNoInInitialTx++; + tableWithSmallestLsn.next(); + continue; + } + if (tableWithSmallestLsn.getChangeTable().getStopLsn().isAvailable() && + tableWithSmallestLsn.getChangeTable().getStopLsn().compareTo(tableWithSmallestLsn.getChangePosition().getCommitLsn()) <= 0) { + LOGGER.debug("Skipping table change {} as its stop LSN is smaller than the last recorded LSN {}", tableWithSmallestLsn, + tableWithSmallestLsn.getChangePosition()); + tableWithSmallestLsn.next(); + continue; + } + LOGGER.trace("Processing change {}", tableWithSmallestLsn); + if (!schemaChangeCheckpoints.isEmpty()) { + if (tableWithSmallestLsn.getChangePosition().getCommitLsn().compareTo(schemaChangeCheckpoints.peek().getStopLsn()) >= 0) { + migrateTable(schemaChangeCheckpoints); + } + } + final TableId tableId = tableWithSmallestLsn.getChangeTable().getSourceTableId(); + final int operation = tableWithSmallestLsn.getOperation(); + final Object[] data = tableWithSmallestLsn.getData(); + + // UPDATE consists of two consecutive events, first event contains + // the row before it was updated and the second the row after + // it was updated + int eventCount = 1; + if (operation == Db2ChangeRecordEmitter.OP_UPDATE_BEFORE) { + if (!tableWithSmallestLsn.next() || tableWithSmallestLsn.getOperation() != Db2ChangeRecordEmitter.OP_UPDATE_AFTER) { + throw new IllegalStateException("The update before event at " + tableWithSmallestLsn.getChangePosition() + " for table " + tableId + + " was not followed by after event.\n Please report this as a bug together with a events around given LSN."); + } + eventCount = 2; + } + final Object[] dataNext = (operation == Db2ChangeRecordEmitter.OP_UPDATE_BEFORE) ? tableWithSmallestLsn.getData() : null; + + offsetContext.setChangePosition(tableWithSmallestLsn.getChangePosition(), eventCount); + offsetContext.event(tableWithSmallestLsn.getChangeTable().getSourceTableId(), + metadataConnection.timestampOfLsn(tableWithSmallestLsn.getChangePosition().getCommitLsn())); + + dispatcher + .dispatchDataChangeEvent( + tableId, + new Db2ChangeRecordEmitter( + offsetContext, + operation, + data, + dataNext, + clock)); + tableWithSmallestLsn.next(); + } + }); + lastProcessedPosition = TxLogPosition.valueOf(currentMaxLsn); + // Terminate the transaction otherwise CDC could not be disabled for tables + dataConnection.rollback(); + } + catch (SQLException e) { + tablesSlot.set(processErrorFromChangeTableQuery(e, tablesSlot.get())); + } + } + } + catch (Exception e) { + errorHandler.setProducerThrowable(e); + } + } + + private void migrateTable(final Queue schemaChangeCheckpoints) + throws InterruptedException, SQLException { + final ChangeTable newTable = schemaChangeCheckpoints.poll(); + LOGGER.info("Migrating schema to {}", newTable); + dispatcher.dispatchSchemaChangeEvent(newTable.getSourceTableId(), + new Db2SchemaChangeEventEmitter(offsetContext, newTable, metadataConnection.getTableSchemaFromTable(newTable), SchemaChangeEventType.ALTER)); + } + + private ChangeTable[] processErrorFromChangeTableQuery(SQLException exception, ChangeTable[] currentChangeTables) throws Exception { + final Matcher m = MISSING_CDC_FUNCTION_CHANGES_ERROR.matcher(exception.getMessage()); + if (m.matches()) { + final String captureName = m.group(1); + LOGGER.info("Table is no longer captured with capture instance {}", captureName); + return Arrays.asList(currentChangeTables).stream() + .filter(x -> !x.getCaptureInstance().equals(captureName)) + .collect(Collectors.toList()).toArray(new ChangeTable[0]); + } + throw exception; + } + + private ChangeTable[] getCdcTablesToQuery() throws SQLException, InterruptedException { + final Set cdcEnabledTables = dataConnection.listOfChangeTables(); + + if (cdcEnabledTables.isEmpty()) { + LOGGER.warn("No table has enabled CDC or security constraints prevents getting the list of change tables"); + } + + final Map> whitelistedCdcEnabledTables = cdcEnabledTables.stream() + .filter(changeTable -> { + if (connectorConfig.getTableFilters().dataCollectionFilter().isIncluded(changeTable.getSourceTableId())) { + return true; + } + else { + LOGGER.info("CDC is enabled for table {} but the table is not whitelisted by connector", changeTable); + return false; + } + }) + .collect(Collectors.groupingBy(x -> x.getSourceTableId())); + + if (whitelistedCdcEnabledTables.isEmpty()) { + LOGGER.warn( + "No whitelisted table has enabled CDC, whitelisted table list does not contain any table with CDC enabled or no table match the white/blacklist filter(s)"); + } + + final List tables = new ArrayList<>(); + for (List captures : whitelistedCdcEnabledTables.values()) { + ChangeTable currentTable = captures.get(0); + if (captures.size() > 1) { + ChangeTable futureTable; + if (captures.get(0).getStartLsn().compareTo(captures.get(1).getStartLsn()) < 0) { + futureTable = captures.get(1); + } + else { + currentTable = captures.get(1); + futureTable = captures.get(0); + } + currentTable.setStopLsn(futureTable.getStartLsn()); + tables.add(futureTable); + LOGGER.info("Multiple capture instances present for the same table: {} and {}", currentTable, futureTable); + } + if (schema.tableFor(currentTable.getSourceTableId()) == null) { + LOGGER.info("Table {} is new to be monitored by capture instance {}", currentTable.getSourceTableId(), currentTable.getCaptureInstance()); + // We need to read the source table schema - nullability information cannot be obtained from change table + dispatcher.dispatchSchemaChangeEvent( + currentTable.getSourceTableId(), + new Db2SchemaChangeEventEmitter( + offsetContext, + currentTable, + dataConnection.getTableSchemaFromTable(currentTable), + SchemaChangeEventType.CREATE)); + } + tables.add(currentTable); + } + + return tables.toArray(new ChangeTable[tables.size()]); + } + + @Override + public void commitOffset(Map offset) { + } + + /** + * The logical representation of a position for the change in the transaction log. + * During each sourcing cycle it is necessary to query all change tables and then + * make a total order of changes across all tables.
+ * This class represents an open database cursor over the change table that is + * able to move the cursor forward and report the LSN for the change to which the cursor + * now points. + * + * @author Jiri Pechanec + * + */ + private static class ChangeTablePointer { + + private final ChangeTable changeTable; + private final ResultSet resultSet; + private boolean completed = false; + private TxLogPosition currentChangePosition; + + public ChangeTablePointer(ChangeTable changeTable, ResultSet resultSet) { + this.changeTable = changeTable; + this.resultSet = resultSet; + } + + public ChangeTable getChangeTable() { + return changeTable; + } + + public TxLogPosition getChangePosition() throws SQLException { + return currentChangePosition; + } + + public int getOperation() throws SQLException { + return resultSet.getInt(COL_OPERATION); + } + + public Object[] getData() throws SQLException { + final int dataColumnCount = resultSet.getMetaData().getColumnCount() - (COL_DATA - 1); + final Object[] data = new Object[dataColumnCount]; + for (int i = 0; i < dataColumnCount; i++) { + data[i] = resultSet.getObject(COL_DATA + i); + } + return data; + } + + public boolean next() throws SQLException { + completed = !resultSet.next(); + currentChangePosition = completed ? TxLogPosition.NULL + : TxLogPosition.valueOf(Lsn.valueOf(resultSet.getBytes(COL_COMMIT_LSN)), Lsn.valueOf(resultSet.getBytes(COL_ROW_LSN))); + if (completed) { + LOGGER.trace("Closing result set of change tables for table {}", changeTable); + resultSet.close(); + } + return !completed; + } + + public boolean isCompleted() { + return completed; + } + + public int compareTo(ChangeTablePointer o) throws SQLException { + return getChangePosition().compareTo(o.getChangePosition()); + } + + @Override + public String toString() { + return "ChangeTablePointer [changeTable=" + changeTable + ", resultSet=" + resultSet + ", completed=" + + completed + ", currentChangePosition=" + currentChangePosition + "]"; + } + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TaskContext.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TaskContext.java new file mode 100644 index 000000000..927c0825e --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TaskContext.java @@ -0,0 +1,21 @@ +/* + * 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.db2; + +import io.debezium.connector.common.CdcSourceTaskContext; + +/** + * A state (context) associated with a DB2 task + * + * @author Jiri Pechanec + * + */ +public class Db2TaskContext extends CdcSourceTaskContext { + + public Db2TaskContext(Db2ConnectorConfig config, Db2DatabaseSchema schema) { + super(config.getContextName(), config.getLogicalName(), schema::tableIds); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TopicSelector.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TopicSelector.java new file mode 100644 index 000000000..056856e18 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2TopicSelector.java @@ -0,0 +1,23 @@ +/* + * 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.db2; + +import io.debezium.relational.TableId; +import io.debezium.schema.TopicSelector; + +/** + * The topic naming strategy based on connector configuration and table name + * + * @author Jiri Pechanec + * + */ +public class Db2TopicSelector { + + public static TopicSelector defaultSelector(Db2ConnectorConfig connectorConfig) { + return TopicSelector.defaultSelector(connectorConfig, + (tableId, prefix, delimiter) -> String.join(delimiter, prefix, tableId.schema(), tableId.table())); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ValueConverters.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ValueConverters.java new file mode 100644 index 000000000..0de3be0d1 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Db2ValueConverters.java @@ -0,0 +1,85 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +import java.sql.Types; +import java.time.ZoneOffset; + +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.SchemaBuilder; + +import io.debezium.jdbc.JdbcValueConverters; +import io.debezium.jdbc.TemporalPrecisionMode; +import io.debezium.relational.Column; +import io.debezium.relational.ValueConverter; + +/** + * Conversion of DB2 specific datatypes. + * + * @author Jiri Pechanec, Peter Urbanetz + * + */ +public class Db2ValueConverters extends JdbcValueConverters { + + public Db2ValueConverters() { + } + + /** + * Create a new instance that always uses UTC for the default time zone when + * converting values without timezone information to values that require + * timezones. + *

+ * + * @param decimalMode + * how {@code DECIMAL} and {@code NUMERIC} values should be + * treated; may be null if + * {@link io.debezium.jdbc.JdbcValueConverters.DecimalMode#PRECISE} + * is to be used + * @param temporalPrecisionMode + * date/time value will be represented either as Connect datatypes or Debezium specific datatypes + */ + public Db2ValueConverters(DecimalMode decimalMode, TemporalPrecisionMode temporalPrecisionMode) { + super(decimalMode, temporalPrecisionMode, ZoneOffset.UTC, null, null); + } + + @Override + public SchemaBuilder schemaBuilder(Column column) { + switch (column.jdbcType()) { + // Numeric integers + case Types.TINYINT: + // values are an 8-bit unsigned integer value between 0 and 255, we thus need to store it in short int + return SchemaBuilder.int16(); + default: + return super.schemaBuilder(column); + } + } + + @Override + public ValueConverter converter(Column column, Field fieldDefn) { + switch (column.jdbcType()) { + // Numeric integers + case Types.TINYINT: + // values are an 8-bit unsigned integer value between 0 and 255, we thus need to store it in short int + return (data) -> convertSmallInt(column, fieldDefn, data); + default: + return super.converter(column, fieldDefn); + } + } + + /** + * Time precision in DB2 is defined in scale, the default one is 7 + */ + @Override + protected int getTimePrecision(Column column) { + return column.scale().get(); + } + + protected Object convertTimestampWithZone(Column column, Field fieldDefn, Object data) { + // dummy return + return super.convertTimestampWithZone(column, fieldDefn, data); + } + +} \ No newline at end of file diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Lsn.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Lsn.java new file mode 100644 index 000000000..b950a0c70 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Lsn.java @@ -0,0 +1,181 @@ +/* + * 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.db2; + +import java.math.BigInteger; +import java.util.Arrays; + +import io.debezium.util.Strings; + +/** + * A logical representation of DB2 LSN (log sequence number) position. When LSN is not available + * it is replaced with {@link Lsn.NULL} constant. + * + * @author Jiri Pechanec, Luis Garcés-Erice + * + */ +public class Lsn implements Comparable, Nullable { + private static final String NULL_STRING = "NULL"; + + public static final Lsn NULL = new Lsn(null); + + private final byte[] binary; + private int[] unsignedBinary; + + private String string; + + private Lsn(byte[] binary) { + this.binary = binary; + } + + /** + * @return binary representation of the stored LSN + */ + public byte[] getBinary() { + return binary; + } + + /** + * @return true if this is a real LSN or false it it is {@code NULL} + */ + @Override + public boolean isAvailable() { + return binary != null; + } + + private int[] getUnsignedBinary() { + if (unsignedBinary != null || binary == null) { + return unsignedBinary; + } + + unsignedBinary = new int[binary.length]; + for (int i = 0; i < binary.length; i++) { + unsignedBinary[i] = Byte.toUnsignedInt(binary[i]); + } + return unsignedBinary; + } + + /** + * @return textual representation of the stored LSN + */ + public String toString() { + if (string != null) { + return string; + } + final StringBuilder sb = new StringBuilder(); + if (binary == null) { + return NULL_STRING; + } + final int[] unsigned = getUnsignedBinary(); + for (int i = 0; i < unsigned.length; i++) { + final String byteStr = Integer.toHexString(unsigned[i]); + if (byteStr.length() == 1) { + sb.append('0'); + } + sb.append(byteStr); + if (i == 3 || i == 7) { + sb.append(':'); + } + } + string = sb.toString(); + return string; + } + + /** + * @param lsnString - textual representation of Lsn + * @return LSN converted from its textual representation + */ + public static Lsn valueOf(String lsnString) { + return (lsnString == null || NULL_STRING.equals(lsnString)) ? NULL : new Lsn(Strings.hexStringToByteArray(lsnString.replace(":", ""))); + } + + /** + * @param lsnBinary - binary representation of Lsn + * @return LSN converted from its binary representation + */ + public static Lsn valueOf(byte[] lsnBinary) { + return (lsnBinary == null) ? NULL : new Lsn(lsnBinary); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + Arrays.hashCode(binary); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + Lsn other = (Lsn) obj; + if (!Arrays.equals(binary, other.binary)) { + return false; + } + return true; + } + + /** + * Enables ordering of LSNs. The {@code NULL} LSN is always the smallest one. + */ + @Override + public int compareTo(Lsn o) { + if (this == o) { + return 0; + } + if (!this.isAvailable()) { + if (!o.isAvailable()) { + return 0; + } + return -1; + } + if (!o.isAvailable()) { + return 1; + } + final int[] thisU = getUnsignedBinary(); + final int[] thatU = o.getUnsignedBinary(); + for (int i = 0; i < thisU.length; i++) { + final int diff = thisU[i] - thatU[i]; + if (diff != 0) { + return diff; + } + } + return 0; + } + + /** + * Verifies whether the LSN falls into a LSN interval + * + * @param from start of the interval (included) + * @param to end of the interval (excluded) + * + * @return true if the LSN falls into the interval + */ + public boolean isBetween(Lsn from, Lsn to) { + return this.compareTo(from) >= 0 && this.compareTo(to) < 0; + } + + /** + * Return the next LSN in sequence + */ + public Lsn increment() { + final BigInteger bi = new BigInteger(this.toString().replace(":", ""), 16).add(BigInteger.ONE); + final byte[] biByteArray = bi.toByteArray(); + final byte[] lsnByteArray = new byte[16]; + for (int i = 0; i < biByteArray.length; i++) { + lsnByteArray[i + 16 - biByteArray.length] = biByteArray[i]; + } + return Lsn.valueOf(lsnByteArray); + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Module.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Module.java new file mode 100644 index 000000000..b6bd4e8d2 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Module.java @@ -0,0 +1,38 @@ +/* + * 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.db2; + +import java.util.Properties; + +import io.debezium.util.IoUtil; + +/** + * Information about this module. + * + * @author Peter Urbanetz + */ +public final class Module { + + private static final Properties INFO = IoUtil.loadProperties(Module.class, "io/debezium/connector/db2/build.version"); + + public static String version() { + return INFO.getProperty("version"); + } + + /** + * @return symbolic name of the connector plugin + */ + public static String name() { + return "db2"; + } + + /** + * @return context name used in log MDC and JMX metrics + */ + public static String contextName() { + return "DB2_Server"; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Nullable.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Nullable.java new file mode 100644 index 000000000..84407edcb --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/Nullable.java @@ -0,0 +1,20 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +/** + * + * @author Jiri Pechanec + * + */ +public interface Nullable { + + /** + * @return true if this object has real value, false if it is NULL object + */ + boolean isAvailable(); + +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/SourceInfo.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/SourceInfo.java new file mode 100644 index 000000000..1896befed --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/SourceInfo.java @@ -0,0 +1,99 @@ +/* + * 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.db2; + +import java.time.Instant; + +import io.debezium.annotation.NotThreadSafe; +import io.debezium.connector.common.BaseSourceInfo; +import io.debezium.relational.TableId; + +/** + * Coordinates from the database log to establish the relation between the change streamed and the source log position. + * Maps to {@code source} field in {@code Envelope}. + * + * @author Jiri Pechanec + * + */ +@NotThreadSafe +public class SourceInfo extends BaseSourceInfo { + + public static final String CHANGE_LSN_KEY = "change_lsn"; + public static final String COMMIT_LSN_KEY = "commit_lsn"; + + private Lsn changeLsn; + private Lsn commitLsn; + private Instant sourceTime; + private TableId tableId; + + private String databaseName; + + protected SourceInfo(Db2ConnectorConfig connectorConfig) { + super(connectorConfig); + this.databaseName = connectorConfig.getDatabaseName(); + } + + /** + * @param lsn - LSN of the change in the database log + */ + public void setChangeLsn(Lsn lsn) { + changeLsn = lsn; + } + + public Lsn getChangeLsn() { + return changeLsn; + } + + public Lsn getCommitLsn() { + return commitLsn; + } + + /** + * @param commitLsn - LSN of the {@code COMMIT} of the transaction whose part the change is + */ + public void setCommitLsn(Lsn commitLsn) { + this.commitLsn = commitLsn; + } + + /** + * @param instant a time at which the transaction commit was executed + */ + public void setSourceTime(Instant instant) { + sourceTime = instant; + } + + public TableId getTableId() { + return tableId; + } + + /** + * @param tableId - source table of the event + */ + public void setTableId(TableId tableId) { + this.tableId = tableId; + } + + @Override + public String toString() { + return "SourceInfo [" + + "serverName=" + serverName() + + ", changeLsn=" + changeLsn + + ", commitLsn=" + commitLsn + + ", snapshot=" + snapshotRecord + + ", sourceTime=" + sourceTime + + "]"; + } + + @Override + protected Instant timestamp() { + return sourceTime; + } + + @Override + protected String database() { + return databaseName; + } +} diff --git a/debezium-connector-db2/src/main/java/io/debezium/connector/db2/TxLogPosition.java b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/TxLogPosition.java new file mode 100644 index 000000000..cdef59b13 --- /dev/null +++ b/debezium-connector-db2/src/main/java/io/debezium/connector/db2/TxLogPosition.java @@ -0,0 +1,102 @@ +/* + * 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.db2; + +/** + * Defines a position of change in the transaction log. The position is defined as a combination of commit LSN + * and sequence number of the change in the given transaction. + * The sequence number is monotonically increasing in transaction but it is not guaranteed across multiple + * transactions so the combination is necessary to get total order. + * + * @author Jiri Pechanec + * + */ +public class TxLogPosition implements Nullable, Comparable { + + public static final TxLogPosition NULL = new TxLogPosition(null, null); + private final Lsn commitLsn; + private final Lsn inTxLsn; + + private TxLogPosition(Lsn commitLsn, Lsn inTxLsn) { + this.commitLsn = commitLsn; + this.inTxLsn = inTxLsn; + } + + public Lsn getCommitLsn() { + return commitLsn; + } + + public Lsn getInTxLsn() { + return inTxLsn; + } + + @Override + public String toString() { + return this == NULL ? "NULL" : commitLsn + "(" + inTxLsn + ")"; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((commitLsn == null) ? 0 : commitLsn.hashCode()); + result = prime * result + ((inTxLsn == null) ? 0 : inTxLsn.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + TxLogPosition other = (TxLogPosition) obj; + if (commitLsn == null) { + if (other.commitLsn != null) { + return false; + } + } + else if (!commitLsn.equals(other.commitLsn)) { + return false; + } + if (inTxLsn == null) { + if (other.inTxLsn != null) { + return false; + } + } + else if (!inTxLsn.equals(other.inTxLsn)) { + return false; + } + return true; + } + + @Override + public int compareTo(TxLogPosition o) { + final int comparison = commitLsn.compareTo(o.getCommitLsn()); + return comparison == 0 ? inTxLsn.compareTo(o.inTxLsn) : comparison; + } + + public static TxLogPosition valueOf(Lsn commitLsn, Lsn inTxLsn) { + return commitLsn == null && inTxLsn == null ? NULL + : new TxLogPosition( + commitLsn == null ? Lsn.NULL : commitLsn, + inTxLsn == null ? Lsn.NULL : inTxLsn); + } + + public static TxLogPosition valueOf(Lsn commitLsn) { + return valueOf(commitLsn, Lsn.NULL); + } + + @Override + public boolean isAvailable() { + return inTxLsn != null && commitLsn != null; + } +} diff --git a/debezium-connector-db2/src/main/resources/io/debezium/connector/db2/build.version b/debezium-connector-db2/src/main/resources/io/debezium/connector/db2/build.version new file mode 100644 index 000000000..e5683df88 --- /dev/null +++ b/debezium-connector-db2/src/main/resources/io/debezium/connector/db2/build.version @@ -0,0 +1 @@ +version=${project.version} \ No newline at end of file diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/Dockerfile b/debezium-connector-db2/src/test/docker/db2-cdc-docker/Dockerfile new file mode 100644 index 000000000..7c068652a --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/Dockerfile @@ -0,0 +1,22 @@ +FROM ibmcom/db2 + +MAINTAINER Peter Urbanetz + + +RUN mkdir -p /asncdctools/src + +ADD asncdc_UDF.sql /asncdctools/src +ADD asncdcaddremove.sql /asncdctools/src +ADD asncdctables.sql /asncdctools/src +ADD dbsetup.sh /asncdctools/src +ADD asncdc.c /asncdctools/src + + +RUN chmod -R 777 /asncdctools + +RUN mkdir /var/custom +RUN chmod -R 777 /var/custom + +ADD cdcsetup.sh /var/custom + +RUN chmod 777 /var/custom/cdcsetup.sh \ No newline at end of file diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc.c b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc.c new file mode 100644 index 000000000..be1fc483f --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc.c @@ -0,0 +1,162 @@ +#include +#include +#include +#include +#include +#include + +void SQL_API_FN asncdcservice( + SQLUDF_VARCHAR *asnCommand, /* input */ + SQLUDF_VARCHAR *asnService, + SQLUDF_CLOB *fileData, /* output */ + /* null indicators */ + SQLUDF_NULLIND *asnCommand_ind, /* input */ + SQLUDF_NULLIND *asnService_ind, + SQLUDF_NULLIND *fileData_ind, + SQLUDF_TRAIL_ARGS, + struct sqludf_dbinfo *dbinfo) +{ + + int fd; + char tmpFileName[] = "/tmp/fileXXXXXX"; + fd = mkstemp(tmpFileName); + + int strcheck = 0; + char cmdstring[256]; + + + char* szDb2path = getenv("HOME"); + + + + char str[20]; + int len = 0; + char c; + char *buffer = NULL; + FILE *pidfile; + + char dbname[129]; + memset(dbname, '\0', 129); + strncpy(dbname, (char *)(dbinfo->dbname), dbinfo->dbnamelen); + dbname[dbinfo->dbnamelen] = '\0'; + + int pid; + if (strcmp(asnService, "asncdc") == 0) + { + strcheck = sprintf(cmdstring, "pgrep -fx \"%s/sqllib/bin/asncap capture_schema=%s capture_server=%s\" > %s", szDb2path, asnService, dbname, tmpFileName); + int callcheck; + callcheck = system(cmdstring); + pidfile = fopen(tmpFileName, "r"); + while ((c = fgetc(pidfile)) != EOF) + { + if (c == '\n') + { + break; + } + len++; + } + buffer = (char *)malloc(sizeof(char) * len); + fseek(pidfile, 0, SEEK_SET); + fread(buffer, sizeof(char), len, pidfile); + fclose(pidfile); + pidfile = fopen(tmpFileName, "w"); + if (strcmp(asnCommand, "start") == 0) + { + if (len == 0) // is not running + { + strcheck = sprintf(cmdstring, "%s/sqllib/bin/asncap capture_schema=%s capture_server=%s &", szDb2path, asnService, dbname); + fprintf(pidfile, "start --> %s \n", cmdstring); + callcheck = system(cmdstring); + } + else + { + fprintf(pidfile, "asncap is already running"); + } + } + if ((strcmp(asnCommand, "prune") == 0) || + (strcmp(asnCommand, "reinit") == 0) || + (strcmp(asnCommand, "suspend") == 0) || + (strcmp(asnCommand, "resume") == 0) || + (strcmp(asnCommand, "status") == 0) || + (strcmp(asnCommand, "stop") == 0)) + { + if (len > 0) + { + //buffer[len] = '\0'; + //strcheck = sprintf(cmdstring, "/bin/kill -SIGINT %s ", buffer); + //fprintf(pidfile, "stop --> %s", cmdstring); + //callcheck = system(cmdstring); + strcheck = sprintf(cmdstring, "%s/sqllib/bin/asnccmd capture_schema=%s capture_server=%s %s >> %s", szDb2path, asnService, dbname, asnCommand, tmpFileName); + //fprintf(pidfile, "%s --> %s \n", cmdstring, asnCommand); + callcheck = system(cmdstring); + } + else + { + fprintf(pidfile, "asncap is not running"); + } + } + + fclose(pidfile); + } + /* system(cmdstring); */ + + int rc = 0; + long fileSize = 0; + size_t readCnt = 0; + FILE *f = NULL; + + f = fopen(tmpFileName, "r"); + if (!f) + { + strcpy(SQLUDF_MSGTX, "Could not open file "); + strncat(SQLUDF_MSGTX, tmpFileName, + SQLUDF_MSGTEXT_LEN - strlen(SQLUDF_MSGTX) - 1); + strncpy(SQLUDF_STATE, "38100", SQLUDF_SQLSTATE_LEN); + return; + } + + rc = fseek(f, 0, SEEK_END); + if (rc) + { + sprintf(SQLUDF_MSGTX, "fseek() failed with rc = %d", rc); + strncpy(SQLUDF_STATE, "38101", SQLUDF_SQLSTATE_LEN); + return; + } + + /* verify the file size */ + fileSize = ftell(f); + if (fileSize > fileData->length) + { + strcpy(SQLUDF_MSGTX, "File too large"); + strncpy(SQLUDF_STATE, "38102", SQLUDF_SQLSTATE_LEN); + return; + } + + /* go to the beginning and read the entire file */ + rc = fseek(f, 0, 0); + if (rc) + { + sprintf(SQLUDF_MSGTX, "fseek() failed with rc = %d", rc); + strncpy(SQLUDF_STATE, "38103", SQLUDF_SQLSTATE_LEN); + return; + } + + readCnt = fread(fileData->data, 1, fileSize, f); + if (readCnt != fileSize) + { + /* raise a warning that something weird is going on */ + sprintf(SQLUDF_MSGTX, "Could not read entire file " + "(%d vs %d)", + readCnt, fileSize); + strncpy(SQLUDF_STATE, "01H10", SQLUDF_SQLSTATE_LEN); + *fileData_ind = -1; + } + else + { + fileData->length = readCnt; + *fileData_ind = 0; + } + // remove temorary file + rc = remove(tmpFileName); + //fclose(pFile); +} diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc_UDF.sql b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc_UDF.sql new file mode 100644 index 000000000..fc1da4f9a --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdc_UDF.sql @@ -0,0 +1,17 @@ +DROP SPECIFIC FUNCTION ASNCDC.asncdcservice; + +CREATE FUNCTION ASNCDC.ASNCDCSERVICES(command VARCHAR(6), service VARCHAR(8)) + RETURNS CLOB(100K) + SPECIFIC asncdcservice + EXTERNAL NAME 'asncdc!asncdcservice' + LANGUAGE C + PARAMETER STYLE SQL + DBINFO + DETERMINISTIC + NOT FENCED + RETURNS NULL ON NULL INPUT + NO SQL + NO EXTERNAL ACTION + NO SCRATCHPAD + ALLOW PARALLEL + NO FINAL CALL; \ No newline at end of file diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdcaddremove.sql b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdcaddremove.sql new file mode 100644 index 000000000..73d0aa61a --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdcaddremove.sql @@ -0,0 +1,194 @@ + +-- +-- Define ASNCDC.REMOVETABLE() and ASNCDC.ADDTABLE() +-- ASNCDC.ADDTABLE() puts a table in CDC mode, making the ASNCapture server collect changes for the table +-- ASNCDC.REMOVETABLE() makes the ASNCapture server stop collecting changes for that table +-- + +--#SET TERMINATOR @ +CREATE OR REPLACE PROCEDURE ASNCDC.REMOVETABLE( +in tableschema VARCHAR(128), +in tablename VARCHAR(128) +) +LANGUAGE SQL +P1: +BEGIN + +DECLARE stmtSQL VARCHAR(2048); + +DECLARE SQLCODE INT; +DECLARE SQLSTATE CHAR(5); +DECLARE RC_SQLCODE INT DEFAULT 0; +DECLARE RC_SQLSTATE CHAR(5) DEFAULT '00000'; + +DECLARE CONTINUE HANDLER FOR SQLEXCEPTION, SQLWARNING, NOT FOUND VALUES (SQLCODE, SQLSTATE) INTO RC_SQLCODE, RC_SQLSTATE; + +-- delete ASN.IBMSNAP_PRUNCTL entries / source +SET stmtSQL = 'DELETE FROM ASNCDC.IBMSNAP_PRUNCNTL WHERE SOURCE_OWNER=''' || tableschema || ''' AND SOURCE_TABLE=''' || tablename || ''''; + EXECUTE IMMEDIATE stmtSQL; + +-- delete ASN.IBMSNAP_Register entries / source +SET stmtSQL = 'DELETE FROM ASNCDC.IBMSNAP_REGISTER WHERE SOURCE_OWNER=''' || tableschema || ''' AND SOURCE_TABLE=''' || tablename || ''''; + EXECUTE IMMEDIATE stmtSQL; + +-- drop CD Table / source +SET stmtSQL = 'DROP TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"'; + EXECUTE IMMEDIATE stmtSQL; + +-- delete ASN.IBMSNAP_SUBS_COLS entries /target +SET stmtSQL = 'DELETE FROM ASNCDC.IBMSNAP_SUBS_COLS WHERE TARGET_OWNER=''' || tableschema || ''' AND TARGET_TABLE=''' || tablename || ''''; + EXECUTE IMMEDIATE stmtSQL; + +-- delete ASN.IBMSNAP_SUSBS_MEMBER entries /target +SET stmtSQL = 'DELETE FROM ASNCDC.IBMSNAP_SUBS_MEMBR WHERE TARGET_OWNER=''' || tableschema || ''' AND TARGET_TABLE=''' || tablename || ''''; + EXECUTE IMMEDIATE stmtSQL; + +-- delete ASN.IBMQREP_COLVERSION +SET stmtSQL = 'DELETE FROM ASNCDC.IBMQREP_COLVERSION col WHERE EXISTS (SELECT * FROM ASNCDC.IBMQREP_TABVERSION tab WHERE SOURCE_OWNER=''' || tableschema || ''' AND SOURCE_NAME=''' || tablename || '''AND col.TABLEID1 = tab.TABLEID1 AND col.TABLEID2 = tab.TABLEID2'; + EXECUTE IMMEDIATE stmtSQL; + +-- delete ASN.IBMQREP_TABVERSION +SET stmtSQL = 'DELETE FROM ASNCDC.IBMQREP_TABVERSION WHERE SOURCE_OWNER=''' || tableschema || ''' AND SOURCE_NAME=''' || tablename || ''''; + EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'ALTER TABLE "' || tableschema || '"."' || tablename || '" DATA CAPTURE NONE'; +EXECUTE IMMEDIATE stmtSQL; + +END P1@ +--#SET TERMINATOR ; + +--#SET TERMINATOR @ +CREATE OR REPLACE PROCEDURE ASNCDC.ADDTABLE( +in tableschema VARCHAR(128), +in tablename VARCHAR(128) +) +LANGUAGE SQL +P1: +BEGIN + +DECLARE SQLSTATE CHAR(5); + +DECLARE stmtSQL VARCHAR(2048); + +SET stmtSQL = 'ALTER TABLE "' || tableschema || '"."' || tablename || '" DATA CAPTURE CHANGES'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'CREATE TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' AS ( SELECT ' || + ' CAST('''' AS VARCHAR ( 16 ) FOR BIT DATA) AS IBMSNAP_COMMITSEQ, ' || + ' CAST('''' AS VARCHAR ( 16 ) FOR BIT DATA) AS IBMSNAP_INTENTSEQ, ' || + ' CAST ('''' AS CHAR(1)) ' || + ' AS IBMSNAP_OPERATION, t.* FROM "' || tableschema || '"."' || tablename || '" as t ) WITH NO DATA ORGANIZE BY ROW '; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'ALTER TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' ALTER COLUMN IBMSNAP_COMMITSEQ SET NOT NULL'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'ALTER TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' ALTER COLUMN IBMSNAP_INTENTSEQ SET NOT NULL'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'ALTER TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' ALTER COLUMN IBMSNAP_OPERATION SET NOT NULL'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'CREATE UNIQUE INDEX ASNCDC."IXCDC_' || + tableschema || '_' || tablename || '"' || + ' ON ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' ( IBMSNAP_COMMITSEQ ASC, IBMSNAP_INTENTSEQ ASC ) PCTFREE 0 MINPCTUSED 0'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'ALTER TABLE ASNCDC."CDC_' || + tableschema || '_' || tablename || '"' || + ' VOLATILE CARDINALITY'; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'INSERT INTO ASNCDC.IBMSNAP_REGISTER (SOURCE_OWNER, SOURCE_TABLE, ' || + 'SOURCE_VIEW_QUAL, GLOBAL_RECORD, SOURCE_STRUCTURE, SOURCE_CONDENSED, ' || + 'SOURCE_COMPLETE, CD_OWNER, CD_TABLE, PHYS_CHANGE_OWNER, ' || + 'PHYS_CHANGE_TABLE, CD_OLD_SYNCHPOINT, CD_NEW_SYNCHPOINT, ' || + 'DISABLE_REFRESH, CCD_OWNER, CCD_TABLE, CCD_OLD_SYNCHPOINT, ' || + 'SYNCHPOINT, SYNCHTIME, CCD_CONDENSED, CCD_COMPLETE, ARCH_LEVEL, ' || + 'DESCRIPTION, BEFORE_IMG_PREFIX, CONFLICT_LEVEL, ' || + 'CHG_UPD_TO_DEL_INS, CHGONLY, RECAPTURE, OPTION_FLAGS, ' || + 'STOP_ON_ERROR, STATE, STATE_INFO ) VALUES( ' || + '''' || tableschema || ''', ' || + '''' || tablename || ''', ' || + '0, ' || + '''N'', ' || + '1, ' || + '''Y'', ' || + '''Y'', ' || + '''ASNCDC'', ' || + '''CDC_' || tableschema || '_' || tablename || ''', ' || + '''ASNCDC'', ' || + '''CDC_' || tableschema || '_' || tablename || ''', ' || + 'null, ' || + 'null, ' || + '0, ' || + 'null, ' || + 'null, ' || + 'null, ' || + 'null, ' || + 'null, ' || + 'null, ' || + 'null, ' || + '''0801'', ' || + 'null, ' || + 'null, ' || + '''0'', ' || + '''Y'', ' || + '''N'', ' || + '''Y'', ' || + '''NNNN'', ' || + '''Y'', ' || + '''A'',' || + 'null ) '; +EXECUTE IMMEDIATE stmtSQL; + +SET stmtSQL = 'INSERT INTO ASNCDC.IBMSNAP_PRUNCNTL ( ' || + 'TARGET_SERVER, ' || + 'TARGET_OWNER, ' || + 'TARGET_TABLE, ' || + 'SYNCHTIME, ' || + 'SYNCHPOINT, ' || + 'SOURCE_OWNER, ' || + 'SOURCE_TABLE, ' || + 'SOURCE_VIEW_QUAL, ' || + 'APPLY_QUAL, ' || + 'SET_NAME, ' || + 'CNTL_SERVER , ' || + 'TARGET_STRUCTURE , ' || + 'CNTL_ALIAS , ' || + 'PHYS_CHANGE_OWNER , ' || + 'PHYS_CHANGE_TABLE , ' || + 'MAP_ID ' || + ') VALUES ( ' || + '''KAFKA'', ' || + '''' || tableschema || ''', ' || + '''' || tablename || ''', ' || + 'NULL, ' || + 'NULL, ' || + '''' || tableschema || ''', ' || + '''' || tablename || ''', ' || + '0, ' || + '''KAFKAQUAL'', ' || + '''SET001'', ' || + ' (Select CURRENT_SERVER from sysibm.sysdummy1 ), ' || + '8, ' || + ' (Select CURRENT_SERVER from sysibm.sysdummy1 ), ' || + '''ASNCDC'', ' || + '''"CDC_' || tableschema || '_' || tablename || '"'', ' || + ' ( SELECT CASE WHEN max(CAST(MAP_ID AS INT)) IS NULL THEN CAST(1 AS VARCHAR(10)) ELSE CAST(CAST(max(MAP_ID) AS INT) + 1 AS VARCHAR(10)) END AS MYINT from ASNCDC.IBMSNAP_PRUNCNTL ) ' || + ' )'; +EXECUTE IMMEDIATE stmtSQL; + +END P1@ +--#SET TERMINATOR ; diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdctables.sql b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdctables.sql new file mode 100644 index 000000000..e36b42a82 --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/asncdctables.sql @@ -0,0 +1,544 @@ +-- 1021 db2 LEVEL Version 10.2.0 --> 11.5.0 1150 + +CREATE TABLE ASNCDC.IBMQREP_COLVERSION( +LSN VARCHAR( 16) FOR BIT DATA NOT NULL, +TABLEID1 SMALLINT NOT NULL, +TABLEID2 SMALLINT NOT NULL, +POSITION SMALLINT NOT NULL, +NAME VARCHAR(128) NOT NULL, +TYPE SMALLINT NOT NULL, +LENGTH INTEGER NOT NULL, +NULLS CHAR( 1) NOT NULL, +DEFAULT VARCHAR(1536), +CODEPAGE INTEGER, +SCALE INTEGER, +VERSION_TIME TIMESTAMP NOT NULL WITH DEFAULT ) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMQREP_COLVERSIOX +ON ASNCDC.IBMQREP_COLVERSION( +LSN ASC, +TABLEID1 ASC, +TABLEID2 ASC, +POSITION ASC); + + +CREATE INDEX ASNCDC.IX2COLVERSION +ON ASNCDC.IBMQREP_COLVERSION( +TABLEID1 ASC, +TABLEID2 ASC); + + + + +CREATE TABLE ASNCDC.IBMQREP_TABVERSION( +LSN VARCHAR( 16) FOR BIT DATA NOT NULL, +TABLEID1 SMALLINT NOT NULL, +TABLEID2 SMALLINT NOT NULL, +VERSION INTEGER NOT NULL, +SOURCE_OWNER VARCHAR(128) NOT NULL, +SOURCE_NAME VARCHAR(128) NOT NULL, +VERSION_TIME TIMESTAMP NOT NULL WITH DEFAULT ) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMQREP_TABVERSIOX +ON ASNCDC.IBMQREP_TABVERSION( +LSN ASC, +TABLEID1 ASC, +TABLEID2 ASC, +VERSION ASC); + + +CREATE INDEX ASNCDC.IX2TABVERSION +ON ASNCDC.IBMQREP_TABVERSION( +TABLEID1 ASC, +TABLEID2 ASC); + + +CREATE INDEX ASNCDC.IX3TABVERSION +ON ASNCDC.IBMQREP_TABVERSION( +SOURCE_OWNER ASC, +SOURCE_NAME ASC); + + +CREATE TABLE ASNCDC.IBMSNAP_APPLEVEL( +ARCH_LEVEL CHAR( 4) NOT NULL WITH DEFAULT '1021') +ORGANIZE BY ROW; + + +INSERT INTO ASNCDC.IBMSNAP_APPLEVEL(ARCH_LEVEL) VALUES ( +'1021'); + + +CREATE TABLE ASNCDC.IBMSNAP_CAPMON( +MONITOR_TIME TIMESTAMP NOT NULL, +RESTART_TIME TIMESTAMP NOT NULL, +CURRENT_MEMORY INT NOT NULL, +CD_ROWS_INSERTED INT NOT NULL, +RECAP_ROWS_SKIPPED INT NOT NULL, +TRIGR_ROWS_SKIPPED INT NOT NULL, +CHG_ROWS_SKIPPED INT NOT NULL, +TRANS_PROCESSED INT NOT NULL, +TRANS_SPILLED INT NOT NULL, +MAX_TRANS_SIZE INT NOT NULL, +LOCKING_RETRIES INT NOT NULL, +JRN_LIB CHAR( 10), +JRN_NAME CHAR( 10), +LOGREADLIMIT INT NOT NULL, +CAPTURE_IDLE INT NOT NULL, +SYNCHTIME TIMESTAMP NOT NULL, +CURRENT_LOG_TIME TIMESTAMP NOT NULL WITH DEFAULT , +LAST_EOL_TIME TIMESTAMP, +RESTART_SEQ VARCHAR( 16) FOR BIT DATA NOT NULL WITH DEFAULT , +CURRENT_SEQ VARCHAR( 16) FOR BIT DATA NOT NULL WITH DEFAULT , +RESTART_MAXCMTSEQ VARCHAR( 16) FOR BIT DATA NOT NULL WITH DEFAULT , +LOGREAD_API_TIME INT, +NUM_LOGREAD_CALLS INT, +NUM_END_OF_LOGS INT, +LOGRDR_SLEEPTIME INT, +NUM_LOGREAD_F_CALLS INT, +TRANS_QUEUED INT, +NUM_WARNTXS INT, +NUM_WARNLOGAPI INT) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_CAPMONX +ON ASNCDC.IBMSNAP_CAPMON( +MONITOR_TIME ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_CAPMON VOLATILE CARDINALITY; + + + +CREATE TABLE ASNCDC.IBMSNAP_CAPPARMS( +RETENTION_LIMIT INT, +LAG_LIMIT INT, +COMMIT_INTERVAL INT, +PRUNE_INTERVAL INT, +TRACE_LIMIT INT, +MONITOR_LIMIT INT, +MONITOR_INTERVAL INT, +MEMORY_LIMIT SMALLINT, +REMOTE_SRC_SERVER CHAR( 18), +AUTOPRUNE CHAR( 1), +TERM CHAR( 1), +AUTOSTOP CHAR( 1), +LOGREUSE CHAR( 1), +LOGSTDOUT CHAR( 1), +SLEEP_INTERVAL SMALLINT, +CAPTURE_PATH VARCHAR(1040), +STARTMODE VARCHAR( 10), +LOGRDBUFSZ INT NOT NULL WITH DEFAULT 256, +ARCH_LEVEL CHAR( 4) NOT NULL WITH DEFAULT '1021', +COMPATIBILITY CHAR( 4) NOT NULL WITH DEFAULT '1021') + ORGANIZE BY ROW; + + +INSERT INTO ASNCDC.IBMSNAP_CAPPARMS( +RETENTION_LIMIT, +LAG_LIMIT, +COMMIT_INTERVAL, +PRUNE_INTERVAL, +TRACE_LIMIT, +MONITOR_LIMIT, +MONITOR_INTERVAL, +MEMORY_LIMIT, +SLEEP_INTERVAL, +AUTOPRUNE, +TERM, +AUTOSTOP, +LOGREUSE, +LOGSTDOUT, +CAPTURE_PATH, +STARTMODE, +COMPATIBILITY) +VALUES ( +10080, +10080, +30, +300, +10080, +10080, +300, +32, +5, +'Y', +'Y', +'N', +'N', +'N', +NULL, +'WARMSI', +'1021' +); + + + +CREATE TABLE ASNCDC.IBMSNAP_CAPSCHEMAS ( + CAP_SCHEMA_NAME VARCHAR(128 OCTETS) NOT NULL + ) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_CAPSCHEMASX + ON ASNCDC.IBMSNAP_CAPSCHEMAS + (CAP_SCHEMA_NAME ASC); + + +INSERT INTO ASNCDC.IBMSNAP_CAPSCHEMAS(CAP_SCHEMA_NAME) VALUES ( +'ASNCDC'); + + + +CREATE TABLE ASNCDC.IBMSNAP_CAPTRACE( +OPERATION CHAR( 8) NOT NULL, +TRACE_TIME TIMESTAMP NOT NULL, +DESCRIPTION VARCHAR(1024) NOT NULL) + ORGANIZE BY ROW; + + +CREATE INDEX ASNCDC.IBMSNAP_CAPTRACEX +ON ASNCDC.IBMSNAP_CAPTRACE( +TRACE_TIME ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_CAPTRACE VOLATILE CARDINALITY; + + + +CREATE TABLE ASNCDC.IBMSNAP_PRUNCNTL( +TARGET_SERVER CHAR(18) NOT NULL, +TARGET_OWNER VARCHAR(128) NOT NULL, +TARGET_TABLE VARCHAR(128) NOT NULL, +SYNCHTIME TIMESTAMP, +SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +SOURCE_OWNER VARCHAR(128) NOT NULL, +SOURCE_TABLE VARCHAR(128) NOT NULL, +SOURCE_VIEW_QUAL SMALLINT NOT NULL, +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +CNTL_SERVER CHAR( 18) NOT NULL, +TARGET_STRUCTURE SMALLINT NOT NULL, +CNTL_ALIAS CHAR( 8), +PHYS_CHANGE_OWNER VARCHAR(128), +PHYS_CHANGE_TABLE VARCHAR(128), +MAP_ID VARCHAR(10) NOT NULL) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_PRUNCNTLX +ON ASNCDC.IBMSNAP_PRUNCNTL( +SOURCE_OWNER ASC, +SOURCE_TABLE ASC, +SOURCE_VIEW_QUAL ASC, +APPLY_QUAL ASC, +SET_NAME ASC, +TARGET_SERVER ASC, +TARGET_TABLE ASC, +TARGET_OWNER ASC); + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_PRUNCNTLX1 +ON ASNCDC.IBMSNAP_PRUNCNTL( +MAP_ID ASC); + + +CREATE INDEX ASNCDC.IBMSNAP_PRUNCNTLX2 +ON ASNCDC.IBMSNAP_PRUNCNTL( +PHYS_CHANGE_OWNER ASC, +PHYS_CHANGE_TABLE ASC); + + +CREATE INDEX ASNCDC.IBMSNAP_PRUNCNTLX3 +ON ASNCDC.IBMSNAP_PRUNCNTL( +APPLY_QUAL ASC, +SET_NAME ASC, +TARGET_SERVER ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_PRUNCNTL VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_PRUNE_LOCK( +DUMMY CHAR( 1)) + ORGANIZE BY ROW; + + + +CREATE TABLE ASNCDC.IBMSNAP_PRUNE_SET( +TARGET_SERVER CHAR( 18) NOT NULL, +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +SYNCHTIME TIMESTAMP, +SYNCHPOINT VARCHAR( 16) FOR BIT DATA NOT NULL) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_PRUNE_SETX +ON ASNCDC.IBMSNAP_PRUNE_SET( +TARGET_SERVER ASC, +APPLY_QUAL ASC, +SET_NAME ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_PRUNE_SET VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_REGISTER( +SOURCE_OWNER VARCHAR(128) NOT NULL, +SOURCE_TABLE VARCHAR(128) NOT NULL, +SOURCE_VIEW_QUAL SMALLINT NOT NULL, +GLOBAL_RECORD CHAR( 1) NOT NULL, +SOURCE_STRUCTURE SMALLINT NOT NULL, +SOURCE_CONDENSED CHAR( 1) NOT NULL, +SOURCE_COMPLETE CHAR( 1) NOT NULL, +CD_OWNER VARCHAR(128), +CD_TABLE VARCHAR(128), +PHYS_CHANGE_OWNER VARCHAR(128), +PHYS_CHANGE_TABLE VARCHAR(128), +CD_OLD_SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +CD_NEW_SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +DISABLE_REFRESH SMALLINT NOT NULL, +CCD_OWNER VARCHAR(128), +CCD_TABLE VARCHAR(128), +CCD_OLD_SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +SYNCHTIME TIMESTAMP, +CCD_CONDENSED CHAR( 1), +CCD_COMPLETE CHAR( 1), +ARCH_LEVEL CHAR( 4) NOT NULL, +DESCRIPTION CHAR(254), +BEFORE_IMG_PREFIX VARCHAR( 4), +CONFLICT_LEVEL CHAR( 1), +CHG_UPD_TO_DEL_INS CHAR( 1), +CHGONLY CHAR( 1), +RECAPTURE CHAR( 1), +OPTION_FLAGS CHAR( 4) NOT NULL, +STOP_ON_ERROR CHAR( 1) WITH DEFAULT 'Y', +STATE CHAR( 1) WITH DEFAULT 'I', +STATE_INFO CHAR( 8)) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_REGISTERX +ON ASNCDC.IBMSNAP_REGISTER( +SOURCE_OWNER ASC, +SOURCE_TABLE ASC, +SOURCE_VIEW_QUAL ASC); + + +CREATE INDEX ASNCDC.IBMSNAP_REGISTERX1 +ON ASNCDC.IBMSNAP_REGISTER( +PHYS_CHANGE_OWNER ASC, +PHYS_CHANGE_TABLE ASC); + + +CREATE INDEX ASNCDC.IBMSNAP_REGISTERX2 +ON ASNCDC.IBMSNAP_REGISTER( +GLOBAL_RECORD ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_REGISTER VOLATILE CARDINALITY; + + + +CREATE TABLE ASNCDC.IBMSNAP_RESTART( +MAX_COMMITSEQ VARCHAR( 16) FOR BIT DATA NOT NULL, +MAX_COMMIT_TIME TIMESTAMP NOT NULL, +MIN_INFLIGHTSEQ VARCHAR( 16) FOR BIT DATA NOT NULL, +CURR_COMMIT_TIME TIMESTAMP NOT NULL, +CAPTURE_FIRST_SEQ VARCHAR( 16) FOR BIT DATA NOT NULL) + ORGANIZE BY ROW; + + + +CREATE TABLE ASNCDC.IBMSNAP_SIGNAL( +SIGNAL_TIME TIMESTAMP NOT NULL WITH DEFAULT , +SIGNAL_TYPE VARCHAR( 30) NOT NULL, +SIGNAL_SUBTYPE VARCHAR( 30), +SIGNAL_INPUT_IN VARCHAR(500), +SIGNAL_STATE CHAR( 1) NOT NULL, +SIGNAL_LSN VARCHAR( 16) FOR BIT DATA) +DATA CAPTURE CHANGES + ORGANIZE BY ROW; + + +CREATE INDEX ASNCDC.IBMSNAP_SIGNALX +ON ASNCDC.IBMSNAP_SIGNAL( +SIGNAL_TIME ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_SIGNAL VOLATILE CARDINALITY; + +CREATE TABLE ASNCDC.IBMSNAP_SUBS_COLS( +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +WHOS_ON_FIRST CHAR( 1) NOT NULL, +TARGET_OWNER VARCHAR(128) NOT NULL, +TARGET_TABLE VARCHAR(128) NOT NULL, +COL_TYPE CHAR( 1) NOT NULL, +TARGET_NAME VARCHAR(128) NOT NULL, +IS_KEY CHAR( 1) NOT NULL, +COLNO SMALLINT NOT NULL, +EXPRESSION VARCHAR(1024) NOT NULL) +ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_SUBS_COLSX +ON ASNCDC.IBMSNAP_SUBS_COLS( +APPLY_QUAL ASC, +SET_NAME ASC, +WHOS_ON_FIRST ASC, +TARGET_OWNER ASC, +TARGET_TABLE ASC, +TARGET_NAME ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_SUBS_COLS VOLATILE CARDINALITY; + + +--CREATE UNIQUE INDEX ASNCDC.IBMSNAP_SUBS_EVENTX +--ON ASNCDC.IBMSNAP_SUBS_EVENT( +--EVENT_NAME ASC, +--EVENT_TIME ASC); + + +--ALTER TABLE ASNCDC.IBMSNAP_SUBS_EVENT VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_SUBS_MEMBR( +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +WHOS_ON_FIRST CHAR( 1) NOT NULL, +SOURCE_OWNER VARCHAR(128) NOT NULL, +SOURCE_TABLE VARCHAR(128) NOT NULL, +SOURCE_VIEW_QUAL SMALLINT NOT NULL, +TARGET_OWNER VARCHAR(128) NOT NULL, +TARGET_TABLE VARCHAR(128) NOT NULL, +TARGET_CONDENSED CHAR( 1) NOT NULL, +TARGET_COMPLETE CHAR( 1) NOT NULL, +TARGET_STRUCTURE SMALLINT NOT NULL, +PREDICATES VARCHAR(1024), +MEMBER_STATE CHAR( 1), +TARGET_KEY_CHG CHAR( 1) NOT NULL, +UOW_CD_PREDICATES VARCHAR(1024), +JOIN_UOW_CD CHAR( 1), +LOADX_TYPE SMALLINT, +LOADX_SRC_N_OWNER VARCHAR( 128), +LOADX_SRC_N_TABLE VARCHAR(128)) +ORGANIZE BY ROW; + + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_SUBS_MEMBRX +ON ASNCDC.IBMSNAP_SUBS_MEMBR( +APPLY_QUAL ASC, +SET_NAME ASC, +WHOS_ON_FIRST ASC, +SOURCE_OWNER ASC, +SOURCE_TABLE ASC, +SOURCE_VIEW_QUAL ASC, +TARGET_OWNER ASC, +TARGET_TABLE ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_SUBS_MEMBR VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_SUBS_SET( +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +SET_TYPE CHAR( 1) NOT NULL, +WHOS_ON_FIRST CHAR( 1) NOT NULL, +ACTIVATE SMALLINT NOT NULL, +SOURCE_SERVER CHAR( 18) NOT NULL, +SOURCE_ALIAS CHAR( 8), +TARGET_SERVER CHAR( 18) NOT NULL, +TARGET_ALIAS CHAR( 8), +STATUS SMALLINT NOT NULL, +LASTRUN TIMESTAMP NOT NULL, +REFRESH_TYPE CHAR( 1) NOT NULL, +SLEEP_MINUTES INT, +EVENT_NAME CHAR( 18), +LASTSUCCESS TIMESTAMP, +SYNCHPOINT VARCHAR( 16) FOR BIT DATA, +SYNCHTIME TIMESTAMP, +CAPTURE_SCHEMA VARCHAR(128) NOT NULL, +TGT_CAPTURE_SCHEMA VARCHAR(128), +FEDERATED_SRC_SRVR VARCHAR( 18), +FEDERATED_TGT_SRVR VARCHAR( 18), +JRN_LIB CHAR( 10), +JRN_NAME CHAR( 10), +OPTION_FLAGS CHAR( 4) NOT NULL, +COMMIT_COUNT SMALLINT, +MAX_SYNCH_MINUTES SMALLINT, +AUX_STMTS SMALLINT NOT NULL, +ARCH_LEVEL CHAR( 4) NOT NULL) +ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_SUBS_SETX +ON ASNCDC.IBMSNAP_SUBS_SET( +APPLY_QUAL ASC, +SET_NAME ASC, +WHOS_ON_FIRST ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_SUBS_SET VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_SUBS_STMTS( +APPLY_QUAL CHAR( 18) NOT NULL, +SET_NAME CHAR( 18) NOT NULL, +WHOS_ON_FIRST CHAR( 1) NOT NULL, +BEFORE_OR_AFTER CHAR( 1) NOT NULL, +STMT_NUMBER SMALLINT NOT NULL, +EI_OR_CALL CHAR( 1) NOT NULL, +SQL_STMT VARCHAR(1024), +ACCEPT_SQLSTATES VARCHAR( 50)) +ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_SUBS_STMTSX +ON ASNCDC.IBMSNAP_SUBS_STMTS( +APPLY_QUAL ASC, +SET_NAME ASC, +WHOS_ON_FIRST ASC, +BEFORE_OR_AFTER ASC, +STMT_NUMBER ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_SUBS_STMTS VOLATILE CARDINALITY; + + +CREATE TABLE ASNCDC.IBMSNAP_UOW( +IBMSNAP_UOWID CHAR( 10) FOR BIT DATA NOT NULL, +IBMSNAP_COMMITSEQ VARCHAR( 16) FOR BIT DATA NOT NULL, +IBMSNAP_LOGMARKER TIMESTAMP NOT NULL, +IBMSNAP_AUTHTKN VARCHAR(30) NOT NULL, +IBMSNAP_AUTHID VARCHAR(128) NOT NULL, +IBMSNAP_REJ_CODE CHAR( 1) NOT NULL WITH DEFAULT , +IBMSNAP_APPLY_QUAL CHAR( 18) NOT NULL WITH DEFAULT ) + ORGANIZE BY ROW; + + +CREATE UNIQUE INDEX ASNCDC.IBMSNAP_UOWX +ON ASNCDC.IBMSNAP_UOW( +IBMSNAP_COMMITSEQ ASC, +IBMSNAP_LOGMARKER ASC); + + +ALTER TABLE ASNCDC.IBMSNAP_UOW VOLATILE CARDINALITY; + + + +CREATE TABLE ASNCDC.IBMSNAP_CAPENQ ( + LOCK_NAME CHAR(9 OCTETS) + ) + ORGANIZE BY ROW + DATA CAPTURE NONE + COMPRESS NO; diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/cdcsetup.sh b/debezium-connector-db2/src/test/docker/db2-cdc-docker/cdcsetup.sh new file mode 100644 index 000000000..5c00f8c2e --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/cdcsetup.sh @@ -0,0 +1,18 @@ +#/bin/bash + +if [ ! -f /asncdctools/src/asncdc.nlk ]; then +rc=1 +echo "Waiting for db2inst1 to exist ..." +while [ "$rc" -ne 0 ] +do + sleep 5 + id db2inst1 + rc=$? + echo '.' +done + +su -c "/asncdctools/src/dbsetup.sh $DBNAME" - db2inst1 +fi +touch /asncdctools/src/asncdc.nlk + +echo "done" \ No newline at end of file diff --git a/debezium-connector-db2/src/test/docker/db2-cdc-docker/dbsetup.sh b/debezium-connector-db2/src/test/docker/db2-cdc-docker/dbsetup.sh new file mode 100644 index 000000000..34c775be3 --- /dev/null +++ b/debezium-connector-db2/src/test/docker/db2-cdc-docker/dbsetup.sh @@ -0,0 +1,45 @@ +#/bin/bash + +echo "Compile ASN tool ..." +cd /asncdctools/src +/opt/ibm/db2/V11.5/samples/c/bldrtn asncdc + +DBNAME=$1 +DB2DIR=/opt/ibm/db2/V11.5 +rc=1 +echo "Waiting for DB2 start ( $DBNAME ) ." +while [ "$rc" -ne 0 ] +do + sleep 5 + db2 connect to $DBNAME + rc=$? + echo '.' +done + +# enable metacatalog read via JDBC +cd $HOME/sqllib/bnd +db2 bind db2schema.bnd blocking all grant public sqlerror continue + +# do a backup and restart the db +db2 backup db $DBNAME to /dev/null +db2 restart db $DBNAME + +db2 connect to $DBNAME + +cp /asncdctools/src/asncdc /database/config/db2inst1/sqllib/function +chmod 777 /database/config/db2inst1/sqllib/function + +# add UDF / start stop asncap +db2 -tvmf /asncdctools/src/asncdc_UDF.sql + +# create asntables +db2 -tvmf /asncdctools/src/asncdctables.sql + +# add UDF / add remove asntables + +db2 -tvmf /asncdctools/src/asncdcaddremove.sql + + + + +echo "done" \ No newline at end of file diff --git a/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectionIT.java b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectionIT.java new file mode 100644 index 000000000..1e513f669 --- /dev/null +++ b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectionIT.java @@ -0,0 +1,28 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ + +package io.debezium.connector.db2; + +import org.junit.Test; + +import io.debezium.connector.db2.util.TestHelper; + +/** + * Integration test for {@link Db2Connection} + * + * @author Horia Chiorean (hchiorea@redhat.com) + */ +public class Db2ConnectionIT { + + @Test + public void shouldEnableCdcForDatabase() throws Exception { + try (Db2Connection connection = TestHelper.adminConnection()) { + connection.connect(); + TestHelper.enableDbCdc(connection); + } + } + +} diff --git a/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectorIT.java b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectorIT.java new file mode 100644 index 000000000..c832f007d --- /dev/null +++ b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/Db2ConnectorIT.java @@ -0,0 +1,702 @@ +/* + * 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.db2; + +import static org.fest.assertions.Assertions.assertThat; +import static org.junit.Assert.assertNull; + +import java.sql.SQLException; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.TimeUnit; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.fest.assertions.Assertions; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import io.debezium.config.Configuration; +import io.debezium.connector.db2.Db2ConnectorConfig.SnapshotMode; +import io.debezium.connector.db2.util.TestHelper; +import io.debezium.data.Envelope; +import io.debezium.data.SchemaAndValueField; +import io.debezium.doc.FixFor; +import io.debezium.embedded.AbstractConnectorTest; +import io.debezium.junit.logging.LogInterceptor; +import io.debezium.util.Testing; + +/** + * Integration test for the Debezium DB2 connector. + * + * @author Jiri Pechanec, Luis Garcés-Erice, Peter Urbanetz + */ +public class Db2ConnectorIT extends AbstractConnectorTest { + + private Db2Connection connection; + + @Before + public void before() throws SQLException { + connection = TestHelper.testConnection(); + connection.execute("DELETE FROM ASNCDC.IBMSNAP_REGISTER"); + connection.execute( + "CREATE TABLE tablea (id int not null, cola varchar(30), primary key (id))", + "CREATE TABLE tableb (id int not null, colb varchar(30), primary key (id))", + "INSERT INTO tablea VALUES(1, 'a')"); + TestHelper.enableTableCdc(connection, "TABLEA"); + TestHelper.enableTableCdc(connection, "TABLEB"); + initializeConnectorTestFramework(); + Testing.Files.delete(TestHelper.DB_HISTORY_PATH); + Testing.Print.enable(); + } + + @After + public void after() throws SQLException { + if (connection != null) { + TestHelper.disableDbCdc(connection); + TestHelper.disableTableCdc(connection, "TABLEB"); + TestHelper.disableTableCdc(connection, "TABLEA"); + connection.execute("DROP TABLE tablea", "DROP TABLE tableb"); + connection.execute("DELETE FROM ASNCDC.IBMSNAP_REGISTER"); + connection.execute("DELETE FROM ASNCDC.IBMQREP_COLVERSION"); + connection.execute("DELETE FROM ASNCDC.IBMQREP_TABVERSION"); + connection.close(); + } + } + + @Test + public void deleteWithoutTombstone() throws Exception { + final int RECORDS_PER_TABLE = 5; + final int TABLES = 2; + final int ID_START = 10; + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .with(Db2ConnectorConfig.TOMBSTONES_ON_DELETE, false) + .build(); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + // Wait for snapshot completion + consumeRecordsByTopic(1); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_START + i; + connection.execute( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.execute( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + + TestHelper.refreshAndWait(connection); + + final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES); + + connection.execute("DELETE FROM tableB"); + + TestHelper.refreshAndWait(connection); + + final SourceRecords deleteRecords = consumeRecordsByTopic(RECORDS_PER_TABLE); + final List deleteTableA = deleteRecords.recordsForTopic("testdb.DB2INST1.TABLEA"); + final List deleteTableB = deleteRecords.recordsForTopic("testdb.DB2INST1.TABLEB"); + Assertions.assertThat(deleteTableA).isNullOrEmpty(); + Assertions.assertThat(deleteTableB).hasSize(RECORDS_PER_TABLE); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final SourceRecord deleteRecord = deleteTableB.get(i); + final List expectedDeleteRow = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, i + ID_START), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + + final Struct deleteKey = (Struct) deleteRecord.key(); + final Struct deleteValue = (Struct) deleteRecord.value(); + assertRecord((Struct) deleteValue.get("before"), expectedDeleteRow); + assertNull(deleteValue.get("after")); + } + + stopConnector(); + } + + @Test + public void updatePrimaryKey() throws Exception { + + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .build(); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + connection.execute("INSERT INTO tableb VALUES(1, 'b')"); + consumeRecordsByTopic(2); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + connection.setAutoCommit(false); + + connection.execute( + "UPDATE tablea SET id=100 WHERE id=1", + "UPDATE tableb SET id=100 WHERE id=1"); + + TestHelper.refreshAndWait(connection); + + final SourceRecords records = consumeRecordsByTopic(6); + final List tableA = records.recordsForTopic("testdb.DB2INST1.TABLEA"); + final List tableB = records.recordsForTopic("testdb.DB2INST1.TABLEB"); + Assertions.assertThat(tableA).hasSize(3); + Assertions.assertThat(tableB).hasSize(3); + + final List expectedDeleteRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedDeleteKeyA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1)); + final List expectedInsertRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedInsertKeyA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100)); + + final SourceRecord deleteRecordA = tableA.get(0); + final SourceRecord tombstoneRecordA = tableA.get(1); + final SourceRecord insertRecordA = tableA.get(2); + + final Struct deleteKeyA = (Struct) deleteRecordA.key(); + final Struct deleteValueA = (Struct) deleteRecordA.value(); + assertRecord(deleteValueA.getStruct("before"), expectedDeleteRowA); + assertRecord(deleteKeyA, expectedDeleteKeyA); + assertNull(deleteValueA.get("after")); + + final Struct tombstoneKeyA = (Struct) tombstoneRecordA.key(); + final Struct tombstoneValueA = (Struct) tombstoneRecordA.value(); + assertRecord(tombstoneKeyA, expectedDeleteKeyA); + assertNull(tombstoneValueA); + + final Struct insertKeyA = (Struct) insertRecordA.key(); + final Struct insertValueA = (Struct) insertRecordA.value(); + assertRecord(insertValueA.getStruct("after"), expectedInsertRowA); + assertRecord(insertKeyA, expectedInsertKeyA); + assertNull(insertValueA.get("before")); + + final List expectedDeleteRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + final List expectedDeleteKeyB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1)); + final List expectedInsertRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + final List expectedInsertKeyB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100)); + + final SourceRecord deleteRecordB = tableB.get(0); + final SourceRecord tombstoneRecordB = tableB.get(1); + final SourceRecord insertRecordB = tableB.get(2); + + final Struct deletekeyB = (Struct) deleteRecordB.key(); + final Struct deleteValueB = (Struct) deleteRecordB.value(); + assertRecord(deleteValueB.getStruct("before"), expectedDeleteRowB); + assertRecord(deletekeyB, expectedDeleteKeyB); + assertNull(deleteValueB.get("after")); + // assertThat(deleteValueB.getStruct("source").getInt64("event_serial_no")).isEqualTo(1L); + + final Struct tombstonekeyB = (Struct) tombstoneRecordB.key(); + final Struct tombstoneValueB = (Struct) tombstoneRecordB.value(); + assertRecord(tombstonekeyB, expectedDeleteKeyB); + assertNull(tombstoneValueB); + + final Struct insertkeyB = (Struct) insertRecordB.key(); + final Struct insertValueB = (Struct) insertRecordB.value(); + assertRecord(insertValueB.getStruct("after"), expectedInsertRowB); + assertRecord(insertkeyB, expectedInsertKeyB); + assertNull(insertValueB.get("before")); + // assertThat(insertValueB.getStruct("source").getInt64("event_serial_no")).isEqualTo(2L); + + stopConnector(); + } + + @Test + @FixFor("DBZ-1152") + public void updatePrimaryKeyWithRestartInMiddle() throws Exception { + + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .build(); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + connection.execute("INSERT INTO tableb VALUES(1, 'b')"); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + consumeRecordsByTopic(2); + + connection.setAutoCommit(false); + + connection.execute( + "UPDATE tablea SET id=100 WHERE id=1", + "UPDATE tableb SET id=100 WHERE id=1"); + + TestHelper.refreshAndWait(connection); + + final SourceRecords records1 = consumeRecordsByTopic(2); + stopConnector(); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + final SourceRecords records2 = consumeRecordsByTopic(4); + + final List tableA = records1.recordsForTopic("testdb.DB2INST1.TABLEA"); + tableA.addAll(records2.recordsForTopic("testdb.DB2INST1.TABLEA")); + final List tableB = records2.recordsForTopic("testdb.DB2INST1.TABLEB"); + Assertions.assertThat(tableA).hasSize(3); + Assertions.assertThat(tableB).hasSize(3); + + final List expectedDeleteRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedDeleteKeyA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1)); + final List expectedInsertRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedInsertKeyA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100)); + + final SourceRecord deleteRecordA = tableA.get(0); + final SourceRecord tombstoneRecordA = tableA.get(1); + final SourceRecord insertRecordA = tableA.get(2); + + final Struct deleteKeyA = (Struct) deleteRecordA.key(); + final Struct deleteValueA = (Struct) deleteRecordA.value(); + assertRecord(deleteValueA.getStruct("before"), expectedDeleteRowA); + assertRecord(deleteKeyA, expectedDeleteKeyA); + assertNull(deleteValueA.get("after")); + + final Struct tombstoneKeyA = (Struct) tombstoneRecordA.key(); + final Struct tombstoneValueA = (Struct) tombstoneRecordA.value(); + assertRecord(tombstoneKeyA, expectedDeleteKeyA); + assertNull(tombstoneValueA); + + final Struct insertKeyA = (Struct) insertRecordA.key(); + final Struct insertValueA = (Struct) insertRecordA.value(); + assertRecord(insertValueA.getStruct("after"), expectedInsertRowA); + assertRecord(insertKeyA, expectedInsertKeyA); + assertNull(insertValueA.get("before")); + + final List expectedDeleteRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + final List expectedDeleteKeyB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 1)); + final List expectedInsertRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + final List expectedInsertKeyB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, 100)); + + final SourceRecord deleteRecordB = tableB.get(0); + final SourceRecord tombstoneRecordB = tableB.get(1); + final SourceRecord insertRecordB = tableB.get(2); + + final Struct deletekeyB = (Struct) deleteRecordB.key(); + final Struct deleteValueB = (Struct) deleteRecordB.value(); + assertRecord(deleteValueB.getStruct("before"), expectedDeleteRowB); + assertRecord(deletekeyB, expectedDeleteKeyB); + assertNull(deleteValueB.get("after")); + + final Struct tombstonekeyB = (Struct) tombstoneRecordB.key(); + final Struct tombstoneValueB = (Struct) tombstoneRecordB.value(); + assertRecord(tombstonekeyB, expectedDeleteKeyB); + assertNull(tombstoneValueB); + + final Struct insertkeyB = (Struct) insertRecordB.key(); + final Struct insertValueB = (Struct) insertRecordB.value(); + assertRecord(insertValueB.getStruct("after"), expectedInsertRowB); + assertRecord(insertkeyB, expectedInsertKeyB); + assertNull(insertValueB.get("before")); + + stopConnector(); + } + + @Test + @FixFor("DBZ-1069") + public void verifyOffsets() throws Exception { + final int RECORDS_PER_TABLE = 5; + final int TABLES = 2; + final int ID_START = 10; + final int ID_RESTART = 100; + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .build(); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_START + i; + connection.execute( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.execute( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + TestHelper.refreshAndWait(connection); + for (int i = 0; !connection.getMaxLsn().isAvailable(); i++) { + if (i == 30) { + org.junit.Assert.fail("Initial changes not written to CDC structures"); + } + Testing.debug("Waiting for initial changes to be propagated to CDC structures"); + Thread.sleep(1000); + } + start(Db2Connector.class, config); + assertConnectorIsRunning(); + TestHelper.refreshAndWait(connection); + + List records = consumeRecordsByTopic(1 + RECORDS_PER_TABLE * TABLES).allRecordsInOrder(); + records = records.subList(1, records.size()); + for (Iterator it = records.iterator(); it.hasNext();) { + SourceRecord record = it.next(); + assertThat(record.sourceOffset().get("snapshot")).as("Snapshot phase").isEqualTo(true); + if (it.hasNext()) { + assertThat(record.sourceOffset().get("snapshot_completed")).as("Snapshot in progress").isEqualTo(false); + } + else { + assertThat(record.sourceOffset().get("snapshot_completed")).as("Snapshot completed").isEqualTo(true); + } + } + + stopConnector(); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_RESTART + i; + connection.execute( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.execute( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + TestHelper.refreshAndWait(connection); + + final SourceRecords sourceRecords = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES); + final List tableA = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEA"); + final List tableB = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEB"); + + Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE); + Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = i + ID_RESTART; + final SourceRecord recordA = tableA.get(i); + final SourceRecord recordB = tableB.get(i); + final List expectedRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + + final Struct valueA = (Struct) recordA.value(); + assertRecord((Struct) valueA.get("after"), expectedRowA); + assertNull(valueA.get("before")); + + final Struct valueB = (Struct) recordB.value(); + assertRecord((Struct) valueB.get("after"), expectedRowB); + assertNull(valueB.get("before")); + + assertThat(recordA.sourceOffset().get("snapshot")).as("Streaming phase").isNull(); + assertThat(recordA.sourceOffset().get("snapshot_completed")).as("Streaming phase").isNull(); + assertThat(recordA.sourceOffset().get("change_lsn")).as("LSN present").isNotNull(); + + assertThat(recordB.sourceOffset().get("snapshot")).as("Streaming phase").isNull(); + assertThat(recordB.sourceOffset().get("snapshot_completed")).as("Streaming phase").isNull(); + assertThat(recordB.sourceOffset().get("change_lsn")).as("LSN present").isNotNull(); + } + } + + @Test + public void whitelistTable() throws Exception { + final int RECORDS_PER_TABLE = 5; + final int TABLES = 1; + final int ID_START = 10; + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL_SCHEMA_ONLY) + .with(Db2ConnectorConfig.TABLE_WHITELIST, "db2inst1.tableb") + .build(); + connection.execute( + "INSERT INTO tableb VALUES(1, 'b')"); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + // Wait for snapshot completion + consumeRecordsByTopic(1); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_START + i; + connection.execute( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.execute( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + + TestHelper.refreshAndWait(connection); + + final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES); + final List tableA = records.recordsForTopic("testdb.DB2INST1.TABLEA"); + final List tableB = records.recordsForTopic("testdb.DB2INST1.TABLEB"); + Assertions.assertThat(tableA == null || tableA.isEmpty()).isTrue(); + Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE); + + stopConnector(); + } + + @Test + public void blacklistTable() throws Exception { + final int RECORDS_PER_TABLE = 5; + final int TABLES = 1; + final int ID_START = 10; + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .with(Db2ConnectorConfig.TABLE_BLACKLIST, "db2inst1.tablea") + .build(); + connection.execute( + "INSERT INTO tableb VALUES(1, 'b')"); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + // Wait for snapshot completion + consumeRecordsByTopic(1); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_START + i; + connection.execute( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.execute( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + + TestHelper.refreshAndWait(connection); + + final SourceRecords records = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES); + final List tableA = records.recordsForTopic("testdb.DB2INST1.TABLEA"); + final List tableB = records.recordsForTopic("testdb.DB2INST1.TABLEB"); + Assertions.assertThat(tableA == null || tableA.isEmpty()).isTrue(); + Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE); + + stopConnector(); + } + + private void restartInTheMiddleOfTx(boolean restartJustAfterSnapshot, boolean afterStreaming) throws Exception { + final int RECORDS_PER_TABLE = 30; + final int TABLES = 2; + final int ID_START = 10; + final int ID_RESTART = 1000; + final int HALF_ID = ID_START + RECORDS_PER_TABLE / 2; + final Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .build(); + + if (restartJustAfterSnapshot) { + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + // Wait for snapshot to be completed + consumeRecordsByTopic(1); + stopConnector(); + connection.execute("INSERT INTO tablea VALUES(-1, '-a')"); + TestHelper.refreshAndWait(connection); + + } + + start(Db2Connector.class, config, record -> { + if (!"testdb.DB2INST1.TABLEA.Envelope".equals(record.valueSchema().name())) { + return false; + } + final Struct envelope = (Struct) record.value(); + final Struct after = envelope.getStruct("after"); + final Integer id = after.getInt32("ID"); + final String value = after.getString("COLA"); + return id != null && id == HALF_ID && "a".equals(value); + }); + assertConnectorIsRunning(); + + // Wait for snapshot to be completed or a first streaming message delivered + consumeRecordsByTopic(1); + + TestHelper.enableDbCdc(connection); + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1'"); + TestHelper.refreshAndWait(connection); + + if (afterStreaming) { + connection.execute("INSERT INTO tablea VALUES(-2, '-a')"); + TestHelper.refreshAndWait(connection); + final SourceRecords records = consumeRecordsByTopic(1); + final List expectedRow = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, -2), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "-a")); + assertRecord(((Struct) records.allRecordsInOrder().get(0).value()).getStruct(Envelope.FieldName.AFTER), expectedRow); + } + + connection.setAutoCommit(false); + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_START + i; + connection.executeWithoutCommitting( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.executeWithoutCommitting( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + } + connection.connection().commit(); + + // TestHelper.waitForCDC(); + TestHelper.refreshAndWait(connection); + + List records = consumeRecordsByTopic(RECORDS_PER_TABLE).allRecordsInOrder(); + + assertThat(records).hasSize(RECORDS_PER_TABLE); + SourceRecord lastRecordForOffset = records.get(RECORDS_PER_TABLE - 1); + Struct value = (Struct) lastRecordForOffset.value(); + final List expectedLastRow = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, HALF_ID - 1), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + assertRecord((Struct) value.get("after"), expectedLastRow); + + stopConnector(); + start(Db2Connector.class, config); + assertConnectorIsRunning(); + + // TestHelper.waitForCDC(); + TestHelper.refreshAndWait(connection); + + SourceRecords sourceRecords = consumeRecordsByTopic(RECORDS_PER_TABLE); + records = sourceRecords.allRecordsInOrder(); + assertThat(records).hasSize(RECORDS_PER_TABLE); + + List tableA = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEA"); + List tableB = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEB"); + for (int i = 0; i < RECORDS_PER_TABLE / 2; i++) { + final int id = HALF_ID + i; + final SourceRecord recordA = tableA.get(i); + final SourceRecord recordB = tableB.get(i); + final List expectedRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + + final Struct valueA = (Struct) recordA.value(); + assertRecord((Struct) valueA.get("after"), expectedRowA); + assertNull(valueA.get("before")); + + final Struct valueB = (Struct) recordB.value(); + assertRecord((Struct) valueB.get("after"), expectedRowB); + assertNull(valueB.get("before")); + } + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = ID_RESTART + i; + connection.executeWithoutCommitting( + "INSERT INTO tablea VALUES(" + id + ", 'a')"); + connection.executeWithoutCommitting( + "INSERT INTO tableb VALUES(" + id + ", 'b')"); + connection.connection().commit(); + } + + TestHelper.refreshAndWait(connection); + + sourceRecords = consumeRecordsByTopic(RECORDS_PER_TABLE * TABLES); + tableA = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEA"); + tableB = sourceRecords.recordsForTopic("testdb.DB2INST1.TABLEB"); + + Assertions.assertThat(tableA).hasSize(RECORDS_PER_TABLE); + Assertions.assertThat(tableB).hasSize(RECORDS_PER_TABLE); + + for (int i = 0; i < RECORDS_PER_TABLE; i++) { + final int id = i + ID_RESTART; + final SourceRecord recordA = tableA.get(i); + final SourceRecord recordB = tableB.get(i); + final List expectedRowA = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLA", Schema.OPTIONAL_STRING_SCHEMA, "a")); + final List expectedRowB = Arrays.asList( + new SchemaAndValueField("ID", Schema.INT32_SCHEMA, id), + new SchemaAndValueField("COLB", Schema.OPTIONAL_STRING_SCHEMA, "b")); + + final Struct valueA = (Struct) recordA.value(); + assertRecord((Struct) valueA.get("after"), expectedRowA); + assertNull(valueA.get("before")); + + final Struct valueB = (Struct) recordB.value(); + assertRecord((Struct) valueB.get("after"), expectedRowB); + assertNull(valueB.get("before")); + } + } + + @Test + @FixFor("DBZ-1128") + public void restartInTheMiddleOfTxAfterSnapshot() throws Exception { + restartInTheMiddleOfTx(true, false); + } + + @Test + @FixFor("DBZ-1128") + public void restartInTheMiddleOfTxAfterCompletedTx() throws Exception { + restartInTheMiddleOfTx(false, true); + } + + @Test + // @FixFor("DBZ-1128") + public void restartInTheMiddleOfTx() throws Exception { + restartInTheMiddleOfTx(false, false); + } + + @Test + @FixFor("DBZ-1242") + public void testEmptySchemaWarningAfterApplyingFilters() throws Exception { + // This captures all logged messages, allowing us to verify log message was written. + final LogInterceptor logInterceptor = new LogInterceptor(); + + Configuration config = TestHelper.defaultConfig() + .with(Db2ConnectorConfig.SNAPSHOT_MODE, SnapshotMode.INITIAL) + .with(Db2ConnectorConfig.TABLE_WHITELIST, "my_products") + .build(); + + start(Db2Connector.class, config); + assertConnectorIsRunning(); + waitForAvailableRecords(100, TimeUnit.MILLISECONDS); + + stopConnector(value -> assertThat(logInterceptor.containsWarnMessage(NO_MONITORED_TABLES_WARNING)).isTrue()); + } + + private void assertRecord(Struct record, List expected) { + expected.forEach(schemaAndValueField -> schemaAndValueField.assertFor(record)); + } +} diff --git a/debezium-connector-db2/src/test/java/io/debezium/connector/db2/SourceInfoTest.java b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/SourceInfoTest.java new file mode 100644 index 000000000..f5aa3ff5b --- /dev/null +++ b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/SourceInfoTest.java @@ -0,0 +1,101 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.db2; + +import static org.fest.assertions.Assertions.assertThat; + +import java.time.Instant; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.junit.Before; +import org.junit.Test; + +import io.debezium.config.Configuration; +import io.debezium.connector.AbstractSourceInfoStructMaker; +import io.debezium.connector.SnapshotRecord; +import io.debezium.relational.TableId; + +public class SourceInfoTest { + + private SourceInfo source; + + @Before + public void beforeEach() { + final Db2ConnectorConfig connectorConfig = new Db2ConnectorConfig( + Configuration.create() + .with(Db2ConnectorConfig.SERVER_NAME, "serverX") + .with(Db2ConnectorConfig.DATABASE_NAME, "c") + .build()); + source = new SourceInfo(connectorConfig); + source.setChangeLsn(Lsn.valueOf(new byte[]{ 0x01 })); + source.setCommitLsn(Lsn.valueOf(new byte[]{ 0x02 })); + source.setSnapshot(SnapshotRecord.TRUE); + source.setSourceTime(Instant.ofEpochMilli(3000)); + source.setTableId(new TableId("c", "s", "t")); + } + + @Test + public void versionIsPresent() { + assertThat(source.struct().getString(SourceInfo.DEBEZIUM_VERSION_KEY)).isEqualTo(Module.version()); + } + + @Test + public void connectorIsPresent() { + assertThat(source.struct().getString(SourceInfo.DEBEZIUM_CONNECTOR_KEY)).isEqualTo(Module.name()); + } + + @Test + public void serverNameIsPresent() { + assertThat(source.struct().getString(SourceInfo.SERVER_NAME_KEY)).isEqualTo("serverX"); + } + + @Test + public void changeLsnIsPresent() { + assertThat(source.struct().getString(SourceInfo.CHANGE_LSN_KEY)).isEqualTo(Lsn.valueOf(new byte[]{ 0x01 }).toString()); + } + + @Test + public void commitLsnIsPresent() { + assertThat(source.struct().getString(SourceInfo.COMMIT_LSN_KEY)).isEqualTo(Lsn.valueOf(new byte[]{ 0x02 }).toString()); + } + + @Test + public void snapshotIsPresent() { + assertThat(source.struct().getString(SourceInfo.SNAPSHOT_KEY)).isEqualTo("true"); + } + + @Test + public void timestampIsPresent() { + assertThat(source.struct().getInt64(SourceInfo.TIMESTAMP_KEY)).isEqualTo(3000); + } + + @Test + public void tableIdIsPresent() { + assertThat(source.struct().getString(SourceInfo.DATABASE_NAME_KEY)).isEqualTo("c"); + assertThat(source.struct().getString(SourceInfo.SCHEMA_NAME_KEY)).isEqualTo("s"); + assertThat(source.struct().getString(SourceInfo.TABLE_NAME_KEY)).isEqualTo("t"); + } + + @Test + public void schemaIsCorrect() { + final Schema schema = SchemaBuilder.struct() + .name("io.debezium.connector.db2.Source") + .field("version", Schema.STRING_SCHEMA) + .field("connector", Schema.STRING_SCHEMA) + .field("name", Schema.STRING_SCHEMA) + .field("ts_ms", Schema.INT64_SCHEMA) + .field("snapshot", AbstractSourceInfoStructMaker.SNAPSHOT_RECORD_SCHEMA) + .field("db", Schema.STRING_SCHEMA) + .field("schema", Schema.STRING_SCHEMA) + .field("table", Schema.STRING_SCHEMA) + .field("change_lsn", Schema.OPTIONAL_STRING_SCHEMA) + .field("commit_lsn", Schema.OPTIONAL_STRING_SCHEMA) + .build(); + + assertThat(source.struct().schema()).isEqualTo(schema); + } +} diff --git a/debezium-connector-db2/src/test/java/io/debezium/connector/db2/util/TestHelper.java b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/util/TestHelper.java new file mode 100644 index 000000000..c45b62f49 --- /dev/null +++ b/debezium-connector-db2/src/test/java/io/debezium/connector/db2/util/TestHelper.java @@ -0,0 +1,213 @@ +/* + * 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.db2.util; + +import java.lang.management.ManagementFactory; +import java.nio.file.Path; +import java.sql.Clob; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; +import java.time.Duration; +import java.util.Objects; + +import javax.management.InstanceNotFoundException; +import javax.management.MBeanServer; +import javax.management.ObjectName; + +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.debezium.config.Configuration; +import io.debezium.connector.db2.Db2Connection; +import io.debezium.connector.db2.Db2ConnectorConfig; +import io.debezium.jdbc.JdbcConfiguration; +import io.debezium.relational.RelationalDatabaseConnectorConfig; +import io.debezium.relational.history.FileDatabaseHistory; +import io.debezium.util.Clock; +import io.debezium.util.Metronome; +import io.debezium.util.Testing; + +/** + * @author Horia Chiorean (hchiorea@redhat.com), Luis Garcés-Erice + */ +public class TestHelper { + + private static final Logger LOGGER = LoggerFactory.getLogger(TestHelper.class); + + public static final Path DB_HISTORY_PATH = Testing.Files.createTestingPath("file-db-history-connect.txt").toAbsolutePath(); + public static final String TEST_DATABASE = "testdb"; + public static final int WAIT_FOR_CDC = 60 * 1000; + + private static final String STATEMENTS_PLACEHOLDER = "#"; + + private static final String ENABLE_DB_CDC = "VALUES ASNCDC.ASNCDCSERVICES('start','asncdc')"; + private static final String DISABLE_DB_CDC = "VALUES ASNCDC.ASNCDCSERVICES('stop','asncdc')"; + private static final String STATUS_DB_CDC = "VALUES ASNCDC.ASNCDCSERVICES('status','asncdc')"; + private static final String ENABLE_TABLE_CDC = "CALL ASNCDC.ADDTABLE('DB2INST1', '#' )"; + private static final String DISABLE_TABLE_CDC = "CALL ASNCDC.REMOVETABLE('DB2INST1', '#' )"; + private static final String RESTART_ASN_CDC = "VALUES ASNCDC.ASNCDCSERVICES('reinit','asncdc')"; + + public static JdbcConfiguration adminJdbcConfig() { + return JdbcConfiguration.copy(Configuration.fromSystemProperties("database.")) + .withDefault(JdbcConfiguration.DATABASE, "testdb") + .withDefault(JdbcConfiguration.HOSTNAME, "localhost") + .withDefault(JdbcConfiguration.PORT, 50000) + .withDefault(JdbcConfiguration.USER, "db2inst1") + .withDefault(JdbcConfiguration.PASSWORD, "admin") + .build(); + } + + public static JdbcConfiguration defaultJdbcConfig() { + return JdbcConfiguration.copy(Configuration.fromSystemProperties("database.")) + .withDefault(JdbcConfiguration.DATABASE, TEST_DATABASE) + .withDefault(JdbcConfiguration.HOSTNAME, "localhost") + .withDefault(JdbcConfiguration.PORT, 50000) + .withDefault(JdbcConfiguration.USER, "db2inst1") + .withDefault(JdbcConfiguration.PASSWORD, "admin") + .build(); + } + + /** + * Returns a default configuration suitable for most test cases. Can be amended/overridden in individual tests as + * needed. + */ + public static Configuration.Builder defaultConfig() { + JdbcConfiguration jdbcConfiguration = defaultJdbcConfig(); + Configuration.Builder builder = Configuration.create(); + + jdbcConfiguration.forEach( + (field, value) -> builder.with(Db2ConnectorConfig.DATABASE_CONFIG_PREFIX + field, value)); + + return builder.with(RelationalDatabaseConnectorConfig.SERVER_NAME, "testdb") + .with(Db2ConnectorConfig.DATABASE_HISTORY, FileDatabaseHistory.class) + .with(FileDatabaseHistory.FILE_PATH, DB_HISTORY_PATH); + } + + public static Db2Connection adminConnection() { + return new Db2Connection(TestHelper.adminJdbcConfig()); + } + + public static Db2Connection testConnection() { + return new Db2Connection(TestHelper.defaultJdbcConfig()); + } + + /** + * Enables CDC for a given database, if not already enabled. + * + * @throws SQLException + * if anything unexpected fails + */ + public static void enableDbCdc(Db2Connection connection) throws SQLException { + connection.execute(ENABLE_DB_CDC); + Statement stmt = connection.connection().createStatement(); + boolean isNotrunning = true; + int count = 0; + while (isNotrunning) { + ResultSet rs = stmt.executeQuery(STATUS_DB_CDC); + while (rs.next()) { + Clob clob = rs.getClob(1); + String test = clob.getSubString(1, (int) clob.length()); + if (test.contains("is doing work")) { + isNotrunning = false; + } + else { + try { + Thread.sleep(1000); + } + catch (InterruptedException e) { + } + } + if (count++ > 30) { + throw new SQLException("ASNCAP server did not start."); + } + } + } + } + + /** + * Disables CDC for a given database, if not already disabled. + * + * @throws SQLException + * if anything unexpected fails + */ + public static void disableDbCdc(Db2Connection connection) throws SQLException { + connection.execute(DISABLE_DB_CDC); + } + + /** + * Enables CDC for a table if not already enabled and generates the wrapper + * functions for that table. + * + * @param name + * the name of the table, may not be {@code null} + * @throws SQLException if anything unexpected fails + */ + public static void enableTableCdc(Db2Connection connection, String name) throws SQLException { + Objects.requireNonNull(name); + String enableCdcForTableStmt = ENABLE_TABLE_CDC.replace(STATEMENTS_PLACEHOLDER, name); + connection.execute(enableCdcForTableStmt); + + connection.execute("UPDATE ASNCDC.IBMSNAP_REGISTER SET STATE = 'A' WHERE SOURCE_OWNER = 'DB2INST1' AND SOURCE_TABLE = '" + name + "'"); + connection.execute(RESTART_ASN_CDC); + } + + /** + * Disables CDC for a table for which it was enabled before. + * + * @param name + * the name of the table, may not be {@code null} + * @throws SQLException if anything unexpected fails + */ + public static void disableTableCdc(Db2Connection connection, String name) throws SQLException { + Objects.requireNonNull(name); + String disableCdcForTableStmt = DISABLE_TABLE_CDC.replace(STATEMENTS_PLACEHOLDER, name); + connection.execute(disableCdcForTableStmt); + connection.execute(RESTART_ASN_CDC); + } + + public static void waitForSnapshotToBeCompleted() throws InterruptedException { + int waitForSeconds = 60; + final MBeanServer mbeanServer = ManagementFactory.getPlatformMBeanServer(); + final Metronome metronome = Metronome.sleeper(Duration.ofSeconds(1), Clock.system()); + + while (true) { + if (waitForSeconds-- <= 0) { + Assert.fail("Snapshot was not completed on time"); + } + try { + final boolean completed = (boolean) mbeanServer.getAttribute(new ObjectName("debezium.sql_server:type=connector-metrics,context=snapshot,server=server1"), + "SnapshotCompleted"); + if (completed) { + break; + } + } + catch (InstanceNotFoundException e) { + // Metrics has not started yet + } + catch (Exception e) { + throw new IllegalStateException(e); + } + metronome.pause(); + } + } + + public static void refreshAndWait(Db2Connection connection) throws SQLException { + connection.execute(RESTART_ASN_CDC); + waitForCDC(); + } + + public static void waitForCDC() { + try { + Thread.sleep(WAIT_FOR_CDC); + } + catch (Exception e) { + + } + } +} diff --git a/debezium-connector-db2/src/test/resources/inventory.sql b/debezium-connector-db2/src/test/resources/inventory.sql new file mode 100644 index 000000000..1b68121c7 --- /dev/null +++ b/debezium-connector-db2/src/test/resources/inventory.sql @@ -0,0 +1,86 @@ + + +-- Create and populate our products using a single insert with many rows +CREATE TABLE products ( + id INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY + (START WITH 101, INCREMENT BY 1) PRIMARY KEY, + name VARCHAR(255) NOT NULL, + description VARCHAR(512), + weight FLOAT +); +INSERT INTO products(name,description,weight) + VALUES ('scooter','Small 2-wheel scooter',3.14); +INSERT INTO products(name,description,weight) + VALUES ('car battery','12V car battery',8.1); +INSERT INTO products(name,description,weight) + VALUES ('12-pack drill bits','12-pack of drill bits with sizes ranging from #40 to #3',0.8); +INSERT INTO products(name,description,weight) + VALUES ('hammer','12oz carpenter''s hammer',0.75); +INSERT INTO products(name,description,weight) + VALUES ('hammer','14oz carpenter''s hammer',0.875); +INSERT INTO products(name,description,weight) + VALUES ('hammer','16oz carpenter''s hammer',1.0); +INSERT INTO products(name,description,weight) + VALUES ('rocks','box of assorted rocks',5.3); +INSERT INTO products(name,description,weight) + VALUES ('jacket','water resistent black wind breaker',0.1); +INSERT INTO products(name,description,weight) + VALUES ('spare tire','24 inch spare tire',22.2); + + +CREATE TABLE products_on_hand ( + product_id INTEGER NOT NULL PRIMARY KEY, + quantity INTEGER NOT NULL, + FOREIGN KEY (product_id) REFERENCES products(id) +); +INSERT INTO products_on_hand VALUES (101,3); +INSERT INTO products_on_hand VALUES (102,8); +INSERT INTO products_on_hand VALUES (103,18); +INSERT INTO products_on_hand VALUES (104,4); +INSERT INTO products_on_hand VALUES (105,5); +INSERT INTO products_on_hand VALUES (106,0); +INSERT INTO products_on_hand VALUES (107,44); +INSERT INTO products_on_hand VALUES (108,2); +INSERT INTO products_on_hand VALUES (109,5); + + + +CREATE TABLE customers ( + id INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY + (START WITH 1001, INCREMENT BY 1) PRIMARY KEY, + first_name VARCHAR(255) NOT NULL, + last_name VARCHAR(255) NOT NULL, + email VARCHAR(255) NOT NULL UNIQUE +); +INSERT INTO customers(first_name,last_name,email) + VALUES ('Sally','Thomas','sally.thomas@acme.com'); +INSERT INTO customers(first_name,last_name,email) + VALUES ('George','Bailey','gbailey@foobar.com'); +INSERT INTO customers(first_name,last_name,email) + VALUES ('Edward','Walker','ed@walker.com'); +INSERT INTO customers(first_name,last_name,email) + VALUES ('Anne','Kretchmar','annek@noanswer.org'); + + +CREATE TABLE orders ( + id INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY + (START WITH 10001, INCREMENT BY 1) PRIMARY KEY, + order_date DATE NOT NULL, + purchaser INTEGER NOT NULL, + quantity INTEGER NOT NULL, + product_id INTEGER NOT NULL, + FOREIGN KEY (purchaser) REFERENCES customers(id), + FOREIGN KEY (product_id) REFERENCES products(id) +); +INSERT INTO orders(order_date,purchaser,quantity,product_id) + VALUES ('2016-01-16', 1001, 1, 102); +INSERT INTO orders(order_date,purchaser,quantity,product_id) + VALUES ('2016-01-17', 1002, 2, 105); +INSERT INTO orders(order_date,purchaser,quantity,product_id) + VALUES ('2016-02-19', 1002, 2, 106); +INSERT INTO orders(order_date,purchaser,quantity,product_id) + VALUES ('2016-02-21', 1003, 1, 107); + + + + diff --git a/debezium-connector-db2/src/test/resources/log4j.properties b/debezium-connector-db2/src/test/resources/log4j.properties new file mode 100644 index 000000000..9d728aed9 --- /dev/null +++ b/debezium-connector-db2/src/test/resources/log4j.properties @@ -0,0 +1,15 @@ +# Direct log messages to stdout +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.Target=System.out +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %X{dbz.connectorType}|%X{dbz.connectorName}|%X{dbz.connectorContext} %m [%c]%n + +# Root logger option +log4j.rootLogger=INFO, stdout + +# Set up the default logging to be INFO level, then override specific units +log4j.logger.io.debezium=INFO +log4j.logger.io.debezium.embedded.EmbeddedEngine$EmbeddedConfig=WARN +#log4j.logger.io.debezium.embedded.EmbeddedEngine=DEBUG +log4j.logger.io.debezium.core=DEBUG +log4j.logger.io.debezium.connector.db2=DEBUG diff --git a/debezium-connector-db2/src/test/resources/startup-agent.sql b/debezium-connector-db2/src/test/resources/startup-agent.sql new file mode 100644 index 000000000..ae9150e34 --- /dev/null +++ b/debezium-connector-db2/src/test/resources/startup-agent.sql @@ -0,0 +1 @@ +VALUES ASNCDC.ASNCDCSERVICES('start','asncdc'); \ No newline at end of file diff --git a/debezium-connector-db2/src/test/resources/startup-cdc-demo.sql b/debezium-connector-db2/src/test/resources/startup-cdc-demo.sql new file mode 100644 index 000000000..319ca926b --- /dev/null +++ b/debezium-connector-db2/src/test/resources/startup-cdc-demo.sql @@ -0,0 +1,9 @@ + +VALUES ASNCDC.ASNCDCSERVICES('status','asncdc'); + +CALL ASNCDC.ADDTABLE('db2inst1', 'products' ); +CALL ASNCDC.ADDTABLE('db2inst1', 'products_on_hand' ); +CALL ASNCDC.ADDTABLE('db2inst1', 'customers' ); +CALL ASNCDC.ADDTABLE('db2inst1', 'orders' ); + +VALUES ASNCDC.ASNCDCSERVICES('reinit','asncdc'); \ No newline at end of file diff --git a/pom.xml b/pom.xml index e67d1d62b..ea94feb6d 100644 --- a/pom.xml +++ b/pom.xml @@ -45,6 +45,11 @@ debezium-connector-cassandra ${project.version} + + io.debezium + debezium-connector-db2 + ${project.version} + io.debezium debezium-core @@ -81,6 +86,7 @@ debezium-connector-cassandra + debezium-connector-db2