From e9f9d9e7eeeac93b90f395188a59f1b4bf3d607e Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic Date: Thu, 24 Oct 2024 16:04:56 +0200 Subject: [PATCH] DBZ-8363 creation of single test artifact --- cassandra-3/pom.xml | 6 +- .../cassandra/Cassandra3TypeProvider.java | 7 - .../AbstractCommitLogProcessorTest.java | 113 ----------- .../connector/cassandra/BatchTest.java | 97 --------- .../cassandra/Cassandra3TestProvider.java | 79 ++++++++ ...ngPartitionKeyCommitLogProcessingTest.java | 58 ------ .../CommitLogRealTimeParserTest.java | 29 +-- .../EmbeddedCassandra3ConnectorTestBase.java | 28 --- .../MultipleTablesProcessingTest.java | 107 ---------- .../cassandra/QueueProcessorTest.java | 22 --- .../RowInsertionCommitLogProcessingTest.java | 51 ----- .../cassandra/SnapshotProcessorTest.java | 177 ----------------- ...nector.cassandra.spi.CassandraTestProvider | 1 + .../src/test/resources/docker/cassandra.yaml | 4 +- cassandra-4/pom.xml | 6 +- .../cassandra/Cassandra4TypeProvider.java | 7 - .../AbstractCommitLogProcessorTest.java | 115 ----------- .../cassandra/Cassandra4TestProvider.java | 79 ++++++++ ...ngPartitionKeyCommitLogProcessingTest.java | 58 ------ .../CommitLogRealTimeParserTest.java | 109 ----------- .../EmbeddedCassandra4ConnectorTestBase.java | 28 --- ...KeyRowDeletionCommitLogProcessingTest.java | 47 ----- .../cassandra/QueueProcessorTest.java | 22 --- ...RangeTombstoneCommitLogProcessingTest.java | 96 --------- ...onModificationCommitLogProcessingTest.java | 51 ----- ...nector.cassandra.spi.CassandraTestProvider | 1 + cassandra-5/pom.xml | 6 +- .../cassandra/Cassandra5TypeProvider.java | 7 - .../connector/cassandra/BatchTest.java | 97 --------- .../cassandra/Cassandra5TestProvider.java | 79 ++++++++ .../EmbeddedCassandra5ConnectorTestBase.java | 28 --- .../MultipleTablesProcessingTest.java | 107 ---------- ...KeyRowDeletionCommitLogProcessingTest.java | 47 ----- .../cassandra/QueueProcessorTest.java | 22 --- ...RangeTombstoneCommitLogProcessingTest.java | 96 --------- .../cassandra/SchemaChangeListenerTest.java | 63 ------ .../cassandra/SnapshotProcessorTest.java | 177 ----------------- ...nector.cassandra.spi.CassandraTestProvider | 1 + core/pom.xml | 11 -- .../cassandra/CassandraConnectorConfig.java | 19 ++ .../cassandra/CassandraConnectorContext.java | 96 ++------- .../CassandraConnectorTaskTemplate.java | 4 +- .../cassandra/CassandraTypeProvider.java | 2 - .../DefaultCassandraConnectorContext.java | 122 ++++++++++++ .../cassandra/CommitLogPostProcessorTest.java | 6 +- .../cassandra/FileOffsetWriterTest.java | 1 + .../connector/cassandra/utils/TestUtils.java | 85 ++++++++ dse/pom.xml | 11 +- .../connector/dse/DseTypeProvider.java | 7 - .../dse/AbstractCommitLogProcessorTest.java | 121 ------------ .../io/debezium/connector/dse/BatchTest.java | 97 --------- ...ngPartitionKeyCommitLogProcessingTest.java | 61 ------ .../dse/CommitLogRealTimeParserTest.java | 116 ----------- .../connector/dse/DseConnectorTestBase.java | 30 --- .../connector/dse/DseTestProvider.java | 86 ++++++++ .../dse/MultipleTablesProcessingTest.java | 110 ----------- ...KeyRowDeletionCommitLogProcessingTest.java | 50 ----- .../connector/dse/QueueProcessorTest.java | 25 --- ...RangeTombstoneCommitLogProcessingTest.java | 98 ---------- ...onModificationCommitLogProcessingTest.java | 55 ------ .../connector/dse/SnapshotProcessorTest.java | 185 ------------------ ...nector.cassandra.spi.CassandraTestProvider | 1 + pom.xml | 36 ++-- tests/pom.xml | 50 +++++ .../AbstractCommitLogProcessorTest.java | 94 +++++---- .../connector/cassandra/BatchTest.java | 8 +- .../cassandra/CassandraConnectorTestBase.java | 36 ++-- ...ngPartitionKeyCommitLogProcessingTest.java | 8 +- .../MultipleTablesProcessingTest.java | 14 +- ...KeyRowDeletionCommitLogProcessingTest.java | 8 +- .../cassandra/QueueProcessorTest.java | 10 +- ...RangeTombstoneCommitLogProcessingTest.java | 8 +- ...onModificationCommitLogProcessingTest.java | 14 +- .../cassandra/SchemaChangeListenerTest.java | 25 ++- .../cassandra/SnapshotProcessorTest.java | 78 ++++---- .../cassandra/TestingKafkaRecordEmitter.java | 0 .../cassandra/spi/CassandraTestProvider.java | 19 ++ .../cassandra/spi/CommitLogProcessing.java | 19 ++ .../cassandra/spi/ProvidersResolver.java | 24 +++ .../connector/cassandra/utils}/TestUtils.java | 65 +----- 80 files changed, 918 insertions(+), 3125 deletions(-) delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/BatchTest.java create mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/Cassandra3TestProvider.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java rename {cassandra-5 => cassandra-3}/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java (83%) delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra3ConnectorTestBase.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/RowInsertionCommitLogProcessingTest.java delete mode 100644 cassandra-3/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java create mode 100644 cassandra-3/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java create mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/Cassandra4TestProvider.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra4ConnectorTestBase.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java delete mode 100644 cassandra-4/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java create mode 100644 cassandra-4/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/Cassandra5TestProvider.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java delete mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java create mode 100644 cassandra-5/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider create mode 100644 core/src/main/java/io/debezium/connector/cassandra/DefaultCassandraConnectorContext.java create mode 100644 core/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/AbstractCommitLogProcessorTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/BatchTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/ClusteringPartitionKeyCommitLogProcessingTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/CommitLogRealTimeParserTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/DseConnectorTestBase.java create mode 100644 dse/src/test/java/io/debezium/connector/dse/DseTestProvider.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/MultipleTablesProcessingTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/PartitionKeyRowDeletionCommitLogProcessingTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/QueueProcessorTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/RangeTombstoneCommitLogProcessingTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/RowInsertionModificationCommitLogProcessingTest.java delete mode 100644 dse/src/test/java/io/debezium/connector/dse/SnapshotProcessorTest.java create mode 100644 dse/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider create mode 100644 tests/pom.xml rename {cassandra-5 => tests}/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java (51%) rename {cassandra-4 => tests}/src/test/java/io/debezium/connector/cassandra/BatchTest.java (93%) rename {core => tests}/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java (82%) rename {cassandra-5 => tests}/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java (88%) rename {cassandra-4 => tests}/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java (88%) rename {cassandra-3 => tests}/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java (85%) rename core/src/test/java/io/debezium/connector/cassandra/AbstractQueueProcessorTest.java => tests/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java (93%) rename {cassandra-3 => tests}/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java (93%) rename {cassandra-5 => tests}/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java (77%) rename {cassandra-4 => tests}/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java (75%) rename {cassandra-4 => tests}/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java (75%) rename {core => tests}/src/test/java/io/debezium/connector/cassandra/TestingKafkaRecordEmitter.java (100%) create mode 100644 tests/src/test/java/io/debezium/connector/cassandra/spi/CassandraTestProvider.java create mode 100644 tests/src/test/java/io/debezium/connector/cassandra/spi/CommitLogProcessing.java create mode 100644 tests/src/test/java/io/debezium/connector/cassandra/spi/ProvidersResolver.java rename {core/src/test/java/io/debezium/connector/cassandra => tests/src/test/java/io/debezium/connector/cassandra/utils}/TestUtils.java (75%) diff --git a/cassandra-3/pom.xml b/cassandra-3/pom.xml index 8d9038d0..aeae78f5 100644 --- a/cassandra-3/pom.xml +++ b/cassandra-3/pom.xml @@ -36,10 +36,9 @@ io.debezium - debezium-connector-cassandra-core + debezium-connector-cassandra-tests ${project.version} tests - test-jar test @@ -86,6 +85,9 @@ ${version.cassandra3} ${project.basedir}/src/test/resources/docker + + io.debezium:debezium-connector-cassandra-tests + diff --git a/cassandra-3/src/main/java/io/debezium/connector/cassandra/Cassandra3TypeProvider.java b/cassandra-3/src/main/java/io/debezium/connector/cassandra/Cassandra3TypeProvider.java index d44cb833..d631962e 100644 --- a/cassandra-3/src/main/java/io/debezium/connector/cassandra/Cassandra3TypeProvider.java +++ b/cassandra-3/src/main/java/io/debezium/connector/cassandra/Cassandra3TypeProvider.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.function.Function; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BooleanType; @@ -110,10 +109,4 @@ public Object deserialize(Object abstractType, ByteBuffer bb) { public Function baseTypeForReversedType() { return abstractType -> ((AbstractType) abstractType).isReversed() ? ((ReversedType) abstractType).baseType : abstractType; } - - @Override - public String getClusterName() { - return DatabaseDescriptor.getClusterName(); - } - } diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java deleted file mode 100644 index bbe0bb46..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static java.lang.String.format; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.commitlog.CommitLogReadHandler; -import org.apache.cassandra.db.commitlog.CommitLogReader; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import io.debezium.connector.base.ChangeEventQueue; - -public abstract class AbstractCommitLogProcessorTest extends EmbeddedCassandra3ConnectorTestBase { - public CassandraConnectorContext context; - private CommitLogProcessorMetrics metrics = new CommitLogProcessorMetrics(); - private CommitLogReadHandler commitLogReadHandler; - - @Before - public void setUp() throws Exception { - initialiseData(); - context = generateTaskContext(); - await().atMost(Duration.ofSeconds(60)).until(() -> context.getSchemaHolder() - .getKeyValueSchema(new KeyspaceTable(TEST_KEYSPACE_NAME, TEST_TABLE_NAME)) != null); - commitLogReadHandler = new Cassandra3CommitLogReadHandlerImpl(context, metrics); - metrics.registerMetrics(); - } - - @After - public void tearDown() throws Exception { - deleteTestOffsets(context); - metrics.unregisterMetrics(); - deleteTestKeyspaceTables(); - context.cleanUp(); - } - - @Test - public void test() throws Exception { - verifyEvents(); - } - - public abstract void initialiseData() throws Exception; - - public abstract void verifyEvents() throws Exception; - - public void createTable(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - } - - public void createTable2(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - } - - public void createTable(String query, String keyspace, String tableName) throws Exception { - runCql(format(query, keyspace, tableName)); - } - - public List getEvents(final int expectedSize) throws Exception { - ChangeEventQueue queue = context.getQueues().get(0); - final List events = new ArrayList<>(); - await().atMost(60, TimeUnit.SECONDS).until(() -> { - readLogs(queue); - events.clear(); - events.addAll(queue.poll()); - return events.size() >= expectedSize; - }); - assertEquals(expectedSize, events.size()); - return events; - } - - private void readLogs(ChangeEventQueue queue) throws Exception { - // check to make sure there are no records in the queue to begin with - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - // process the logs in commit log directory - File cdcLoc = new File(DatabaseDescriptor.getCommitLogLocation()); - File[] commitLogs = CommitLogUtil.getCommitLogs(cdcLoc); - - CommitLogReader reader = new CommitLogReader(); - - for (File commitLog : commitLogs) { - reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); - } - } - - public void assertEventTypes(List events, Event.EventType eventType, Record.Operation... operations) { - assertEquals(events.size(), operations.length); - for (int i = 0; i < events.size(); i++) { - Record record = (Record) events.get(i); - assertEquals(record.getEventType(), eventType); - assertEquals(operations[i], record.getOp()); - } - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/BatchTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/BatchTest.java deleted file mode 100644 index 7a09259e..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/BatchTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.core.cql.BatchType.LOGGED; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static com.datastax.oss.driver.api.querybuilder.relation.Relation.column; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; - -import java.util.HashMap; - -import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder; - -public class BatchTest extends AbstractCommitLogProcessorTest { - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE %s.%s (\n" + - " p1 text,\n" + - " p2 text,\n" + - " p3 text,\n" + - " c1 text,\n" + - " col1 text,\n" + - " col2 text,\n" + - " amap map,\n" + - " PRIMARY KEY ((p1, p2, p3), c1)\n" + - ") WITH CLUSTERING ORDER BY (c1 ASC)\n" + - " AND cdc = true;"); - - runCql(new BatchStatementBuilder(LOGGED) - .addStatement(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .usingTimestamp(1683810323861L) - .where(column("p1").isEqualTo(literal("abc"))) - .where(column("p2").isEqualTo(literal("p2value"))) - .where(column("p3").isEqualTo(literal("p3value"))) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .value("col1", literal("")) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value1")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value2")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - assertEventTypes(getEvents(4), CHANGE_EVENT, DELETE, INSERT, INSERT, INSERT); - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/Cassandra3TestProvider.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/Cassandra3TestProvider.java new file mode 100644 index 00000000..c12455af --- /dev/null +++ b/cassandra-3/src/test/java/io/debezium/connector/cassandra/Cassandra3TestProvider.java @@ -0,0 +1,79 @@ +/* + * 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.cassandra; + +import java.io.File; +import java.io.IOException; + +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.commitlog.CommitLogReader; + +import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.spi.CommitLogProcessing; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; + +public class Cassandra3TestProvider implements CassandraTestProvider { + @Override + public CommitLogProcessing provideCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + return new Cassandra3CommitLogProcessing(context, metrics); + } + + private static class Cassandra3CommitLogProcessing implements CommitLogProcessing { + + private final CommitLogReadHandler commitLogReadHandler; + private final CommitLogSegmentReader commitLogSegmentReader; + private final CassandraConnectorContext context; + + Cassandra3CommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + commitLogReadHandler = new Cassandra3CommitLogReadHandlerImpl(context, metrics); + commitLogSegmentReader = new Cassandra3CommitLogSegmentReader(context, metrics); + this.context = context; + } + + @Override + public void readAllCommitLogs(File[] commitLogs) throws IOException { + CommitLogReader reader = new CommitLogReader(); + File cdcLoc = new File(context.getCassandraConnectorConfig().getCDCLogLocation()); + for (File commitLog : CommitLogUtil.getCommitLogs(cdcLoc)) { + reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); + } + } + + @Override + public void readCommitLogSegment(File file, long segmentId, int position) throws IOException { + commitLogSegmentReader.readCommitLogSegment(file, segmentId, position); + } + + @Override + public CommitLogSegmentReader getCommitLogSegmentReader() { + return commitLogSegmentReader; + } + } + + @Override + public CassandraConnectorContext provideContext(Configuration configuration) throws Exception { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra3TypeProvider provider = new Cassandra3TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(config, + new CassandraConnectorTask.Cassandra3SchemaLoader(), + new CassandraConnectorTask.Cassandra3SchemaChangeListenerProvider(), + new FileOffsetWriter(config)); + } + + @Override + public CassandraConnectorContext provideContextWithoutSchemaManagement(Configuration configuration) { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra3TypeProvider provider = new Cassandra3TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(new CassandraConnectorConfig(configuration)); + } +} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java deleted file mode 100644 index d720f720..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class ClusteringPartitionKeyCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record delete = (Record) events.get(2); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java similarity index 83% rename from cassandra-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java rename to cassandra-3/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java index 8b9b3359..ee440fd9 100644 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java +++ b/cassandra-3/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java @@ -7,9 +7,9 @@ import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; @@ -21,7 +21,6 @@ import java.util.Properties; import java.util.concurrent.TimeUnit; -import org.apache.cassandra.config.DatabaseDescriptor; import org.junit.Assert; import org.junit.Before; import org.slf4j.Logger; @@ -29,30 +28,35 @@ import io.debezium.config.Configuration; import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.cassandra.utils.TestUtils; public class CommitLogRealTimeParserTest extends AbstractCommitLogProcessorTest { private static final Logger LOGGER = LoggerFactory.getLogger(CommitLogRealTimeParserTest.class); private CommitLogIdxProcessor commitLogProcessor; + @Override + public void assumeTestRuns() { + assumeNotCassandra3(); + } + @Before - public void setUp() throws Exception { + public void setUp() throws Throwable { super.setUp(); commitLogProcessor = new CommitLogIdxProcessor(context, metrics, - new Cassandra5CommitLogSegmentReader(context, metrics), - new File(DatabaseDescriptor.getCDCLogLocation())); - readLogs(); + commitLogProcessing.getCommitLogSegmentReader(), + new File(context.getCassandraConnectorConfig().getCDCLogLocation())); } @Override - public CassandraConnectorContext generateTaskContext() throws Exception { + public Configuration getContextConfiguration() throws Throwable { Properties properties = TestUtils.generateDefaultConfigMap(); properties.put(CassandraConnectorConfig.COMMIT_LOG_REAL_TIME_PROCESSING_ENABLED.name(), "true"); properties.put(CassandraConnectorConfig.COMMIT_LOG_MARKED_COMPLETE_POLL_INTERVAL_IN_MS.name(), "1000"); - return generateTaskContext(Configuration.from(properties)); + return Configuration.from(properties); } @Override - public void initialiseData() throws Exception { + public void initialiseData() { createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); insertRows(3, 10); } @@ -69,6 +73,7 @@ private void insertRows(int count, int keyInc) { @Override public void verifyEvents() { + readLogs(); verify(3, 10); insertRows(2, 20); verify(2, 20); @@ -96,7 +101,7 @@ private void readLogs() { // check to make sure there are no records in the queue to begin with ChangeEventQueue queue = context.getQueues().get(0); assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - String cdcLoc = DatabaseDescriptor.getCDCLogLocation(); + String cdcLoc = context.getCassandraConnectorConfig().getCDCLogLocation(); LOGGER.info("CDC Location: {}", cdcLoc); await().timeout(Duration.ofSeconds(3)).until(() -> CommitLogUtil.getIndexes(new File(cdcLoc)).length >= 1); File[] commitLogIndexes = CommitLogUtil.getIndexes(new File(cdcLoc)); diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra3ConnectorTestBase.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra3ConnectorTestBase.java deleted file mode 100644 index f762881a..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra3ConnectorTestBase.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra3SchemaChangeListenerProvider; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra3SchemaLoader; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public class EmbeddedCassandra3ConnectorTestBase extends CassandraConnectorTestBase { - - @Override - protected CassandraConnectorContext generateTaskContext(Configuration configuration) throws Exception { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra3TypeProvider provider = new Cassandra3TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(config, - new Cassandra3SchemaLoader(), - new Cassandra3SchemaChangeListenerProvider(), - new FileOffsetWriter(config)); - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java deleted file mode 100644 index 85b98e66..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.createTestKeyspace; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class MultipleTablesProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - - createTestKeyspace(TEST_KEYSPACE_NAME_2); - - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - final List events = getEvents(8); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record insert3 = (Record) events.get(2); - assertEquals(insert3.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert3.getOp()); - - Record insert4 = (Record) events.get(3); - assertEquals(insert4.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert4.getOp()); - - Record delete = (Record) events.get(4); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - - Record delete2 = (Record) events.get(5); - assertEquals(delete2.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete2.getOp()); - - Record delete3 = (Record) events.get(6); - assertEquals(delete3.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete3.getOp()); - - Record delete4 = (Record) events.get(7); - assertEquals(delete4.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete4.getOp()); - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java deleted file mode 100644 index 6c6f66a8..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public class QueueProcessorTest extends AbstractQueueProcessorTest { - @Override - public CassandraConnectorContext generateTaskContext(Configuration configuration) { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra3TypeProvider provider = new Cassandra3TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(new CassandraConnectorConfig(configuration)); - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/RowInsertionCommitLogProcessingTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/RowInsertionCommitLogProcessingTest.java deleted file mode 100644 index 88b3fb06..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/RowInsertionCommitLogProcessingTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -import org.apache.cassandra.config.DatabaseDescriptor; - -public class RowInsertionCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - for (int i = 0; i < 10; i++) { - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(i)) - .value("b", literal(i)) - .build()); - } - } - - @Override - public void verifyEvents() throws Exception { - for (Event event : getEvents(10)) { - if (event instanceof Record) { - Record record = (Record) event; - assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertFalse(record.getSource().snapshot); - assertEquals(record.getSource().keyspaceTable.name(), keyspaceTable(TEST_TABLE_NAME)); - } - else if (event instanceof EOFEvent) { - EOFEvent eofEvent = (EOFEvent) event; - assertFalse(context.getErroneousCommitLogs().contains(eofEvent.file.getName())); - } - else { - throw new Exception("unexpected event type"); - } - } - } -} diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java b/cassandra-3/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java deleted file mode 100644 index 9b48f9e6..00000000 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static io.debezium.connector.cassandra.TestUtils.TEST_KAFKA_SERVERS; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; - -import io.debezium.connector.base.ChangeEventQueue; - -public class SnapshotProcessorTest extends EmbeddedCassandra3ConnectorTestBase { - @Test - public void testSnapshotTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table2") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table2") + "(a, b) VALUES (?, ?)", i + 10, String.valueOf(i + 10)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(2 * tableSize, queue.totalCapacity() - queue.remainingCapacity()); - final List table1 = new ArrayList<>(); - final List table2 = new ArrayList<>(); - for (Event event : queue.poll()) { - ChangeRecord record = (ChangeRecord) event; - Assert.assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - Assert.assertEquals(record.getOp(), Record.Operation.INSERT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertTrue(record.getSource().snapshot); - final String tableName = record.getSource().keyspaceTable.name(); - if (tableName.equals(keyspaceTable("cdc_table"))) { - table1.add(record); - } - else { - table2.add(record); - } - Assert.assertEquals(record.getSource().offsetPosition, OffsetPosition.defaultOffsetPosition()); - } - assertEquals(tableSize, table1.size()); - assertEquals(tableSize, table2.size()); - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotSkipsNonCdcEnabledTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("non_cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = false;"); - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("non_cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotEmptyTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - AtomicBoolean globalTaskState = new AtomicBoolean(true); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); // records empty table to snapshot.offset, so it won't be snapshotted again - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - int tableSize = 5; - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); // newly inserted records should be processed by commit log processor instead - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - globalTaskState.set(false); - context.cleanUp(); - } - - @Test - public void testSnapshotModeAlways() throws Exception { - Map configs = TestUtils.propertiesForContext(); - configs.put(CassandraConnectorConfig.KAFKA_PRODUCER_CONFIG_PREFIX + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TEST_KAFKA_SERVERS); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "always"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(5)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeInitial() throws Exception { - Map configs = TestUtils.propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "initial"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(1)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeNever() throws Exception { - Map configs = TestUtils.propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "never"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra3TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, never()).snapshot(); - - context.cleanUp(); - } -} diff --git a/cassandra-3/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider b/cassandra-3/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider new file mode 100644 index 00000000..ef0a15c2 --- /dev/null +++ b/cassandra-3/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider @@ -0,0 +1 @@ +io.debezium.connector.cassandra.Cassandra3TestProvider \ No newline at end of file diff --git a/cassandra-3/src/test/resources/docker/cassandra.yaml b/cassandra-3/src/test/resources/docker/cassandra.yaml index 993602fe..ad522e76 100644 --- a/cassandra-3/src/test/resources/docker/cassandra.yaml +++ b/cassandra-3/src/test/resources/docker/cassandra.yaml @@ -193,7 +193,7 @@ data_file_directories: # commit log. when running on magnetic HDD, this should be a # separate spindle than the data directories. # If not set, the default directory is $CASSANDRA_HOME/data/commitlog. -commitlog_directory: /var/lib/cassandra/commitlog +commitlog_directory: /var/lib/cassandra/cdc_raw # Enable / disable CDC functionality on a per-node basis. This modifies the logic used # for write path allocation rejection (standard: never reject. cdc: reject Mutation @@ -204,7 +204,7 @@ cdc_enabled: true # segment contains mutations for a CDC-enabled table. This should be placed on a # separate spindle than the data directories. If not set, the default directory is # $CASSANDRA_HOME/data/cdc_raw. -cdc_raw_directory: /var/lib/cassandra/cdc_raw_directory +cdc_raw_directory: /var/lib/cassandra/cdc_raw # Policy for data disk failures: # diff --git a/cassandra-4/pom.xml b/cassandra-4/pom.xml index fab8a78c..c2039366 100644 --- a/cassandra-4/pom.xml +++ b/cassandra-4/pom.xml @@ -21,10 +21,9 @@ io.debezium - debezium-connector-cassandra-core + debezium-connector-cassandra-tests ${project.version} tests - test-jar test @@ -86,6 +85,9 @@ ${version.cassandra4} ${project.basedir}/src/test/resources/docker + + io.debezium:debezium-connector-cassandra-tests + diff --git a/cassandra-4/src/main/java/io/debezium/connector/cassandra/Cassandra4TypeProvider.java b/cassandra-4/src/main/java/io/debezium/connector/cassandra/Cassandra4TypeProvider.java index 284c2d30..b29b0af0 100644 --- a/cassandra-4/src/main/java/io/debezium/connector/cassandra/Cassandra4TypeProvider.java +++ b/cassandra-4/src/main/java/io/debezium/connector/cassandra/Cassandra4TypeProvider.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.function.Function; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BooleanType; @@ -110,10 +109,4 @@ public Object deserialize(Object abstractType, ByteBuffer bb) { public Function baseTypeForReversedType() { return abstractType -> ((AbstractType) abstractType).isReversed() ? ((ReversedType) abstractType).baseType : abstractType; } - - @Override - public String getClusterName() { - return DatabaseDescriptor.getClusterName(); - } - } diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java deleted file mode 100644 index ae317719..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java +++ /dev/null @@ -1,115 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static java.lang.String.format; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.commitlog.CommitLogReadHandler; -import org.apache.cassandra.db.commitlog.CommitLogReader; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.util.Testing; - -public abstract class AbstractCommitLogProcessorTest extends EmbeddedCassandra4ConnectorTestBase { - public CassandraConnectorContext context; - protected CommitLogProcessorMetrics metrics = new CommitLogProcessorMetrics(); - private CommitLogReadHandler commitLogReadHandler; - - @Before - public void setUp() throws Exception { - initialiseData(); - context = generateTaskContext(); - await().atMost(Duration.ofSeconds(60)).until(() -> context.getSchemaHolder() - .getKeyValueSchema(new KeyspaceTable(TEST_KEYSPACE_NAME, TEST_TABLE_NAME)) != null); - commitLogReadHandler = new Cassandra4CommitLogReadHandlerImpl(context, metrics); - metrics.registerMetrics(); - } - - @After - public void tearDown() throws Exception { - deleteTestOffsets(context); - metrics.unregisterMetrics(); - deleteTestKeyspaceTables(); - context.cleanUp(); - Testing.Files.delete(DatabaseDescriptor.getCDCLogLocation()); - } - - @Test - public void test() throws Exception { - verifyEvents(); - } - - public abstract void initialiseData() throws Exception; - - public abstract void verifyEvents() throws Exception; - - public void createTable(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - } - - public void createTable2(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - } - - public void createTable(String query, String keyspace, String tableName) throws Exception { - runCql(format(query, keyspace, tableName)); - } - - public List getEvents(final int expectedSize) throws Exception { - ChangeEventQueue queue = context.getQueues().get(0); - final List events = new ArrayList<>(); - await().atMost(60, TimeUnit.SECONDS).until(() -> { - readLogs(queue); - events.clear(); - events.addAll(queue.poll()); - return events.size() == expectedSize; - }); - assertEquals(expectedSize, events.size()); - return events; - } - - private void readLogs(ChangeEventQueue queue) throws Exception { - // check to make sure there are no records in the queue to begin with - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - // process the logs in commit log directory - File cdcLoc = new File(DatabaseDescriptor.getCommitLogLocation()); - File[] commitLogs = CommitLogUtil.getCommitLogs(cdcLoc); - - CommitLogReader reader = new CommitLogReader(); - - for (File commitLog : commitLogs) { - reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); - } - } - - public void assertEventTypes(List events, Event.EventType eventType, Record.Operation... operations) { - assertEquals(events.size(), operations.length); - for (int i = 0; i < events.size(); i++) { - Record record = (Record) events.get(i); - assertEquals(record.getEventType(), eventType); - assertEquals(operations[i], record.getOp()); - } - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/Cassandra4TestProvider.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/Cassandra4TestProvider.java new file mode 100644 index 00000000..1530ed26 --- /dev/null +++ b/cassandra-4/src/test/java/io/debezium/connector/cassandra/Cassandra4TestProvider.java @@ -0,0 +1,79 @@ +/* + * 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.cassandra; + +import java.io.File; +import java.io.IOException; + +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.commitlog.CommitLogReader; + +import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.spi.CommitLogProcessing; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; + +public class Cassandra4TestProvider implements CassandraTestProvider { + @Override + public CassandraConnectorContext provideContext(Configuration configuration) throws Exception { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra4TypeProvider provider = new Cassandra4TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(config, + new CassandraConnectorTask.Cassandra4SchemaLoader(), + new CassandraConnectorTask.Cassandra4SchemaChangeListenerProvider(), + new FileOffsetWriter(config)); + } + + @Override + public CassandraConnectorContext provideContextWithoutSchemaManagement(Configuration configuration) { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra4TypeProvider provider = new Cassandra4TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(new CassandraConnectorConfig(configuration)); + } + + @Override + public CommitLogProcessing provideCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + return new Cassandra4CommitLogProcessing(context, metrics); + } + + private static class Cassandra4CommitLogProcessing implements CommitLogProcessing { + + private final CommitLogReadHandler commitLogReadHandler; + private final CommitLogSegmentReader commitLogSegmentReader; + private final CassandraConnectorContext context; + + Cassandra4CommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + commitLogReadHandler = new Cassandra4CommitLogReadHandlerImpl(context, metrics); + commitLogSegmentReader = new Cassandra4CommitLogSegmentReader(context, metrics); + this.context = context; + } + + @Override + public void readAllCommitLogs(File[] commitLogs) throws IOException { + CommitLogReader reader = new CommitLogReader(); + File cdcLoc = new File(context.getCassandraConnectorConfig().getCDCLogLocation()); + for (File commitLog : CommitLogUtil.getCommitLogs(cdcLoc)) { + reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); + } + } + + @Override + public void readCommitLogSegment(File file, long segmentId, int position) throws IOException { + commitLogSegmentReader.readCommitLogSegment(file, segmentId, position); + } + + @Override + public CommitLogSegmentReader getCommitLogSegmentReader() { + return commitLogSegmentReader; + } + } +} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java deleted file mode 100644 index d720f720..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class ClusteringPartitionKeyCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record delete = (Record) events.get(2); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java deleted file mode 100644 index 8974a865..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java +++ /dev/null @@ -1,109 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.junit.Assert; -import org.junit.Before; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.debezium.config.Configuration; -import io.debezium.connector.base.ChangeEventQueue; - -public class CommitLogRealTimeParserTest extends AbstractCommitLogProcessorTest { - private static final Logger LOGGER = LoggerFactory.getLogger(CommitLogRealTimeParserTest.class); - private CommitLogIdxProcessor commitLogProcessor; - - @Before - public void setUp() throws Exception { - super.setUp(); - commitLogProcessor = new CommitLogIdxProcessor(context, metrics, - new Cassandra4CommitLogSegmentReader(context, metrics), - new File(DatabaseDescriptor.getCDCLogLocation())); - readLogs(); - } - - @Override - public CassandraConnectorContext generateTaskContext() throws Exception { - Properties properties = TestUtils.generateDefaultConfigMap(); - properties.put(CassandraConnectorConfig.COMMIT_LOG_REAL_TIME_PROCESSING_ENABLED.name(), "true"); - properties.put(CassandraConnectorConfig.COMMIT_LOG_MARKED_COMPLETE_POLL_INTERVAL_IN_MS.name(), "1000"); - return generateTaskContext(Configuration.from(properties)); - } - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - insertRows(3, 10); - } - - private void insertRows(int count, int keyInc) { - for (int i = 0; i < count; i++) { - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(i + keyInc)) - .value("b", literal(i)) - .build()); - } - LOGGER.info("Inserted rows: {}", count); - } - - @Override - public void verifyEvents() { - verify(3, 10); - insertRows(2, 20); - verify(2, 20); - } - - private void verify(int expectedEventsCount, int keyInc) { - List events = new ArrayList<>(); - await().atMost(5, TimeUnit.SECONDS).until(() -> { - events.addAll(context.getQueues().get(0).poll()); - return events.size() == expectedEventsCount; - }); - - LOGGER.info("Total events received: {}", events.size()); - Assert.assertEquals("Total number of events received must be " + expectedEventsCount, expectedEventsCount, events.size()); - - for (int i = 0; i < expectedEventsCount; i++) { - Record record = (Record) events.get(i); - Record.Operation op = record.getOp(); - Assert.assertEquals("Operation type must be insert but it was " + op, Record.Operation.INSERT, op); - Assert.assertEquals("Inserted key should be " + i + keyInc, record.getRowData().getPrimary().get(0).value, i + keyInc); - } - } - - private void readLogs() { - // check to make sure there are no records in the queue to begin with - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - String cdcLoc = DatabaseDescriptor.getCDCLogLocation(); - LOGGER.info("CDC Location: {}", cdcLoc); - await().timeout(Duration.ofSeconds(3)).until(() -> CommitLogUtil.getIndexes(new File(cdcLoc)).length >= 1); - File[] commitLogIndexes = CommitLogUtil.getIndexes(new File(cdcLoc)); - Arrays.sort(commitLogIndexes, (file1, file2) -> CommitLogUtil.compareCommitLogsIndexes(file1, file2)); - Assert.assertTrue("At least one idx file must be generated", commitLogIndexes.length >= 1); - // Submitting the last idx file as that one is generated by current test - commitLogProcessor.submit(commitLogIndexes[commitLogIndexes.length - 1].toPath()); - } - -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra4ConnectorTestBase.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra4ConnectorTestBase.java deleted file mode 100644 index 4a418919..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra4ConnectorTestBase.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra4SchemaChangeListenerProvider; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra4SchemaLoader; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public abstract class EmbeddedCassandra4ConnectorTestBase extends CassandraConnectorTestBase { - - @Override - protected CassandraConnectorContext generateTaskContext(Configuration configuration) throws Exception { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra4TypeProvider provider = new Cassandra4TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(config, - new Cassandra4SchemaLoader(), - new Cassandra4SchemaChangeListenerProvider(), - new FileOffsetWriter(config)); - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java deleted file mode 100644 index 6ba7edfa..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class PartitionKeyRowDeletionCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(2); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record delete = (Record) events.get(1); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java deleted file mode 100644 index 57d2b8be..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public class QueueProcessorTest extends AbstractQueueProcessorTest { - @Override - public CassandraConnectorContext generateTaskContext(Configuration configuration) { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra4TypeProvider provider = new Cassandra4TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(new CassandraConnectorConfig(configuration)); - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java deleted file mode 100644 index 835dad49..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.Record.Operation.RANGE_TOMBSTONE; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import java.util.List; - -import org.apache.commons.lang3.tuple.Pair; - -import io.debezium.connector.cassandra.CassandraSchemaFactory.RangeData; - -public class RangeTombstoneCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, d int, e int, PRIMARY KEY (a,b,c,d)) WITH cdc = true;"); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 1, 1, 1); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .value("d", literal(1)) - .value("e", literal(1)) - .build()); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 2, 3, 2); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(2)) - .value("d", literal(3)) - .value("e", literal(2)) - .build()); - - // "DELETE FROM ks.tb WHERE a = 1 AND b = 1 AND c <= 2"; - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .whereColumn("a").isEqualTo(literal(1)) - .whereColumn("b").isEqualTo(literal(1)) - .whereColumn("c").isLessThanOrEqualTo(literal(2)) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - assertEquals(3, events.size()); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record range1 = (Record) events.get(2); - assertEquals(range1.getEventType(), CHANGE_EVENT); - assertEquals(RANGE_TOMBSTONE, range1.getOp()); - - RangeData startRange = range1.getRowData().getStartRange(); - RangeData endRange = range1.getRowData().getEndRange(); - - assertNotNull(startRange); - - assertEquals("INCL_START_BOUND", startRange.method); - assertEquals("INCL_END_BOUND", endRange.method); - - Pair bStartValue = startRange.values.get("b"); - assertNotNull(bStartValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bStartValue); - - Pair bEndValue = endRange.values.get("b"); - assertNotNull(bEndValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bEndValue); - - Pair cEndValue = endRange.values.get("c"); - assertNotNull(cEndValue); - assertEquals(Pair.of("2", "org.apache.cassandra.db.marshal.Int32Type"), cEndValue); - } -} diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java b/cassandra-4/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java deleted file mode 100644 index ad93e843..00000000 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -import org.apache.cassandra.config.DatabaseDescriptor; - -public class RowInsertionModificationCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - for (int i = 0; i < 10; i++) { - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(i)) - .value("b", literal(i)) - .build()); - } - } - - @Override - public void verifyEvents() throws Exception { - for (Event event : getEvents(10)) { - if (event instanceof Record) { - Record record = (Record) event; - assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertFalse(record.getSource().snapshot); - assertEquals(record.getSource().keyspaceTable.name(), keyspaceTable(TEST_TABLE_NAME)); - } - else if (event instanceof EOFEvent) { - EOFEvent eofEvent = (EOFEvent) event; - assertFalse(context.getErroneousCommitLogs().contains(eofEvent.file.getName())); - } - else { - throw new Exception("unexpected event type"); - } - } - } -} diff --git a/cassandra-4/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider b/cassandra-4/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider new file mode 100644 index 00000000..798d099b --- /dev/null +++ b/cassandra-4/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider @@ -0,0 +1 @@ +io.debezium.connector.cassandra.Cassandra4TestProvider \ No newline at end of file diff --git a/cassandra-5/pom.xml b/cassandra-5/pom.xml index 66cb3825..fdf41abb 100644 --- a/cassandra-5/pom.xml +++ b/cassandra-5/pom.xml @@ -21,10 +21,9 @@ io.debezium - debezium-connector-cassandra-core + debezium-connector-cassandra-tests ${project.version} tests - test-jar test @@ -90,6 +89,9 @@ ${version.cassandra5} ${project.basedir}/src/test/resources/docker + + io.debezium:debezium-connector-cassandra-tests + diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java index 9e144259..d5ed00f0 100644 --- a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.function.Function; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BooleanType; @@ -110,10 +109,4 @@ public Object deserialize(Object abstractType, ByteBuffer bb) { public Function baseTypeForReversedType() { return abstractType -> ((AbstractType) abstractType).isReversed() ? ((ReversedType) abstractType).baseType : abstractType; } - - @Override - public String getClusterName() { - return DatabaseDescriptor.getClusterName(); - } - } diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java deleted file mode 100644 index 7a09259e..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.core.cql.BatchType.LOGGED; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static com.datastax.oss.driver.api.querybuilder.relation.Relation.column; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; - -import java.util.HashMap; - -import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder; - -public class BatchTest extends AbstractCommitLogProcessorTest { - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE %s.%s (\n" + - " p1 text,\n" + - " p2 text,\n" + - " p3 text,\n" + - " c1 text,\n" + - " col1 text,\n" + - " col2 text,\n" + - " amap map,\n" + - " PRIMARY KEY ((p1, p2, p3), c1)\n" + - ") WITH CLUSTERING ORDER BY (c1 ASC)\n" + - " AND cdc = true;"); - - runCql(new BatchStatementBuilder(LOGGED) - .addStatement(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .usingTimestamp(1683810323861L) - .where(column("p1").isEqualTo(literal("abc"))) - .where(column("p2").isEqualTo(literal("p2value"))) - .where(column("p3").isEqualTo(literal("p3value"))) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .value("col1", literal("")) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value1")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value2")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - assertEventTypes(getEvents(4), CHANGE_EVENT, DELETE, INSERT, INSERT, INSERT); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/Cassandra5TestProvider.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/Cassandra5TestProvider.java new file mode 100644 index 00000000..a9d42dd0 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/Cassandra5TestProvider.java @@ -0,0 +1,79 @@ +/* + * 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.cassandra; + +import java.io.File; +import java.io.IOException; + +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.commitlog.CommitLogReader; + +import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.spi.CommitLogProcessing; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; + +public class Cassandra5TestProvider implements CassandraTestProvider { + @Override + public CassandraConnectorContext provideContext(Configuration configuration) throws Exception { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra5TypeProvider provider = new Cassandra5TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(config, + new CassandraConnectorTask.Cassandra5SchemaLoader(), + new CassandraConnectorTask.Cassandra5SchemaChangeListenerProvider(), + new FileOffsetWriter(config)); + } + + @Override + public CassandraConnectorContext provideContextWithoutSchemaManagement(Configuration configuration) { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + Cassandra5TypeProvider provider = new Cassandra5TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(new CassandraConnectorConfig(configuration)); + } + + @Override + public CommitLogProcessing provideCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + return new Cassandra5CommitLogProcessing(context, metrics); + } + + private static class Cassandra5CommitLogProcessing implements CommitLogProcessing { + + private final CommitLogReadHandler commitLogReadHandler; + private final CommitLogSegmentReader commitLogSegmentReader; + private final CassandraConnectorContext context; + + Cassandra5CommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + commitLogReadHandler = new Cassandra5CommitLogReadHandlerImpl(context, metrics); + commitLogSegmentReader = new Cassandra5CommitLogSegmentReader(context, metrics); + this.context = context; + } + + @Override + public void readAllCommitLogs(File[] commitLogs) throws IOException { + CommitLogReader reader = new CommitLogReader(); + File cdcLoc = new File(context.getCassandraConnectorConfig().getCDCLogLocation()); + for (File commitLog : CommitLogUtil.getCommitLogs(cdcLoc)) { + reader.readCommitLogSegment(commitLogReadHandler, new org.apache.cassandra.io.util.File(commitLog), true); + } + } + + @Override + public void readCommitLogSegment(File file, long segmentId, int position) throws IOException { + commitLogSegmentReader.readCommitLogSegment(file, segmentId, position); + } + + @Override + public CommitLogSegmentReader getCommitLogSegmentReader() { + return commitLogSegmentReader; + } + } +} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java deleted file mode 100644 index ebf814aa..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java +++ /dev/null @@ -1,28 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra5SchemaChangeListenerProvider; -import io.debezium.connector.cassandra.CassandraConnectorTask.Cassandra5SchemaLoader; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public abstract class EmbeddedCassandra5ConnectorTestBase extends CassandraConnectorTestBase { - - @Override - protected CassandraConnectorContext generateTaskContext(Configuration configuration) throws Exception { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra5TypeProvider provider = new Cassandra5TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(config, - new Cassandra5SchemaLoader(), - new Cassandra5SchemaChangeListenerProvider(), - new FileOffsetWriter(config)); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java deleted file mode 100644 index bb24ec15..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java +++ /dev/null @@ -1,107 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.createTestKeyspace; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class MultipleTablesProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - - createTestKeyspace(TEST_KEYSPACE_NAME_2); - - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(8); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record insert3 = (Record) events.get(2); - assertEquals(insert3.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert3.getOp()); - - Record insert4 = (Record) events.get(3); - assertEquals(insert4.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert4.getOp()); - - Record delete = (Record) events.get(4); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - - Record delete2 = (Record) events.get(5); - assertEquals(delete2.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete2.getOp()); - - Record delete3 = (Record) events.get(6); - assertEquals(delete3.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete3.getOp()); - - Record delete4 = (Record) events.get(7); - assertEquals(delete4.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete4.getOp()); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java deleted file mode 100644 index 6ba7edfa..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java +++ /dev/null @@ -1,47 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -public class PartitionKeyRowDeletionCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(2); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record delete = (Record) events.get(1); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java deleted file mode 100644 index c20c8ecb..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java +++ /dev/null @@ -1,22 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public class QueueProcessorTest extends AbstractQueueProcessorTest { - @Override - public CassandraConnectorContext generateTaskContext(Configuration configuration) { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - Cassandra5TypeProvider provider = new Cassandra5TypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(new CassandraConnectorConfig(configuration)); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java deleted file mode 100644 index 835dad49..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java +++ /dev/null @@ -1,96 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.Record.Operation.RANGE_TOMBSTONE; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import java.util.List; - -import org.apache.commons.lang3.tuple.Pair; - -import io.debezium.connector.cassandra.CassandraSchemaFactory.RangeData; - -public class RangeTombstoneCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, d int, e int, PRIMARY KEY (a,b,c,d)) WITH cdc = true;"); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 1, 1, 1); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .value("d", literal(1)) - .value("e", literal(1)) - .build()); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 2, 3, 2); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(2)) - .value("d", literal(3)) - .value("e", literal(2)) - .build()); - - // "DELETE FROM ks.tb WHERE a = 1 AND b = 1 AND c <= 2"; - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .whereColumn("a").isEqualTo(literal(1)) - .whereColumn("b").isEqualTo(literal(1)) - .whereColumn("c").isLessThanOrEqualTo(literal(2)) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - assertEquals(3, events.size()); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record range1 = (Record) events.get(2); - assertEquals(range1.getEventType(), CHANGE_EVENT); - assertEquals(RANGE_TOMBSTONE, range1.getOp()); - - RangeData startRange = range1.getRowData().getStartRange(); - RangeData endRange = range1.getRowData().getEndRange(); - - assertNotNull(startRange); - - assertEquals("INCL_START_BOUND", startRange.method); - assertEquals("INCL_END_BOUND", endRange.method); - - Pair bStartValue = startRange.values.get("b"); - assertNotNull(bStartValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bStartValue); - - Pair bEndValue = endRange.values.get("b"); - assertNotNull(bEndValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bEndValue); - - Pair cEndValue = endRange.values.get("c"); - assertNotNull(cEndValue); - assertEquals(Pair.of("2", "org.apache.cassandra.db.marshal.Int32Type"), cEndValue); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java deleted file mode 100644 index 88f730a7..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static java.lang.String.format; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import java.util.List; - -public class SchemaChangeListenerTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE %s.%s (a int, b int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(2)) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - // We have to read the first event before altering the table. - // That way we make sure that the initial schema is registered and the schema change code path is triggered. - List events = getEvents(1); - Record insert1 = (Record) events.get(0); - assertEquals(CHANGE_EVENT, insert1.getEventType()); - assertEquals(INSERT, insert1.getOp()); - assertTrue(insert1.getRowData().hasCell("a")); - assertTrue(insert1.getRowData().hasCell("b")); - assertFalse(insert1.getRowData().hasCell("c")); - - runCql(format("ALTER TABLE %s.%s ADD c int;", TEST_KEYSPACE_NAME, TEST_TABLE_NAME)); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(3)) - .value("b", literal(4)) - .value("c", literal(5)) - .build()); - - events = getEvents(2); - Record insert2 = (Record) events.get(1); - - assertEquals(CHANGE_EVENT, insert2.getEventType()); - assertEquals(INSERT, insert2.getOp()); - assertTrue(insert2.getRowData().hasCell("a")); - assertTrue(insert2.getRowData().hasCell("b")); - assertTrue(insert2.getRowData().hasCell("c")); - } -} diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java deleted file mode 100644 index c1d1d85c..00000000 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.cassandra; - -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.propertiesForContext; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; - -import io.debezium.connector.base.ChangeEventQueue; - -public class SnapshotProcessorTest extends EmbeddedCassandra5ConnectorTestBase { - @Test - public void testSnapshotTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table2") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table2") + "(a, b) VALUES (?, ?)", i + 10, String.valueOf(i + 10)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(2 * tableSize, queue.totalCapacity() - queue.remainingCapacity()); - final List table1 = new ArrayList<>(); - final List table2 = new ArrayList<>(); - for (Event event : queue.poll()) { - ChangeRecord record = (ChangeRecord) event; - Assert.assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - Assert.assertEquals(record.getOp(), Record.Operation.INSERT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertTrue(record.getSource().snapshot); - final String tableName = record.getSource().keyspaceTable.name(); - if (tableName.equals(keyspaceTable("cdc_table"))) { - table1.add(record); - } - else { - table2.add(record); - } - Assert.assertEquals(record.getSource().offsetPosition, OffsetPosition.defaultOffsetPosition()); - } - assertEquals(tableSize, table1.size()); - assertEquals(tableSize, table2.size()); - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotSkipsNonCdcEnabledTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("non_cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = false;"); - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("non_cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotEmptyTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - AtomicBoolean globalTaskState = new AtomicBoolean(true); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); // records empty table to snapshot.offset, so it won't be snapshotted again - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - int tableSize = 5; - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); // newly inserted records should be processed by commit log processor instead - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - globalTaskState.set(false); - context.cleanUp(); - } - - @Test - public void testSnapshotModeAlways() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.KAFKA_PRODUCER_CONFIG_PREFIX + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.TEST_KAFKA_SERVERS); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "always"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(5)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeInitial() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "initial"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(1)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeNever() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "never"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra5TypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, never()).snapshot(); - - context.cleanUp(); - } -} diff --git a/cassandra-5/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider b/cassandra-5/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider new file mode 100644 index 00000000..dda6037e --- /dev/null +++ b/cassandra-5/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider @@ -0,0 +1 @@ +io.debezium.connector.cassandra.Cassandra5TestProvider \ No newline at end of file diff --git a/core/pom.xml b/core/pom.xml index 200964da..c4490496 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -35,17 +35,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - - - --add-exports=java.base/sun.nio.ch=ALL-UNNAMED - --add-opens=java.base/sun.nio.ch=ALL-UNNAMED - --add-opens=java.base/java.io=ALL-UNNAMED - - - diff --git a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorConfig.java b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorConfig.java index 77bb3fdb..de13318f 100644 --- a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorConfig.java +++ b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorConfig.java @@ -347,11 +347,22 @@ public static EventOrderGuaranteeMode parse(String value) { .withType(Type.STRING) .withDescription("The absolute path of the YAML config file used by a Cassandra node."); + public static final Field CASSANDRA_CLUSTER_NAME = Field.create("cassandra.cluster.name") + .withType(Type.STRING) + .withDescription("Name of Cassandra cluster.") + .withDefault("Test Cluster"); + public static final Field COMMIT_LOG_RELOCATION_DIR = Field.create("commit.log.relocation.dir") .withType(Type.STRING) .withValidation(Field::isRequired) .withDescription("The local directory which commit logs get relocated to once processed."); + public static final Field CASSANDRA_CDC_LOG_LOCATION = Field.create("commit.log.location") + .withType(Type.STRING) + .withValidation(Field::isRequired) + .withDescription("Cassandra directory where cdc logs are located. Corresponds to cdc_raw_directory setting in Cassandra") + .withDefault("/var/lib/cassandra/cdc_raw"); + /** * If disabled, commit logs would not be deleted post-process, and this could lead to disk storage */ @@ -599,6 +610,14 @@ public String cassandraConfig() { return this.getConfig().getString(CASSANDRA_CONFIG); } + public String clusterName() { + return this.getConfig().getString(CASSANDRA_CLUSTER_NAME); + } + + public String getCDCLogLocation() { + return this.getConfig().getString(CASSANDRA_CDC_LOG_LOCATION); + } + public String commitLogRelocationDir() { return this.getConfig().getString(COMMIT_LOG_RELOCATION_DIR); } diff --git a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorContext.java b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorContext.java index 54923c0b..34e3fae2 100644 --- a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorContext.java +++ b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorContext.java @@ -5,107 +5,33 @@ */ package io.debezium.connector.cassandra; -import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.connector.cassandra.exceptions.CassandraConnectorTaskException; -import io.debezium.connector.common.CdcSourceTaskContext; +import io.debezium.util.LoggingContext; /** * Contains contextual information and objects scoped to the lifecycle * of CassandraConnectorTask implementation. */ -public class CassandraConnectorContext extends CdcSourceTaskContext { - private final CassandraConnectorConfig config; - private CassandraClient cassandraClient; - private final List> queues = new ArrayList<>(); - private SchemaHolder schemaHolder; - private OffsetWriter offsetWriter; - // Create a HashSet to record names of CommitLog Files which are not successfully read or streamed. - private final Set erroneousCommitLogs = ConcurrentHashMap.newKeySet(); +public interface CassandraConnectorContext { - public CassandraConnectorContext(CassandraConnectorConfig config) { - super(config, config.getCustomMetricTags(), Collections::emptySet); - this.config = config; - prepareQueues(); - } + void cleanUp(); - public CassandraConnectorContext(CassandraConnectorConfig config, - SchemaLoader schemaLoader, - SchemaChangeListenerProvider schemaChangeListenerProvider, - OffsetWriter offsetWriter) { - super(config, config.getCustomMetricTags(), Collections::emptySet); - this.config = config; - this.offsetWriter = offsetWriter; + CassandraConnectorConfig getCassandraConnectorConfig(); - try { - prepareQueues(); + LoggingContext.PreviousContext configureLoggingContext(String contextName); - // Loading up DDL schemas from disk - schemaLoader.load(this.config.cassandraConfig()); + CassandraClient getCassandraClient(); - AbstractSchemaChangeListener schemaChangeListener = schemaChangeListenerProvider.provide(this.config); + List> getQueues(); - // Setting up Cassandra driver - this.cassandraClient = new CassandraClient(config.cassandraDriverConfig(), schemaChangeListener); + OffsetWriter getOffsetWriter(); - // Setting up schema holder ... - this.schemaHolder = schemaChangeListener.getSchemaHolder(); - } - catch (Exception e) { - // Clean up CassandraClient and FileOffsetWrite if connector context fails to be completely initialized. - cleanUp(); - throw new CassandraConnectorTaskException("Failed to initialize Cassandra Connector Context.", e); - } - } + SchemaHolder getSchemaHolder(); - private void prepareQueues() { - int numOfChangeEventQueues = this.config.numOfChangeEventQueues(); - for (int i = 0; i < numOfChangeEventQueues; i++) { - ChangeEventQueue queue = new ChangeEventQueue.Builder() - .pollInterval(this.config.pollInterval()) - .maxBatchSize(this.config.maxBatchSize()) - .maxQueueSize(this.config.maxQueueSize()) - .loggingContextSupplier(() -> this.configureLoggingContext(this.config.getContextName())) - .build(); - queues.add(queue); - } - } + Set getErroneousCommitLogs(); - public void cleanUp() { - if (this.cassandraClient != null) { - this.cassandraClient.close(); - } - if (this.offsetWriter != null) { - this.offsetWriter.close(); - } - } - - public CassandraConnectorConfig getCassandraConnectorConfig() { - return config; - } - - public CassandraClient getCassandraClient() { - return cassandraClient; - } - - public List> getQueues() { - return queues; - } - - public OffsetWriter getOffsetWriter() { - return offsetWriter; - } - - public SchemaHolder getSchemaHolder() { - return schemaHolder; - } - - public Set getErroneousCommitLogs() { - return erroneousCommitLogs; - } + String getClusterName(); } diff --git a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTaskTemplate.java b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTaskTemplate.java index 3a6b20cd..3cca512a 100644 --- a/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTaskTemplate.java +++ b/core/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTaskTemplate.java @@ -108,7 +108,7 @@ public void start() throws Exception { initDeserializer(); LOGGER.info("Initializing Cassandra connector task context ..."); - taskContext = new CassandraConnectorContext(config, schemaLoader, schemaChangeListenerProvider, factory.offsetWriter(config)); + taskContext = new DefaultCassandraConnectorContext(config, schemaLoader, schemaChangeListenerProvider, factory.offsetWriter(config)); LOGGER.info("Starting processor group ..."); AbstractProcessor[] processors = cassandraSpecificProcessors.getProcessors(taskContext); @@ -154,7 +154,7 @@ protected ProcessorGroup initProcessorGroup(CassandraConnectorContext taskContex processorGroup.addProcessor(processor); } - processorGroup.addProcessor(new SnapshotProcessor(taskContext, deserializerProvider.getClusterName())); + processorGroup.addProcessor(new SnapshotProcessor(taskContext, taskContext.getClusterName())); List> queues = taskContext.getQueues(); for (int i = 0; i < queues.size(); i++) { processorGroup.addProcessor(new QueueProcessor(taskContext, i, recordEmitter)); diff --git a/core/src/main/java/io/debezium/connector/cassandra/CassandraTypeProvider.java b/core/src/main/java/io/debezium/connector/cassandra/CassandraTypeProvider.java index 77be2a7b..3efa93cd 100644 --- a/core/src/main/java/io/debezium/connector/cassandra/CassandraTypeProvider.java +++ b/core/src/main/java/io/debezium/connector/cassandra/CassandraTypeProvider.java @@ -16,6 +16,4 @@ public interface CassandraTypeProvider { Function baseTypeForReversedType(); - String getClusterName(); - } diff --git a/core/src/main/java/io/debezium/connector/cassandra/DefaultCassandraConnectorContext.java b/core/src/main/java/io/debezium/connector/cassandra/DefaultCassandraConnectorContext.java new file mode 100644 index 00000000..0d6dac2d --- /dev/null +++ b/core/src/main/java/io/debezium/connector/cassandra/DefaultCassandraConnectorContext.java @@ -0,0 +1,122 @@ +/* + * 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.cassandra; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.cassandra.exceptions.CassandraConnectorTaskException; +import io.debezium.connector.common.CdcSourceTaskContext; + +public class DefaultCassandraConnectorContext extends CdcSourceTaskContext implements CassandraConnectorContext { + private final CassandraConnectorConfig config; + private CassandraClient cassandraClient; + private final List> queues = new ArrayList<>(); + private SchemaHolder schemaHolder; + private OffsetWriter offsetWriter; + // Create a HashSet to record names of CommitLog Files which are not successfully read or streamed. + private final Set erroneousCommitLogs = ConcurrentHashMap.newKeySet(); + + public DefaultCassandraConnectorContext(CassandraConnectorConfig config) { + super(config, config.getCustomMetricTags(), Collections::emptySet); + this.config = config; + prepareQueues(); + } + + public DefaultCassandraConnectorContext(CassandraConnectorConfig config, + SchemaLoader schemaLoader, + SchemaChangeListenerProvider schemaChangeListenerProvider, + OffsetWriter offsetWriter) { + super(config, config.getCustomMetricTags(), Collections::emptySet); + this.config = config; + this.offsetWriter = offsetWriter; + + try { + prepareQueues(); + + // Loading up DDL schemas from disk + schemaLoader.load(this.config.cassandraConfig()); + + AbstractSchemaChangeListener schemaChangeListener = schemaChangeListenerProvider.provide(this.config); + + // Setting up Cassandra driver + this.cassandraClient = new CassandraClient(config.cassandraDriverConfig(), schemaChangeListener); + + // Setting up schema holder ... + this.schemaHolder = schemaChangeListener.getSchemaHolder(); + } + catch (Exception e) { + // Clean up CassandraClient and FileOffsetWrite if connector context fails to be completely initialized. + cleanUp(); + throw new CassandraConnectorTaskException("Failed to initialize Cassandra Connector Context.", e); + } + } + + private void prepareQueues() { + int numOfChangeEventQueues = this.config.numOfChangeEventQueues(); + for (int i = 0; i < numOfChangeEventQueues; i++) { + ChangeEventQueue queue = new ChangeEventQueue.Builder() + .pollInterval(this.config.pollInterval()) + .maxBatchSize(this.config.maxBatchSize()) + .maxQueueSize(this.config.maxQueueSize()) + .loggingContextSupplier(() -> this.configureLoggingContext(this.config.getContextName())) + .build(); + queues.add(queue); + } + } + + public void cleanUp() { + queues.forEach(q -> { + while (q.remainingCapacity() != q.totalCapacity()) { + try { + q.poll(); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + }); + if (this.cassandraClient != null) { + this.cassandraClient.close(); + } + if (this.offsetWriter != null) { + this.offsetWriter.close(); + } + } + + public CassandraConnectorConfig getCassandraConnectorConfig() { + return config; + } + + public CassandraClient getCassandraClient() { + return cassandraClient; + } + + public List> getQueues() { + return queues; + } + + public OffsetWriter getOffsetWriter() { + return offsetWriter; + } + + public SchemaHolder getSchemaHolder() { + return schemaHolder; + } + + public Set getErroneousCommitLogs() { + return erroneousCommitLogs; + } + + @Override + public String getClusterName() { + return this.config.clusterName(); + } +} diff --git a/core/src/test/java/io/debezium/connector/cassandra/CommitLogPostProcessorTest.java b/core/src/test/java/io/debezium/connector/cassandra/CommitLogPostProcessorTest.java index 67767226..422ef7d2 100644 --- a/core/src/test/java/io/debezium/connector/cassandra/CommitLogPostProcessorTest.java +++ b/core/src/test/java/io/debezium/connector/cassandra/CommitLogPostProcessorTest.java @@ -5,9 +5,9 @@ */ package io.debezium.connector.cassandra; -import static io.debezium.connector.cassandra.TestUtils.clearCommitLogFromDirectory; -import static io.debezium.connector.cassandra.TestUtils.generateDefaultConfigMap; -import static io.debezium.connector.cassandra.TestUtils.populateFakeCommitLogsForDirectory; +import static io.debezium.connector.cassandra.utils.TestUtils.clearCommitLogFromDirectory; +import static io.debezium.connector.cassandra.utils.TestUtils.generateDefaultConfigMap; +import static io.debezium.connector.cassandra.utils.TestUtils.populateFakeCommitLogsForDirectory; import static org.junit.Assert.assertEquals; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; diff --git a/core/src/test/java/io/debezium/connector/cassandra/FileOffsetWriterTest.java b/core/src/test/java/io/debezium/connector/cassandra/FileOffsetWriterTest.java index 326ac64f..cbce972e 100644 --- a/core/src/test/java/io/debezium/connector/cassandra/FileOffsetWriterTest.java +++ b/core/src/test/java/io/debezium/connector/cassandra/FileOffsetWriterTest.java @@ -21,6 +21,7 @@ import io.debezium.config.Configuration; import io.debezium.connector.cassandra.exceptions.CassandraConnectorTaskException; +import io.debezium.connector.cassandra.utils.TestUtils; import io.debezium.time.Conversions; public class FileOffsetWriterTest { diff --git a/core/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java b/core/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java new file mode 100644 index 00000000..14bcd174 --- /dev/null +++ b/core/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.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.cassandra.utils; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.Properties; + +import org.apache.kafka.clients.producer.ProducerConfig; + +import io.debezium.connector.cassandra.CassandraConnectorConfig; +import io.debezium.connector.cassandra.CommitLogUtil; + +public class TestUtils { + public static final String TEST_CONNECTOR_NAME = "cassandra-01"; + public static final String TEST_KAFKA_TOPIC_PREFIX = "test_topic"; + public static final String TEST_KAFKA_SERVERS = "localhost:9092"; + + public static Properties generateDefaultConfigMap() throws IOException { + Properties props = new Properties(); + props.put(CassandraConnectorConfig.TOPIC_PREFIX.name(), TEST_CONNECTOR_NAME); + props.put(CassandraConnectorConfig.CASSANDRA_CONFIG.name(), Paths.get("src/test/resources/cassandra-unit-for-context.yaml").toAbsolutePath().toString()); + props.put(CassandraConnectorConfig.TOPIC_PREFIX.name(), TEST_KAFKA_TOPIC_PREFIX); + props.put(CassandraConnectorConfig.OFFSET_BACKING_STORE_DIR.name(), Files.createTempDirectory("offset").toString()); + props.put(CassandraConnectorConfig.KAFKA_PRODUCER_CONFIG_PREFIX + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TEST_KAFKA_SERVERS); + props.put(CassandraConnectorConfig.COMMIT_LOG_RELOCATION_DIR.name(), Files.createTempDirectory("cdc_raw_relocation").toString()); + props.put(CassandraConnectorConfig.KEY_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); + props.put(CassandraConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); + props.put(CassandraConnectorConfig.CASSANDRA_DRIVER_CONFIG_FILE.name(), Paths.get("src/test/resources/application.conf").toAbsolutePath().toString()); + // props.put(CassandraConnectorConfig.MAX_QUEUE_SIZE.name(), 1_000_000); + // props.put(CassandraConnectorConfig.MAX_QUEUE_SIZE_IN_BYTES.name(), 1_000_000_000); + return props; + } + + /** + * Delete all commit log files in directory + */ + public static void clearCommitLogFromDirectory(File directory, boolean recursive) throws IOException { + if (!directory.exists() || !directory.isDirectory()) { + throw new IOException(directory + " is not a valid directory"); + } + + File[] commitLogs = CommitLogUtil.getCommitLogs(directory); + for (File commitLog : commitLogs) { + CommitLogUtil.deleteCommitLog(commitLog); + } + + if (recursive) { + File[] directories = directory.listFiles(File::isDirectory); + if (directories != null) { + for (File dir : directories) { + clearCommitLogFromDirectory(dir, true); + } + } + } + } + + /** + * Generate commit log files in directory + */ + public static void populateFakeCommitLogsForDirectory(int numOfFiles, File directory) throws IOException { + if (directory.exists() && !directory.isDirectory()) { + throw new IOException(directory + " is not a directory"); + } + if (!directory.exists() && !directory.mkdir()) { + throw new IOException("Cannot create directory " + directory); + } + clearCommitLogFromDirectory(directory, true); + long prefix = System.currentTimeMillis(); + for (int i = 0; i < numOfFiles; i++) { + long ts = prefix + i; + Path path = Paths.get(directory.getAbsolutePath(), "CommitLog-6-" + ts + ".log"); + boolean success = path.toFile().createNewFile(); + if (!success) { + throw new IOException("Failed to create new commit log for testing"); + } + } + } +} diff --git a/dse/pom.xml b/dse/pom.xml index 76297b2a..20e86cb7 100644 --- a/dse/pom.xml +++ b/dse/pom.xml @@ -21,10 +21,9 @@ io.debezium - debezium-connector-cassandra-core + debezium-connector-cassandra-tests ${project.version} tests - test-jar test @@ -105,6 +104,11 @@ 1.1 test + + org.yaml + snakeyaml + 1.29 + @@ -146,6 +150,9 @@ ${project.basedir}/src/test/resources/docker false + + io.debezium:debezium-connector-cassandra-tests + diff --git a/dse/src/main/java/io/debezium/connector/dse/DseTypeProvider.java b/dse/src/main/java/io/debezium/connector/dse/DseTypeProvider.java index a909e6ad..4c12a6b2 100644 --- a/dse/src/main/java/io/debezium/connector/dse/DseTypeProvider.java +++ b/dse/src/main/java/io/debezium/connector/dse/DseTypeProvider.java @@ -22,7 +22,6 @@ import java.util.List; import java.util.function.Function; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.cassandra.db.marshal.AbstractType; import org.apache.cassandra.db.marshal.AsciiType; import org.apache.cassandra.db.marshal.BooleanType; @@ -111,10 +110,4 @@ public Object deserialize(Object abstractType, ByteBuffer bb) { public Function baseTypeForReversedType() { return abstractType -> ((AbstractType) abstractType).isReversed() ? ((ReversedType) abstractType).baseType : abstractType; } - - @Override - public String getClusterName() { - return DatabaseDescriptor.getClusterName(); - } - } diff --git a/dse/src/test/java/io/debezium/connector/dse/AbstractCommitLogProcessorTest.java b/dse/src/test/java/io/debezium/connector/dse/AbstractCommitLogProcessorTest.java deleted file mode 100644 index b0411fd7..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/AbstractCommitLogProcessorTest.java +++ /dev/null @@ -1,121 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static java.lang.String.format; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.commitlog.CommitLogReadHandler; -import org.apache.cassandra.db.commitlog.CommitLogReader; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.connector.cassandra.CassandraConnectorContext; -import io.debezium.connector.cassandra.CommitLogProcessorMetrics; -import io.debezium.connector.cassandra.CommitLogUtil; -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.KeyspaceTable; -import io.debezium.connector.cassandra.Record; -import io.debezium.util.Testing; - -public abstract class AbstractCommitLogProcessorTest extends DseConnectorTestBase { - public CassandraConnectorContext context; - protected CommitLogProcessorMetrics metrics = new CommitLogProcessorMetrics(); - private CommitLogReadHandler commitLogReadHandler; - - @Before - public void setUp() throws Exception { - initialiseData(); - context = generateTaskContext(); - await().atMost(Duration.ofSeconds(60)).until(() -> context.getSchemaHolder() - .getKeyValueSchema(new KeyspaceTable(TEST_KEYSPACE_NAME, TEST_TABLE_NAME)) != null); - commitLogReadHandler = new DseCommitLogReadHandlerImpl(context, metrics); - metrics.registerMetrics(); - } - - @After - public void tearDown() throws Exception { - deleteTestOffsets(context); - metrics.unregisterMetrics(); - deleteTestKeyspaceTables(); - context.cleanUp(); - Testing.Files.delete(DatabaseDescriptor.getCDCLogLocation()); - } - - @Test - public void test() throws Exception { - verifyEvents(); - } - - public abstract void initialiseData() throws Exception; - - public abstract void verifyEvents() throws Exception; - - public void createTable(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - } - - public void createTable2(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - } - - public void createTable(String query, String keyspace, String tableName) throws Exception { - runCql(format(query, keyspace, tableName)); - } - - public List getEvents(final int expectedSize) throws Exception { - ChangeEventQueue queue = context.getQueues().get(0); - final List events = new ArrayList<>(); - await().atMost(60, TimeUnit.SECONDS).until(() -> { - readLogs(queue); - events.clear(); - events.addAll(queue.poll()); - return events.size() == expectedSize; - }); - assertEquals(expectedSize, events.size()); - return events; - } - - private void readLogs(ChangeEventQueue queue) throws Exception { - // check to make sure there are no records in the queue to begin with - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - // process the logs in commit log directory - File cdcLoc = DatabaseDescriptor.getCommitLogLocation(); - File[] commitLogs = CommitLogUtil.getCommitLogs(cdcLoc); - - CommitLogReader reader = new CommitLogReader(); - - for (File commitLog : commitLogs) { - reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); - } - } - - public void assertEventTypes(List events, Event.EventType eventType, Record.Operation... operations) { - assertEquals(events.size(), operations.length); - for (int i = 0; i < events.size(); i++) { - Record record = (Record) events.get(i); - assertEquals(record.getEventType(), eventType); - assertEquals(operations[i], record.getOp()); - } - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/BatchTest.java b/dse/src/test/java/io/debezium/connector/dse/BatchTest.java deleted file mode 100644 index 303a6eac..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/BatchTest.java +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.core.cql.BatchType.LOGGED; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static com.datastax.oss.driver.api.querybuilder.relation.Relation.column; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; - -import java.util.HashMap; - -import com.datastax.oss.driver.api.core.cql.BatchStatementBuilder; - -public class BatchTest extends AbstractCommitLogProcessorTest { - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE %s.%s (\n" + - " p1 text,\n" + - " p2 text,\n" + - " p3 text,\n" + - " c1 text,\n" + - " col1 text,\n" + - " col2 text,\n" + - " amap map,\n" + - " PRIMARY KEY ((p1, p2, p3), c1)\n" + - ") WITH CLUSTERING ORDER BY (c1 ASC)\n" + - " AND cdc = true;"); - - runCql(new BatchStatementBuilder(LOGGED) - .addStatement(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .usingTimestamp(1683810323861L) - .where(column("p1").isEqualTo(literal("abc"))) - .where(column("p2").isEqualTo(literal("p2value"))) - .where(column("p3").isEqualTo(literal("p3value"))) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .value("col1", literal("")) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value1")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .addStatement(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("p1", literal("abc")) - .value("p2", literal("p2value")) - .value("p3", literal("p3value")) - .value("c1", literal("c1value2")) - .value("col1", literal("col1value")) - .value("col2", literal("col2value")) - .value("amap", literal(new HashMap() { - { - put("key", "value"); - } - })) - .usingTimestamp(1683810323862L) - .usingTtl(3600) - .build()) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - assertEventTypes(getEvents(4), CHANGE_EVENT, DELETE, INSERT, INSERT, INSERT); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/ClusteringPartitionKeyCommitLogProcessingTest.java b/dse/src/test/java/io/debezium/connector/dse/ClusteringPartitionKeyCommitLogProcessingTest.java deleted file mode 100644 index cb526b46..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/ClusteringPartitionKeyCommitLogProcessingTest.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; - -public class ClusteringPartitionKeyCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record delete = (Record) events.get(2); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/CommitLogRealTimeParserTest.java b/dse/src/test/java/io/debezium/connector/dse/CommitLogRealTimeParserTest.java deleted file mode 100644 index c894d060..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/CommitLogRealTimeParserTest.java +++ /dev/null @@ -1,116 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.awaitility.Awaitility.await; -import static org.junit.Assert.assertEquals; - -import java.io.File; -import java.time.Duration; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.TimeUnit; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.junit.Assert; -import org.junit.Before; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import io.debezium.config.Configuration; -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.connector.cassandra.CassandraConnectorConfig; -import io.debezium.connector.cassandra.CassandraConnectorContext; -import io.debezium.connector.cassandra.CommitLogIdxProcessor; -import io.debezium.connector.cassandra.CommitLogUtil; -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; -import io.debezium.connector.cassandra.TestUtils; - -public class CommitLogRealTimeParserTest extends AbstractCommitLogProcessorTest { - private static final Logger LOGGER = LoggerFactory.getLogger(CommitLogRealTimeParserTest.class); - private CommitLogIdxProcessor commitLogProcessor; - - @Before - public void setUp() throws Exception { - super.setUp(); - commitLogProcessor = new CommitLogIdxProcessor(context, metrics, - new DseCommitLogSegmentReader(context, metrics), - DatabaseDescriptor.getCDCLogLocation()); - readLogs(); - } - - @Override - public CassandraConnectorContext generateTaskContext() throws Exception { - Properties properties = TestUtils.generateDefaultConfigMap(); - properties.put(CassandraConnectorConfig.COMMIT_LOG_REAL_TIME_PROCESSING_ENABLED.name(), "true"); - properties.put(CassandraConnectorConfig.COMMIT_LOG_MARKED_COMPLETE_POLL_INTERVAL_IN_MS.name(), "1000"); - return generateTaskContext(Configuration.from(properties)); - } - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - insertRows(3, 10); - } - - private void insertRows(int count, int keyInc) { - for (int i = 0; i < count; i++) { - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(i + keyInc)) - .value("b", literal(i)) - .build()); - } - LOGGER.info("Inserted rows: {}", count); - } - - @Override - public void verifyEvents() { - verify(3, 10); - insertRows(2, 20); - verify(2, 20); - } - - private void verify(int expectedEventsCount, int keyInc) { - List events = new ArrayList<>(); - await().atMost(5, TimeUnit.SECONDS).until(() -> { - events.addAll(context.getQueues().get(0).poll()); - return events.size() == expectedEventsCount; - }); - - LOGGER.info("Total events received: {}", events.size()); - Assert.assertEquals("Total number of events received must be " + expectedEventsCount, expectedEventsCount, events.size()); - - for (int i = 0; i < expectedEventsCount; i++) { - Record record = (Record) events.get(i); - Record.Operation op = record.getOp(); - Assert.assertEquals("Operation type must be insert but it was " + op, Record.Operation.INSERT, op); - Assert.assertEquals("Inserted key should be " + i + keyInc, record.getRowData().getPrimary().get(0).value, i + keyInc); - } - } - - private void readLogs() { - // check to make sure there are no records in the queue to begin with - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - File cdcLoc = DatabaseDescriptor.getCDCLogLocation(); - LOGGER.info("CDC Location: {}", cdcLoc); - await().timeout(Duration.ofSeconds(3)).until(() -> CommitLogUtil.getIndexes(cdcLoc).length >= 1); - File[] commitLogIndexes = CommitLogUtil.getIndexes(cdcLoc); - Arrays.sort(commitLogIndexes, (file1, file2) -> CommitLogUtil.compareCommitLogsIndexes(file1, file2)); - Assert.assertTrue("At least one idx file must be generated", commitLogIndexes.length >= 1); - // Submitting the last idx file as that one is generated by current test - commitLogProcessor.submit(commitLogIndexes[commitLogIndexes.length - 1].toPath()); - } - -} diff --git a/dse/src/test/java/io/debezium/connector/dse/DseConnectorTestBase.java b/dse/src/test/java/io/debezium/connector/dse/DseConnectorTestBase.java deleted file mode 100644 index ac1a8c28..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/DseConnectorTestBase.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.CassandraConnectorConfig; -import io.debezium.connector.cassandra.CassandraConnectorContext; -import io.debezium.connector.cassandra.CassandraConnectorTestBase; -import io.debezium.connector.cassandra.FileOffsetWriter; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public abstract class DseConnectorTestBase extends CassandraConnectorTestBase { - - @Override - protected CassandraConnectorContext generateTaskContext(Configuration configuration) throws Exception { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - DseTypeProvider provider = new DseTypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(config, - new DseSchemaLoader(), - new DseSchemaChangeListenerProvider(), - new FileOffsetWriter(config)); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/DseTestProvider.java b/dse/src/test/java/io/debezium/connector/dse/DseTestProvider.java new file mode 100644 index 00000000..c1721d71 --- /dev/null +++ b/dse/src/test/java/io/debezium/connector/dse/DseTestProvider.java @@ -0,0 +1,86 @@ +/* + * 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.dse; + +import java.io.File; +import java.io.IOException; + +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.commitlog.CommitLogReader; + +import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.CassandraConnectorConfig; +import io.debezium.connector.cassandra.CassandraConnectorContext; +import io.debezium.connector.cassandra.CommitLogProcessorMetrics; +import io.debezium.connector.cassandra.CommitLogSegmentReader; +import io.debezium.connector.cassandra.CommitLogUtil; +import io.debezium.connector.cassandra.DefaultCassandraConnectorContext; +import io.debezium.connector.cassandra.FileOffsetWriter; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.spi.CommitLogProcessing; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; + +public class DseTestProvider implements CassandraTestProvider { + @Override + public CassandraConnectorContext provideContext(Configuration configuration) throws Exception { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + DseTypeProvider provider = new DseTypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(config, + new DseSchemaLoader(), + new DseSchemaChangeListenerProvider(), + new FileOffsetWriter(config)); + } + + @Override + public CassandraConnectorContext provideContextWithoutSchemaManagement(Configuration configuration) { + CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); + DseTypeProvider provider = new DseTypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), + provider.baseTypeForReversedType()); + + return new DefaultCassandraConnectorContext(new CassandraConnectorConfig(configuration)); + } + + @Override + public CommitLogProcessing provideCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + return new DseCommitLogProcessing(context, metrics); + } + + private static class DseCommitLogProcessing implements CommitLogProcessing { + + private final CommitLogReadHandler commitLogReadHandler; + private final CommitLogSegmentReader commitLogSegmentReader; + private final CassandraConnectorContext context; + + DseCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + commitLogReadHandler = new DseCommitLogReadHandlerImpl(context, metrics); + commitLogSegmentReader = new DseCommitLogSegmentReader(context, metrics); + this.context = context; + } + + @Override + public void readAllCommitLogs(File[] commitLogs) throws IOException { + CommitLogReader reader = new CommitLogReader(); + File cdcLoc = new File(context.getCassandraConnectorConfig().getCDCLogLocation()); + for (File commitLog : CommitLogUtil.getCommitLogs(cdcLoc)) { + reader.readCommitLogSegment(commitLogReadHandler, commitLog, true); + } + } + + @Override + public void readCommitLogSegment(File file, long segmentId, int position) throws IOException { + commitLogSegmentReader.readCommitLogSegment(file, segmentId, position); + } + + @Override + public CommitLogSegmentReader getCommitLogSegmentReader() { + return commitLogSegmentReader; + } + } +} diff --git a/dse/src/test/java/io/debezium/connector/dse/MultipleTablesProcessingTest.java b/dse/src/test/java/io/debezium/connector/dse/MultipleTablesProcessingTest.java deleted file mode 100644 index e148f4fd..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/MultipleTablesProcessingTest.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.createTestKeyspace; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; - -public class MultipleTablesProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); - - createTestKeyspace(TEST_KEYSPACE_NAME_2); - - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME); - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .build()); - - runCql(insertInto(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2) - .value("a", literal(1)) - .value("b", literal(2)) - .value("c", literal(3)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - runCql(deleteFrom(TEST_KEYSPACE_NAME_2, TEST_TABLE_NAME_2).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(8); - - Record insert1 = (Record) events.get(0); - assertEquals(insert1.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert1.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record insert3 = (Record) events.get(2); - assertEquals(insert3.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert3.getOp()); - - Record insert4 = (Record) events.get(3); - assertEquals(insert4.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert4.getOp()); - - Record delete = (Record) events.get(4); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - - Record delete2 = (Record) events.get(5); - assertEquals(delete2.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete2.getOp()); - - Record delete3 = (Record) events.get(6); - assertEquals(delete3.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete3.getOp()); - - Record delete4 = (Record) events.get(7); - assertEquals(delete4.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete4.getOp()); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/PartitionKeyRowDeletionCommitLogProcessingTest.java b/dse/src/test/java/io/debezium/connector/dse/PartitionKeyRowDeletionCommitLogProcessingTest.java deleted file mode 100644 index 8ac407fe..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/PartitionKeyRowDeletionCommitLogProcessingTest.java +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.DELETE; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; - -import java.util.List; - -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; - -public class PartitionKeyRowDeletionCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .build()); - - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME).whereColumn("a").isEqualTo(literal(1)).build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(2); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record delete = (Record) events.get(1); - assertEquals(delete.getEventType(), CHANGE_EVENT); - assertEquals(DELETE, delete.getOp()); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/QueueProcessorTest.java b/dse/src/test/java/io/debezium/connector/dse/QueueProcessorTest.java deleted file mode 100644 index f9736c2a..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/QueueProcessorTest.java +++ /dev/null @@ -1,25 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import io.debezium.config.Configuration; -import io.debezium.connector.cassandra.AbstractQueueProcessorTest; -import io.debezium.connector.cassandra.CassandraConnectorConfig; -import io.debezium.connector.cassandra.CassandraConnectorContext; -import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; - -public class QueueProcessorTest extends AbstractQueueProcessorTest { - @Override - public CassandraConnectorContext generateTaskContext(Configuration configuration) { - - CassandraConnectorConfig config = new CassandraConnectorConfig(configuration); - DseTypeProvider provider = new DseTypeProvider(); - CassandraTypeDeserializer.init(provider.deserializers(), config.getDecimalMode(), config.getVarIntMode(), - provider.baseTypeForReversedType()); - - return new CassandraConnectorContext(new CassandraConnectorConfig(configuration)); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/RangeTombstoneCommitLogProcessingTest.java b/dse/src/test/java/io/debezium/connector/dse/RangeTombstoneCommitLogProcessingTest.java deleted file mode 100644 index 853bd93b..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/RangeTombstoneCommitLogProcessingTest.java +++ /dev/null @@ -1,98 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.deleteFrom; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; -import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.Record.Operation.RANGE_TOMBSTONE; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import java.util.List; - -import org.apache.commons.lang3.tuple.Pair; - -import io.debezium.connector.cassandra.CassandraSchemaFactory.RangeData; -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; - -public class RangeTombstoneCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, c int, d int, e int, PRIMARY KEY (a,b,c,d)) WITH cdc = true;"); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 1, 1, 1); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(1)) - .value("d", literal(1)) - .value("e", literal(1)) - .build()); - - // INSERT INTO test_keyspace.table_name (a, b, c, d, e) VALUES (1, 1, 2, 3, 2); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(1)) - .value("b", literal(1)) - .value("c", literal(2)) - .value("d", literal(3)) - .value("e", literal(2)) - .build()); - - // "DELETE FROM ks.tb WHERE a = 1 AND b = 1 AND c <= 2"; - runCql(deleteFrom(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .whereColumn("a").isEqualTo(literal(1)) - .whereColumn("b").isEqualTo(literal(1)) - .whereColumn("c").isLessThanOrEqualTo(literal(2)) - .build()); - } - - @Override - public void verifyEvents() throws Exception { - List events = getEvents(3); - - assertEquals(3, events.size()); - - Record insert = (Record) events.get(0); - assertEquals(insert.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert.getOp()); - - Record insert2 = (Record) events.get(1); - assertEquals(insert2.getEventType(), CHANGE_EVENT); - assertEquals(INSERT, insert2.getOp()); - - Record range1 = (Record) events.get(2); - assertEquals(range1.getEventType(), CHANGE_EVENT); - assertEquals(RANGE_TOMBSTONE, range1.getOp()); - - RangeData startRange = range1.getRowData().getStartRange(); - RangeData endRange = range1.getRowData().getEndRange(); - - assertNotNull(startRange); - - assertEquals("INCL_START_BOUND", startRange.method); - assertEquals("INCL_END_BOUND", endRange.method); - - Pair bStartValue = startRange.values.get("b"); - assertNotNull(bStartValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bStartValue); - - Pair bEndValue = endRange.values.get("b"); - assertNotNull(bEndValue); - assertEquals(Pair.of("1", "org.apache.cassandra.db.marshal.Int32Type"), bEndValue); - - Pair cEndValue = endRange.values.get("c"); - assertNotNull(cEndValue); - assertEquals(Pair.of("2", "org.apache.cassandra.db.marshal.Int32Type"), cEndValue); - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/RowInsertionModificationCommitLogProcessingTest.java b/dse/src/test/java/io/debezium/connector/dse/RowInsertionModificationCommitLogProcessingTest.java deleted file mode 100644 index 8e39c4d1..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/RowInsertionModificationCommitLogProcessingTest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; - -import org.apache.cassandra.config.DatabaseDescriptor; - -import io.debezium.connector.cassandra.EOFEvent; -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.Record; - -public class RowInsertionModificationCommitLogProcessingTest extends AbstractCommitLogProcessorTest { - - @Override - public void initialiseData() throws Exception { - createTable("CREATE TABLE IF NOT EXISTS %s.%s (a int, b int, PRIMARY KEY(a)) WITH cdc = true;"); - for (int i = 0; i < 10; i++) { - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) - .value("a", literal(i)) - .value("b", literal(i)) - .build()); - } - } - - @Override - public void verifyEvents() throws Exception { - for (Event event : getEvents(10)) { - if (event instanceof Record) { - Record record = (Record) event; - assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertFalse(record.getSource().snapshot); - assertEquals(record.getSource().keyspaceTable.name(), keyspaceTable(TEST_TABLE_NAME)); - } - else if (event instanceof EOFEvent) { - EOFEvent eofEvent = (EOFEvent) event; - assertFalse(context.getErroneousCommitLogs().contains(eofEvent.file.getName())); - } - else { - throw new Exception("unexpected event type"); - } - } - } -} diff --git a/dse/src/test/java/io/debezium/connector/dse/SnapshotProcessorTest.java b/dse/src/test/java/io/debezium/connector/dse/SnapshotProcessorTest.java deleted file mode 100644 index 5b7d5d3e..00000000 --- a/dse/src/test/java/io/debezium/connector/dse/SnapshotProcessorTest.java +++ /dev/null @@ -1,185 +0,0 @@ -/* - * Copyright Debezium Authors. - * - * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 - */ -package io.debezium.connector.dse; - -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.propertiesForContext; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; -import static org.mockito.Mockito.doNothing; -import static org.mockito.Mockito.never; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.junit.Assert; -import org.junit.Test; -import org.mockito.Mockito; - -import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.connector.cassandra.CassandraConnectorConfig; -import io.debezium.connector.cassandra.CassandraConnectorContext; -import io.debezium.connector.cassandra.ChangeRecord; -import io.debezium.connector.cassandra.Event; -import io.debezium.connector.cassandra.OffsetPosition; -import io.debezium.connector.cassandra.Record; -import io.debezium.connector.cassandra.SnapshotProcessor; -import io.debezium.connector.cassandra.TestUtils; - -public class SnapshotProcessorTest extends DseConnectorTestBase { - @Test - public void testSnapshotTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table2") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table2") + "(a, b) VALUES (?, ?)", i + 10, String.valueOf(i + 10)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(2 * tableSize, queue.totalCapacity() - queue.remainingCapacity()); - final List table1 = new ArrayList<>(); - final List table2 = new ArrayList<>(); - for (Event event : queue.poll()) { - ChangeRecord record = (ChangeRecord) event; - Assert.assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - Assert.assertEquals(record.getOp(), Record.Operation.INSERT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); - assertTrue(record.getSource().snapshot); - final String tableName = record.getSource().keyspaceTable.name(); - if (tableName.equals(keyspaceTable("cdc_table"))) { - table1.add(record); - } - else { - table2.add(record); - } - Assert.assertEquals(record.getSource().offsetPosition, OffsetPosition.defaultOffsetPosition()); - } - assertEquals(tableSize, table1.size()); - assertEquals(tableSize, table2.size()); - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotSkipsNonCdcEnabledTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - int tableSize = 5; - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("non_cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = false;"); - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("non_cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - context.cleanUp(); - } - - @Test - public void testSnapshotEmptyTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - AtomicBoolean globalTaskState = new AtomicBoolean(true); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - when(snapshotProcessor.isRunning()).thenReturn(true); - - context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); - - ChangeEventQueue queue = context.getQueues().get(0); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - snapshotProcessor.process(); // records empty table to snapshot.offset, so it won't be snapshotted again - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); - - int tableSize = 5; - for (int i = 0; i < tableSize; i++) { - context.getCassandraClient().execute("INSERT INTO " + keyspaceTable("cdc_table") + "(a, b) VALUES (?, ?)", i, String.valueOf(i)); - } - snapshotProcessor.process(); - assertEquals(queue.totalCapacity(), queue.remainingCapacity()); // newly inserted records should be processed by commit log processor instead - - deleteTestKeyspaceTables(); - deleteTestOffsets(context); - globalTaskState.set(false); - context.cleanUp(); - } - - @Test - public void testSnapshotModeAlways() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.KAFKA_PRODUCER_CONFIG_PREFIX + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.TEST_KAFKA_SERVERS); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "always"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(5)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeInitial() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "initial"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, times(1)).snapshot(); - - context.cleanUp(); - } - - @Test - public void testSnapshotModeNever() throws Exception { - Map configs = propertiesForContext(); - configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "never"); - configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new DseTypeProvider().getClusterName())); - doNothing().when(snapshotProcessorSpy).snapshot(); - - for (int i = 0; i < 5; i++) { - snapshotProcessorSpy.process(); - } - verify(snapshotProcessorSpy, never()).snapshot(); - - context.cleanUp(); - } -} diff --git a/dse/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider b/dse/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider new file mode 100644 index 00000000..521ac0e9 --- /dev/null +++ b/dse/src/test/resources/META-INF/services/io.debezium.connector.cassandra.spi.CassandraTestProvider @@ -0,0 +1 @@ +io.debezium.connector.dse.DseTestProvider \ No newline at end of file diff --git a/pom.xml b/pom.xml index 085beadb..92ec5cc4 100644 --- a/pom.xml +++ b/pom.xml @@ -23,6 +23,7 @@ core + tests cassandra-3 cassandra-4 cassandra-5 @@ -126,19 +127,6 @@ test - - com.github.nosan - embedded-cassandra - ${version.embedded.cassandra} - test - - - com.datastax.oss - java-driver-core - - - - org.testcontainers testcontainers @@ -269,12 +257,6 @@ test - - com.github.nosan - embedded-cassandra - test - - org.awaitility awaitility @@ -311,6 +293,22 @@ + + + org.apache.maven.plugins + maven-jar-plugin + + + test-jar + none + + test-jar + + + + + + diff --git a/tests/pom.xml b/tests/pom.xml new file mode 100644 index 00000000..e2f20e84 --- /dev/null +++ b/tests/pom.xml @@ -0,0 +1,50 @@ + + + + + io.debezium + debezium-connector-reactor-cassandra + 3.0.0-SNAPSHOT + ../pom.xml + + + 4.0.0 + debezium-connector-cassandra-tests + Debezium Connector common tests for Cassandra + jar + + + + io.debezium + debezium-connector-cassandra-core + ${project.version} + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + package + + test-jar + + + + + ${basedir}/target + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + + diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java b/tests/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java similarity index 51% rename from cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java index 41a9da65..264bc4ed 100644 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java @@ -5,44 +5,57 @@ */ package io.debezium.connector.cassandra; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.deleteTestKeyspaceTables; +import static io.debezium.connector.cassandra.utils.TestUtils.deleteTestOffsets; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static java.lang.String.format; import static org.awaitility.Awaitility.await; import static org.junit.Assert.assertEquals; import java.io.File; +import java.io.IOException; import java.time.Duration; import java.util.ArrayList; import java.util.List; +import java.util.ServiceLoader; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; -import org.apache.cassandra.config.DatabaseDescriptor; -import org.apache.cassandra.db.commitlog.CommitLogReadHandler; -import org.apache.cassandra.db.commitlog.CommitLogReader; import org.junit.After; +import org.junit.Assume; import org.junit.Before; import org.junit.Test; +import io.debezium.config.Configuration; import io.debezium.connector.base.ChangeEventQueue; -import io.debezium.util.Testing; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.spi.CommitLogProcessing; +import io.debezium.connector.cassandra.spi.ProvidersResolver; +import io.debezium.connector.cassandra.utils.TestUtils; -public abstract class AbstractCommitLogProcessorTest extends EmbeddedCassandra5ConnectorTestBase { - public CassandraConnectorContext context; +public abstract class AbstractCommitLogProcessorTest extends CassandraConnectorTestBase { protected CommitLogProcessorMetrics metrics = new CommitLogProcessorMetrics(); - private CommitLogReadHandler commitLogReadHandler; + + protected CommitLogProcessing commitLogProcessing; + + public Configuration getContextConfiguration() throws Throwable { + return Configuration.from(TestUtils.generateDefaultConfigMap()); + } @Before - public void setUp() throws Exception { + public void setUp() throws Throwable { initialiseData(); - context = generateTaskContext(); + + provider = ProvidersResolver.resolveConnectorContextProvider(); + context = provider.provideContext(getContextConfiguration()); + + commitLogProcessing = provider.provideCommitLogProcessing(context, metrics); + await().atMost(Duration.ofSeconds(60)).until(() -> context.getSchemaHolder() .getKeyValueSchema(new KeyspaceTable(TEST_KEYSPACE_NAME, TEST_TABLE_NAME)) != null); - commitLogReadHandler = new Cassandra5CommitLogReadHandlerImpl(context, metrics); + metrics.registerMetrics(); } @@ -52,56 +65,69 @@ public void tearDown() throws Exception { metrics.unregisterMetrics(); deleteTestKeyspaceTables(); context.cleanUp(); - Testing.Files.delete(DatabaseDescriptor.getCDCLogLocation()); } @Test - public void test() throws Exception { + public void test() throws Throwable { + assumeTestRuns(); verifyEvents(); } - public abstract void initialiseData() throws Exception; + public void assumeTestRuns() { + } - public abstract void verifyEvents() throws Exception; + protected void assumeNotDse() { + Assume.assumeFalse(ServiceLoader.load(CassandraTestProvider.class).findFirst().get().getClass().getName().contains("io.debezium.connector.dse")); + } - public void createTable(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME); + protected void assumeNotCassandra3() { + Assume.assumeFalse(ServiceLoader.load(CassandraTestProvider.class).findFirst().get().getClass().getName().contains("Cassandra3TestProvider")); } - public void createTable2(String query) throws Exception { - createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME_2); + public abstract void initialiseData() throws Throwable; + + public abstract void verifyEvents() throws Throwable; + + public void createTable(String query) { + createTable(query, TEST_KEYSPACE_NAME, TEST_TABLE_NAME); } - public void createTable(String query, String keyspace, String tableName) throws Exception { + public void createTable(String query, String keyspace, String tableName) { runCql(format(query, keyspace, tableName)); } - public List getEvents(final int expectedSize) throws Exception { + public List getEvents(final int expectedSize) throws Throwable { ChangeEventQueue queue = context.getQueues().get(0); final List events = new ArrayList<>(); + + AtomicReference throwable = new AtomicReference<>(); + await().atMost(60, TimeUnit.SECONDS).until(() -> { - readLogs(queue); + try { + readLogs(queue); + } + catch (IOException t) { + return false; + } + events.clear(); events.addAll(queue.poll()); return events.size() == expectedSize; }); + assertEquals(expectedSize, events.size()); return events; } - private void readLogs(ChangeEventQueue queue) throws Exception { + private void readLogs(ChangeEventQueue queue) throws IOException { // check to make sure there are no records in the queue to begin with assertEquals(queue.totalCapacity(), queue.remainingCapacity()); // process the logs in commit log directory - File cdcLoc = new File(DatabaseDescriptor.getCommitLogLocation()); + File cdcLoc = new File(context.getCassandraConnectorConfig().getCDCLogLocation()); File[] commitLogs = CommitLogUtil.getCommitLogs(cdcLoc); - CommitLogReader reader = new CommitLogReader(); - - for (File commitLog : commitLogs) { - reader.readCommitLogSegment(commitLogReadHandler, new org.apache.cassandra.io.util.File(commitLog), true); - } + commitLogProcessing.readAllCommitLogs(commitLogs); } public void assertEventTypes(List events, Event.EventType eventType, Record.Operation... operations) { diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/BatchTest.java b/tests/src/test/java/io/debezium/connector/cassandra/BatchTest.java similarity index 93% rename from cassandra-4/src/test/java/io/debezium/connector/cassandra/BatchTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/BatchTest.java index 7a09259e..f1dafd02 100644 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/BatchTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/BatchTest.java @@ -13,9 +13,9 @@ import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.DELETE; import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import java.util.HashMap; @@ -91,7 +91,7 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { assertEventTypes(getEvents(4), CHANGE_EVENT, DELETE, INSERT, INSERT, INSERT); } } diff --git a/core/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java b/tests/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java similarity index 82% rename from core/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java rename to tests/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java index 82d3a34d..959d85e5 100644 --- a/core/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/CassandraConnectorTestBase.java @@ -5,9 +5,6 @@ */ package io.debezium.connector.cassandra; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.createTestKeyspace; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; import static java.util.concurrent.TimeUnit.MINUTES; import static java.util.concurrent.TimeUnit.SECONDS; import static org.awaitility.Awaitility.await; @@ -18,7 +15,6 @@ import java.nio.file.attribute.PosixFilePermission; import java.nio.file.attribute.PosixFilePermissions; import java.time.Duration; -import java.util.Map; import java.util.Set; import java.util.function.Consumer; @@ -26,6 +22,7 @@ import org.junit.BeforeClass; import org.junit.ClassRule; import org.testcontainers.containers.BindMode; +import org.testcontainers.containers.Container; import org.testcontainers.containers.GenericContainer; import org.testcontainers.images.builder.ImageFromDockerfile; @@ -36,16 +33,21 @@ import com.github.dockerjava.api.model.PortBinding; import com.github.dockerjava.api.model.Ports; -import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.spi.CassandraTestProvider; +import io.debezium.connector.cassandra.utils.TestUtils; import io.debezium.util.Testing; public abstract class CassandraConnectorTestBase { + public static final String CLUSTER_NAME = "Test Cluster"; public static final String CASSANDRA_SERVER_DIR = "/var/lib/cassandra"; private static final String cassandraDir = createCassandraDir(); private static final String dockerDir = System.getProperty("docker.dir", "docker"); private static final Consumer cmd = e -> e.getHostConfig().withPortBindings(new PortBinding(Ports.Binding.bindPort(9042), new ExposedPort(9042))); + protected CassandraConnectorContext context; + protected CassandraTestProvider provider; + @ClassRule public static GenericContainer cassandra = new GenericContainer(new ImageFromDockerfile().withFileFromPath(".", (new File(dockerDir)).toPath())) .withExposedPorts(9042) @@ -57,16 +59,24 @@ public abstract class CassandraConnectorTestBase { @BeforeClass public static void setUpClass() throws Exception { waitForCql(); - createTestKeyspace(); + TestUtils.createTestKeyspace(); } @AfterClass - public static void tearDownClass() { + public static void tearDownClass() throws Throwable { + Container.ExecResult rm = cassandra.execInContainer("rm", + "-rf", + "/var/lib/cassandra/hints", + "/var/lib/cassandra/data", + "/var/lib/cassandra/metadata", + "/var/lib/cassandra/commitlog", + "/var/lib/cassandra/cdc_raw", + "/var/lib/cassandra/saved_caches"); cassandra.stop(); } public static void destroyTestKeyspace() throws Exception { - deleteTestKeyspaceTables(TEST_KEYSPACE_NAME); + TestUtils.deleteTestKeyspaceTables(TestUtils.TEST_KEYSPACE_NAME); } public static void destroyTestKeyspace(String keyspace) { @@ -75,16 +85,6 @@ public static void destroyTestKeyspace(String keyspace) { } } - protected abstract CassandraConnectorContext generateTaskContext(Configuration configuration) throws Exception; - - protected CassandraConnectorContext generateTaskContext() throws Exception { - return generateTaskContext(Configuration.from(TestUtils.generateDefaultConfigMap())); - } - - protected CassandraConnectorContext generateTaskContext(Map configs) throws Exception { - return generateTaskContext(Configuration.from(configs)); - } - protected static void waitForCql() { await() .pollInterval(1, SECONDS) diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java b/tests/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java similarity index 88% rename from cassandra-5/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java index d720f720..a64c7790 100644 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java @@ -11,9 +11,9 @@ import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.DELETE; import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.junit.Assert.assertEquals; import java.util.List; @@ -40,7 +40,7 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { List events = getEvents(3); Record insert1 = (Record) events.get(0); diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java b/tests/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java similarity index 88% rename from cassandra-4/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java index bb24ec15..ad332eaa 100644 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java @@ -11,12 +11,12 @@ import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.DELETE; import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME_2; -import static io.debezium.connector.cassandra.TestUtils.createTestKeyspace; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME_2; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME_2; +import static io.debezium.connector.cassandra.utils.TestUtils.createTestKeyspace; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.junit.Assert.assertEquals; import java.util.List; @@ -69,7 +69,7 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { List events = getEvents(8); Record insert1 = (Record) events.get(0); diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java b/tests/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java similarity index 85% rename from cassandra-3/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java index 6ba7edfa..dd63baf9 100644 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java @@ -11,9 +11,9 @@ import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.DELETE; import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.junit.Assert.assertEquals; import java.util.List; @@ -33,7 +33,7 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { List events = getEvents(2); Record insert = (Record) events.get(0); diff --git a/core/src/test/java/io/debezium/connector/cassandra/AbstractQueueProcessorTest.java b/tests/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java similarity index 93% rename from core/src/test/java/io/debezium/connector/cassandra/AbstractQueueProcessorTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java index 5cb75a9b..76cb3fa2 100644 --- a/core/src/test/java/io/debezium/connector/cassandra/AbstractQueueProcessorTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java @@ -14,7 +14,7 @@ import static io.debezium.connector.cassandra.KeyValueSchema.getPrimaryKeySchemas; import static io.debezium.connector.cassandra.Record.Operation.INSERT; import static io.debezium.connector.cassandra.Record.Operation.RANGE_TOMBSTONE; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; @@ -30,9 +30,11 @@ import io.debezium.config.Configuration; import io.debezium.connector.base.ChangeEventQueue; import io.debezium.connector.cassandra.CassandraSchemaFactory.RowData; +import io.debezium.connector.cassandra.spi.ProvidersResolver; +import io.debezium.connector.cassandra.utils.TestUtils; import io.debezium.time.Conversions; -public abstract class AbstractQueueProcessorTest { +public class QueueProcessorTest { private CassandraConnectorContext context; private QueueProcessor queueProcessor; private TestingKafkaRecordEmitter emitter; @@ -41,7 +43,9 @@ public abstract class AbstractQueueProcessorTest { private SourceInfo sourceInfo; private CassandraSchemaFactory schemaFactory; - public abstract CassandraConnectorContext generateTaskContext(Configuration configuration); + public CassandraConnectorContext generateTaskContext(Configuration configuration) { + return ProvidersResolver.resolveConnectorContextProvider().provideContextWithoutSchemaManagement(configuration); + } @Before public void setUp() throws Exception { diff --git a/cassandra-3/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java b/tests/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java similarity index 93% rename from cassandra-3/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java index 835dad49..778f3d16 100644 --- a/cassandra-3/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java @@ -11,9 +11,9 @@ import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.INSERT; import static io.debezium.connector.cassandra.Record.Operation.RANGE_TOMBSTONE; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -56,7 +56,7 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { List events = getEvents(3); assertEquals(3, events.size()); diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java b/tests/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java similarity index 77% rename from cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java index ad93e843..647d941c 100644 --- a/cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java @@ -7,15 +7,13 @@ import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.runCql; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_KEYSPACE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.TEST_TABLE_NAME; +import static io.debezium.connector.cassandra.utils.TestUtils.keyspaceTable; +import static io.debezium.connector.cassandra.utils.TestUtils.runCql; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import org.apache.cassandra.config.DatabaseDescriptor; - public class RowInsertionModificationCommitLogProcessingTest extends AbstractCommitLogProcessorTest { @Override @@ -30,12 +28,12 @@ public void initialiseData() throws Exception { } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { for (Event event : getEvents(10)) { if (event instanceof Record) { Record record = (Record) event; assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); + assertEquals(record.getSource().cluster, context.getCassandraConnectorConfig().clusterName()); assertFalse(record.getSource().snapshot); assertEquals(record.getSource().keyspaceTable.name(), keyspaceTable(TEST_TABLE_NAME)); } diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java b/tests/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java similarity index 75% rename from cassandra-4/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java index 88f730a7..47baf55d 100644 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java @@ -5,34 +5,33 @@ */ package io.debezium.connector.cassandra; -import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.insertInto; import static com.datastax.oss.driver.api.querybuilder.QueryBuilder.literal; import static io.debezium.connector.cassandra.Event.EventType.CHANGE_EVENT; import static io.debezium.connector.cassandra.Record.Operation.INSERT; -import static io.debezium.connector.cassandra.TestUtils.TEST_KEYSPACE_NAME; -import static io.debezium.connector.cassandra.TestUtils.TEST_TABLE_NAME; -import static io.debezium.connector.cassandra.TestUtils.runCql; -import static java.lang.String.format; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import java.util.List; +import com.datastax.oss.driver.api.querybuilder.QueryBuilder; + +import io.debezium.connector.cassandra.utils.TestUtils; + public class SchemaChangeListenerTest extends AbstractCommitLogProcessorTest { @Override public void initialiseData() throws Exception { createTable("CREATE TABLE %s.%s (a int, b int, PRIMARY KEY ((a), b)) WITH cdc = true;", - TEST_KEYSPACE_NAME, TEST_TABLE_NAME); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) + TestUtils.TEST_KEYSPACE_NAME, TestUtils.TEST_TABLE_NAME); + TestUtils.runCql(QueryBuilder.insertInto(TestUtils.TEST_KEYSPACE_NAME, TestUtils.TEST_TABLE_NAME) .value("a", literal(1)) .value("b", literal(2)) .build()); } @Override - public void verifyEvents() throws Exception { + public void verifyEvents() throws Throwable { // We have to read the first event before altering the table. // That way we make sure that the initial schema is registered and the schema change code path is triggered. List events = getEvents(1); @@ -43,9 +42,9 @@ public void verifyEvents() throws Exception { assertTrue(insert1.getRowData().hasCell("b")); assertFalse(insert1.getRowData().hasCell("c")); - runCql(format("ALTER TABLE %s.%s ADD c int;", TEST_KEYSPACE_NAME, TEST_TABLE_NAME)); + TestUtils.runCql(String.format("ALTER TABLE %s.%s ADD c int;", TestUtils.TEST_KEYSPACE_NAME, TestUtils.TEST_TABLE_NAME)); - runCql(insertInto(TEST_KEYSPACE_NAME, TEST_TABLE_NAME) + TestUtils.runCql(QueryBuilder.insertInto(TestUtils.TEST_KEYSPACE_NAME, TestUtils.TEST_TABLE_NAME) .value("a", literal(3)) .value("b", literal(4)) .value("c", literal(5)) @@ -60,4 +59,10 @@ public void verifyEvents() throws Exception { assertTrue(insert2.getRowData().hasCell("b")); assertTrue(insert2.getRowData().hasCell("c")); } + + @Override + public void assumeTestRuns() { + assumeNotDse(); + } + } diff --git a/cassandra-4/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java b/tests/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java similarity index 75% rename from cassandra-4/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java rename to tests/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java index 86de1f7e..de0aacc0 100644 --- a/cassandra-4/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java @@ -5,10 +5,10 @@ */ package io.debezium.connector.cassandra; -import static io.debezium.connector.cassandra.TestUtils.deleteTestKeyspaceTables; -import static io.debezium.connector.cassandra.TestUtils.deleteTestOffsets; -import static io.debezium.connector.cassandra.TestUtils.keyspaceTable; -import static io.debezium.connector.cassandra.TestUtils.propertiesForContext; +import static io.debezium.connector.cassandra.utils.TestUtils.deleteTestKeyspaceTables; +import static io.debezium.connector.cassandra.utils.TestUtils.deleteTestOffsets; +import static io.debezium.connector.cassandra.utils.TestUtils.keyspaceTable; +import static io.debezium.connector.cassandra.utils.TestUtils.propertiesForContext; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.doNothing; @@ -22,19 +22,36 @@ import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.cassandra.config.DatabaseDescriptor; import org.apache.kafka.clients.producer.ProducerConfig; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Test; import org.mockito.Mockito; +import io.debezium.config.Configuration; import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.cassandra.spi.ProvidersResolver; +import io.debezium.connector.cassandra.utils.TestUtils; + +public class SnapshotProcessorTest extends CassandraConnectorTestBase { + + @After + public void afterTest() { + if (context != null) { + context.cleanUp(); + } + } + + @Before + public void beforeTest() { + provider = ProvidersResolver.resolveConnectorContextProvider(); + } -public class SnapshotProcessorTest extends EmbeddedCassandra4ConnectorTestBase { @Test - public void testSnapshotTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + public void testSnapshotTable() throws Throwable { + context = provider.provideContext(Configuration.from(TestUtils.generateDefaultConfigMap())); + SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); when(snapshotProcessor.isRunning()).thenReturn(true); int tableSize = 5; @@ -56,7 +73,7 @@ public void testSnapshotTable() throws Exception { ChangeRecord record = (ChangeRecord) event; Assert.assertEquals(record.getEventType(), Event.EventType.CHANGE_EVENT); Assert.assertEquals(record.getOp(), Record.Operation.INSERT); - assertEquals(record.getSource().cluster, DatabaseDescriptor.getClusterName()); + Assert.assertEquals(record.getSource().cluster, CLUSTER_NAME); assertTrue(record.getSource().snapshot); final String tableName = record.getSource().keyspaceTable.name(); if (tableName.equals(keyspaceTable("cdc_table"))) { @@ -71,13 +88,12 @@ public void testSnapshotTable() throws Exception { assertEquals(tableSize, table2.size()); deleteTestKeyspaceTables(); deleteTestOffsets(context); - context.cleanUp(); } @Test - public void testSnapshotSkipsNonCdcEnabledTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + public void testSnapshotSkipsNonCdcEnabledTable() throws Throwable { + context = provider.provideContext(Configuration.from(TestUtils.generateDefaultConfigMap())); + SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); when(snapshotProcessor.isRunning()).thenReturn(true); int tableSize = 5; @@ -93,14 +109,13 @@ public void testSnapshotSkipsNonCdcEnabledTable() throws Exception { deleteTestKeyspaceTables(); deleteTestOffsets(context); - context.cleanUp(); } @Test - public void testSnapshotEmptyTable() throws Exception { - CassandraConnectorContext context = generateTaskContext(); + public void testSnapshotEmptyTable() throws Throwable { + context = provider.provideContext(Configuration.from(TestUtils.generateDefaultConfigMap())); AtomicBoolean globalTaskState = new AtomicBoolean(true); - SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + SnapshotProcessor snapshotProcessor = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); when(snapshotProcessor.isRunning()).thenReturn(true); context.getCassandraClient().execute("CREATE TABLE IF NOT EXISTS " + keyspaceTable("cdc_table") + " (a int, b text, PRIMARY KEY(a)) WITH cdc = true;"); @@ -120,58 +135,53 @@ public void testSnapshotEmptyTable() throws Exception { deleteTestKeyspaceTables(); deleteTestOffsets(context); globalTaskState.set(false); - context.cleanUp(); } @Test - public void testSnapshotModeAlways() throws Exception { + public void testSnapshotModeAlways() throws Throwable { Map configs = propertiesForContext(); configs.put(CassandraConnectorConfig.KAFKA_PRODUCER_CONFIG_PREFIX + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, TestUtils.TEST_KAFKA_SERVERS); configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "always"); configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + + context = provider.provideContext(Configuration.from(configs)); + + SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); doNothing().when(snapshotProcessorSpy).snapshot(); for (int i = 0; i < 5; i++) { snapshotProcessorSpy.process(); } verify(snapshotProcessorSpy, times(5)).snapshot(); - - context.cleanUp(); } @Test - public void testSnapshotModeInitial() throws Exception { + public void testSnapshotModeInitial() throws Throwable { Map configs = propertiesForContext(); configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "initial"); configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + context = provider.provideContext(Configuration.from(configs)); + SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); doNothing().when(snapshotProcessorSpy).snapshot(); for (int i = 0; i < 5; i++) { snapshotProcessorSpy.process(); } verify(snapshotProcessorSpy, times(1)).snapshot(); - - context.cleanUp(); } @Test - public void testSnapshotModeNever() throws Exception { + public void testSnapshotModeNever() throws Throwable { Map configs = propertiesForContext(); configs.put(CassandraConnectorConfig.SNAPSHOT_MODE.name(), "never"); configs.put(CassandraConnectorConfig.SNAPSHOT_POLL_INTERVAL_MS.name(), "0"); - CassandraConnectorContext context = generateTaskContext(configs); - SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, new Cassandra4TypeProvider().getClusterName())); + context = provider.provideContext(Configuration.from(configs)); + SnapshotProcessor snapshotProcessorSpy = Mockito.spy(new SnapshotProcessor(context, context.getClusterName())); doNothing().when(snapshotProcessorSpy).snapshot(); for (int i = 0; i < 5; i++) { snapshotProcessorSpy.process(); } verify(snapshotProcessorSpy, never()).snapshot(); - - context.cleanUp(); } } diff --git a/core/src/test/java/io/debezium/connector/cassandra/TestingKafkaRecordEmitter.java b/tests/src/test/java/io/debezium/connector/cassandra/TestingKafkaRecordEmitter.java similarity index 100% rename from core/src/test/java/io/debezium/connector/cassandra/TestingKafkaRecordEmitter.java rename to tests/src/test/java/io/debezium/connector/cassandra/TestingKafkaRecordEmitter.java diff --git a/tests/src/test/java/io/debezium/connector/cassandra/spi/CassandraTestProvider.java b/tests/src/test/java/io/debezium/connector/cassandra/spi/CassandraTestProvider.java new file mode 100644 index 00000000..96e19137 --- /dev/null +++ b/tests/src/test/java/io/debezium/connector/cassandra/spi/CassandraTestProvider.java @@ -0,0 +1,19 @@ +/* + * 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.cassandra.spi; + +import io.debezium.config.Configuration; +import io.debezium.connector.cassandra.CassandraConnectorContext; +import io.debezium.connector.cassandra.CommitLogProcessorMetrics; + +public interface CassandraTestProvider { + + CassandraConnectorContext provideContext(Configuration configuration) throws Throwable; + + CassandraConnectorContext provideContextWithoutSchemaManagement(Configuration configuration); + + CommitLogProcessing provideCommitLogProcessing(CassandraConnectorContext context, CommitLogProcessorMetrics metrics); +} diff --git a/tests/src/test/java/io/debezium/connector/cassandra/spi/CommitLogProcessing.java b/tests/src/test/java/io/debezium/connector/cassandra/spi/CommitLogProcessing.java new file mode 100644 index 00000000..8d8cc886 --- /dev/null +++ b/tests/src/test/java/io/debezium/connector/cassandra/spi/CommitLogProcessing.java @@ -0,0 +1,19 @@ +/* + * 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.cassandra.spi; + +import java.io.File; +import java.io.IOException; + +import io.debezium.connector.cassandra.CommitLogSegmentReader; + +public interface CommitLogProcessing { + void readAllCommitLogs(File[] commitLogs) throws IOException; + + void readCommitLogSegment(File file, long segmentId, int position) throws IOException; + + CommitLogSegmentReader getCommitLogSegmentReader(); +} diff --git a/tests/src/test/java/io/debezium/connector/cassandra/spi/ProvidersResolver.java b/tests/src/test/java/io/debezium/connector/cassandra/spi/ProvidersResolver.java new file mode 100644 index 00000000..2db890f3 --- /dev/null +++ b/tests/src/test/java/io/debezium/connector/cassandra/spi/ProvidersResolver.java @@ -0,0 +1,24 @@ +/* + * 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.cassandra.spi; + +import java.util.Optional; +import java.util.ServiceLoader; + +public class ProvidersResolver { + + public static CassandraTestProvider resolveConnectorContextProvider() { + ServiceLoader serviceLoader = ServiceLoader.load(CassandraTestProvider.class); + + Optional first = serviceLoader.findFirst(); + + if (first.isEmpty()) { + throw new IllegalStateException("There is no provider of " + CassandraTestProvider.class.getName()); + } + + return first.get(); + } +} diff --git a/core/src/test/java/io/debezium/connector/cassandra/TestUtils.java b/tests/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java similarity index 75% rename from core/src/test/java/io/debezium/connector/cassandra/TestUtils.java rename to tests/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java index 23f5a0e9..d1a1d96e 100644 --- a/core/src/test/java/io/debezium/connector/cassandra/TestUtils.java +++ b/tests/src/test/java/io/debezium/connector/cassandra/utils/TestUtils.java @@ -3,14 +3,13 @@ * * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 */ -package io.debezium.connector.cassandra; +package io.debezium.connector.cassandra.utils; import static com.google.common.collect.ImmutableMap.of; import java.io.File; import java.io.IOException; import java.nio.file.Files; -import java.nio.file.Path; import java.nio.file.Paths; import java.util.HashMap; import java.util.List; @@ -25,6 +24,9 @@ import com.datastax.oss.driver.api.core.cql.Statement; import com.datastax.oss.driver.api.querybuilder.SchemaBuilder; +import io.debezium.connector.cassandra.CassandraConnectorConfig; +import io.debezium.connector.cassandra.CassandraConnectorContext; + public class TestUtils { public static final String TEST_CONNECTOR_NAME = "cassandra-01"; @@ -52,6 +54,7 @@ public static Properties generateDefaultConfigMap() throws IOException { props.put(CassandraConnectorConfig.KEY_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); props.put(CassandraConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); props.put(CassandraConnectorConfig.CASSANDRA_DRIVER_CONFIG_FILE.name(), Paths.get("src/test/resources/application.conf").toAbsolutePath().toString()); + props.put(CassandraConnectorConfig.CASSANDRA_CDC_LOG_LOCATION.name(), Paths.get("target/data/cassandra/cdc_raw").toAbsolutePath().toString()); // props.put(CassandraConnectorConfig.MAX_QUEUE_SIZE.name(), 1_000_000); // props.put(CassandraConnectorConfig.MAX_QUEUE_SIZE_IN_BYTES.name(), 1_000_000_000); return props; @@ -69,6 +72,7 @@ public static HashMap propertiesForContext() throws IOException put(CassandraConnectorConfig.KEY_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); put(CassandraConnectorConfig.VALUE_CONVERTER_CLASS_CONFIG.name(), "org.apache.kafka.connect.json.JsonConverter"); put(CassandraConnectorConfig.CASSANDRA_DRIVER_CONFIG_FILE.name(), Paths.get("src/test/resources/application.conf").toAbsolutePath().toString()); + put(CassandraConnectorConfig.CASSANDRA_CDC_LOG_LOCATION.name(), Paths.get("target/data/cassandra/cdc_raw").toAbsolutePath().toString()); // put(CassandraConnectorConfig.MAX_QUEUE_SIZE.name(), 1_000_000); // put(CassandraConnectorConfig.MAX_QUEUE_SIZE_IN_BYTES.name(), 1_000_000_000); } @@ -98,18 +102,6 @@ public static List getTables(String keyspace, CqlSession session) { .collect(Collectors.toList()); } - public static void truncateTestKeyspaceTableData() { - truncateTestKeyspaceTableData(TEST_KEYSPACE_NAME); - } - - public static void truncateTestKeyspaceTableData(String keyspace) { - try (CqlSession session = CqlSession.builder().build()) { - for (String table : getTables(keyspace, session)) { - session.execute(SimpleStatement.newInstance(String.format("TRUNCATE %s.%s", keyspace, table))); - } - } - } - public static void deleteTestKeyspaceTables() throws Exception { deleteTestKeyspaceTables(TEST_KEYSPACE_NAME); } @@ -154,49 +146,4 @@ public static void deleteTestOffsets(CassandraConnectorContext context) throws I public static String keyspaceTable(String tableName) { return TEST_KEYSPACE_NAME + "." + tableName; } - - /** - * Generate commit log files in directory - */ - public static void populateFakeCommitLogsForDirectory(int numOfFiles, File directory) throws IOException { - if (directory.exists() && !directory.isDirectory()) { - throw new IOException(directory + " is not a directory"); - } - if (!directory.exists() && !directory.mkdir()) { - throw new IOException("Cannot create directory " + directory); - } - clearCommitLogFromDirectory(directory, true); - long prefix = System.currentTimeMillis(); - for (int i = 0; i < numOfFiles; i++) { - long ts = prefix + i; - Path path = Paths.get(directory.getAbsolutePath(), "CommitLog-6-" + ts + ".log"); - boolean success = path.toFile().createNewFile(); - if (!success) { - throw new IOException("Failed to create new commit log for testing"); - } - } - } - - /** - * Delete all commit log files in directory - */ - public static void clearCommitLogFromDirectory(File directory, boolean recursive) throws IOException { - if (!directory.exists() || !directory.isDirectory()) { - throw new IOException(directory + " is not a valid directory"); - } - - File[] commitLogs = CommitLogUtil.getCommitLogs(directory); - for (File commitLog : commitLogs) { - CommitLogUtil.deleteCommitLog(commitLog); - } - - if (recursive) { - File[] directories = directory.listFiles(File::isDirectory); - if (directories != null) { - for (File dir : directories) { - clearCommitLogFromDirectory(dir, true); - } - } - } - } }