From 9ba677ad459a8cd0a4898f46b6e01e147c022143 Mon Sep 17 00:00:00 2001 From: Stefan Miklosovic Date: Wed, 23 Oct 2024 13:10:49 +0200 Subject: [PATCH] DBZ-8347 implementation of Cassandra 5 support --- cassandra-5/pom.xml | 159 ++ .../Cassandra5CommitLogReadHandlerImpl.java | 682 +++++ .../Cassandra5CommitLogSegmentReader.java | 32 + .../cassandra/Cassandra5Connector.java | 17 + .../cassandra/Cassandra5ConnectorTask.java | 15 + .../Cassandra5SchemaChangeListener.java | 277 ++ .../cassandra/Cassandra5TypeProvider.java | 119 + .../cassandra/CassandraConnectorTask.java | 56 + .../type/DurationTypeDeserializer.java | 32 + .../transforms/type/ListTypeDeserializer.java | 32 + .../transforms/type/MapTypeDeserializer.java | 38 + .../transforms/type/SetTypeDeserializer.java | 32 + .../type/TupleTypeDeserializer.java | 42 + .../type/UserDefinedTypeDeserializer.java | 71 + ...pache.kafka.connect.source.SourceConnector | 1 + .../AbstractCommitLogProcessorTest.java | 115 + .../connector/cassandra/BatchTest.java | 97 + ...ngPartitionKeyCommitLogProcessingTest.java | 58 + .../CommitLogRealTimeParserTest.java | 109 + .../EmbeddedCassandra5ConnectorTestBase.java | 28 + .../MultipleTablesProcessingTest.java | 107 + ...KeyRowDeletionCommitLogProcessingTest.java | 47 + .../cassandra/QueueProcessorTest.java | 22 + ...RangeTombstoneCommitLogProcessingTest.java | 96 + ...onModificationCommitLogProcessingTest.java | 51 + .../cassandra/SchemaChangeListenerTest.java | 63 + .../cassandra/SnapshotProcessorTest.java | 177 ++ .../CassandraTypeDeserializerTest.java | 662 +++++ .../src/test/resources/application.conf | 5 + .../resources/cassandra-unit-for-context.yaml | 1434 +++++++++++ .../src/test/resources/docker/Dockerfile | 12 + .../src/test/resources/docker/cassandra.yaml | 2275 +++++++++++++++++ .../src/test/resources/logback-test.xml | 19 + pom.xml | 1 + 34 files changed, 6983 insertions(+) create mode 100644 cassandra-5/pom.xml create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogReadHandlerImpl.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogSegmentReader.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5Connector.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5ConnectorTask.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5SchemaChangeListener.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTask.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/DurationTypeDeserializer.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/ListTypeDeserializer.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/MapTypeDeserializer.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/SetTypeDeserializer.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/TupleTypeDeserializer.java create mode 100644 cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/UserDefinedTypeDeserializer.java create mode 100644 cassandra-5/src/main/resources/META-INF/services/org.apache.kafka.connect.source.SourceConnector create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java create 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/ClusteringPartitionKeyCommitLogProcessingTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java create mode 100644 cassandra-5/src/test/java/io/debezium/connector/cassandra/transforms/CassandraTypeDeserializerTest.java create mode 100644 cassandra-5/src/test/resources/application.conf create mode 100644 cassandra-5/src/test/resources/cassandra-unit-for-context.yaml create mode 100644 cassandra-5/src/test/resources/docker/Dockerfile create mode 100644 cassandra-5/src/test/resources/docker/cassandra.yaml create mode 100644 cassandra-5/src/test/resources/logback-test.xml diff --git a/cassandra-5/pom.xml b/cassandra-5/pom.xml new file mode 100644 index 00000000..84b781ca --- /dev/null +++ b/cassandra-5/pom.xml @@ -0,0 +1,159 @@ + + + + + io.debezium + debezium-connector-reactor-cassandra + ../pom.xml + 3.0.0-SNAPSHOT + + + 4.0.0 + debezium-connector-cassandra-5 + Debezium Connector for Cassandra 5 + jar + + + 5.0.2 + + + + + io.debezium + debezium-connector-cassandra-core + ${project.version} + + + io.debezium + debezium-connector-cassandra-core + ${project.version} + tests + test-jar + test + + + org.apache.cassandra + cassandra-all + + ${version.cassandra5} + + + ch.qos.logback + logback-classic + + + ch.qos.logback + logback-core + + + org.slf4j + log4j-over-slf4j + + + + + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + + org.apache.maven.plugins + maven-assembly-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + + --add-exports java.base/jdk.internal.misc=ALL-UNNAMED + --add-exports java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED + --add-exports java.rmi/sun.rmi.registry=ALL-UNNAMED + --add-exports java.rmi/sun.rmi.server=ALL-UNNAMED + --add-exports java.sql/java.sql=ALL-UNNAMED + --add-exports java.base/java.lang.ref=ALL-UNNAMED + --add-exports jdk.unsupported/sun.misc=ALL-UNNAMED + --add-opens java.base/java.lang.module=ALL-UNNAMED + --add-opens java.base/jdk.internal.loader=ALL-UNNAMED + --add-opens java.base/jdk.internal.ref=ALL-UNNAMED + --add-opens java.base/jdk.internal.reflect=ALL-UNNAMED + --add-opens java.base/jdk.internal.math=ALL-UNNAMED + --add-opens java.base/jdk.internal.module=ALL-UNNAMED + --add-opens java.base/jdk.internal.util.jar=ALL-UNNAMED + --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED + --add-opens java.base/sun.nio.ch=ALL-UNNAMED + --add-opens java.base/java.io=ALL-UNNAMED + --add-opens java.base/java.lang=ALL-UNNAMED + --add-opens java.base/java.lang.reflect=ALL-UNNAMED + --add-opens java.base/java.util=ALL-UNNAMED + --add-opens java.base/java.nio=ALL-UNNAMED + + + ${version.cassandra5} + ${project.basedir}/src/test/resources/docker + + + + + + + + + + assembly + + false + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + io.debezium + debezium-assembly-descriptors + ${project.version} + + + + + default + package + + single + + + ${project.artifactId}-${project.version} + true + + connector-distribution + + posix + + + + + + + + + quick + + false + + quick + + + + true + true + + + + diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogReadHandlerImpl.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogReadHandlerImpl.java new file mode 100644 index 00000000..f6f47309 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogReadHandlerImpl.java @@ -0,0 +1,682 @@ +/* + * 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.Cassandra5CommitLogReadHandlerImpl.PartitionType.PARTITION_AND_CLUSTERING_KEY_ROW_DELETION; +import static io.debezium.connector.cassandra.Cassandra5CommitLogReadHandlerImpl.PartitionType.PARTITION_KEY_ROW_DELETION; +import static io.debezium.connector.cassandra.CassandraSchemaFactory.CellData.ColumnType.CLUSTERING; +import static io.debezium.connector.cassandra.CassandraSchemaFactory.CellData.ColumnType.PARTITION; +import static io.debezium.connector.cassandra.CassandraSchemaFactory.CellData.ColumnType.REGULAR; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.cql3.ColumnSpecification; +import org.apache.cassandra.db.ClusteringBound; +import org.apache.cassandra.db.LivenessInfo; +import org.apache.cassandra.db.Mutation; +import org.apache.cassandra.db.commitlog.CommitLogDescriptor; +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.CollectionType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.partitions.PartitionUpdate; +import org.apache.cassandra.db.rows.ComplexColumnData; +import org.apache.cassandra.db.rows.RangeTombstoneBoundMarker; +import org.apache.cassandra.db.rows.Row; +import org.apache.cassandra.db.rows.Unfiltered; +import org.apache.cassandra.db.rows.UnfilteredRowIterator; +import org.apache.cassandra.utils.ByteBufferUtil; +import org.apache.commons.lang3.tuple.Pair; +import org.apache.kafka.connect.data.Schema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.oss.driver.api.core.metadata.schema.ClusteringOrder; +import com.datastax.oss.driver.api.core.metadata.schema.ColumnMetadata; +import com.datastax.oss.driver.api.core.metadata.schema.TableMetadata; + +import io.debezium.DebeziumException; +import io.debezium.connector.base.ChangeEventQueue; +import io.debezium.connector.cassandra.CassandraSchemaFactory.CellData; +import io.debezium.connector.cassandra.CassandraSchemaFactory.RangeData; +import io.debezium.connector.cassandra.CassandraSchemaFactory.RowData; +import io.debezium.connector.cassandra.exceptions.CassandraConnectorSchemaException; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer; +import io.debezium.time.Conversions; + +/** + * Handler that implements {@link CommitLogReadHandler} interface provided by Cassandra source code. + *

+ * This handler implementation processes each {@link Mutation} and invokes one of the registered partition handler + * for each {@link PartitionUpdate} in the {@link Mutation} (a mutation could have multiple partitions if it is a batch update), + * which in turn makes one or more record via the {@link RecordMaker} and enqueue the record into the {@link ChangeEventQueue}. + */ +public class Cassandra5CommitLogReadHandlerImpl implements CommitLogReadHandler { + private static final Logger LOGGER = LoggerFactory.getLogger(Cassandra5CommitLogReadHandlerImpl.class); + + private static final boolean MARK_OFFSET = true; + + private final List> queues; + private final RecordMaker recordMaker; + private final OffsetWriter offsetWriter; + private final SchemaHolder schemaHolder; + private final CommitLogProcessorMetrics metrics; + private final RangeTombstoneContext rangeTombstoneContext = new RangeTombstoneContext<>(); + private final CassandraSchemaFactory schemaFactory; + private final CassandraConnectorConfig.EventOrderGuaranteeMode eventOrderGuaranteeMode; + + Cassandra5CommitLogReadHandlerImpl(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + this.queues = context.getQueues(); + this.recordMaker = new RecordMaker(context.getCassandraConnectorConfig().tombstonesOnDelete(), + new Filters(context.getCassandraConnectorConfig().fieldExcludeList()), + context.getCassandraConnectorConfig()); + this.offsetWriter = context.getOffsetWriter(); + this.schemaHolder = context.getSchemaHolder(); + this.metrics = metrics; + this.schemaFactory = CassandraSchemaFactory.get(); + this.eventOrderGuaranteeMode = context.getCassandraConnectorConfig().getEventOrderGuaranteeMode(); + } + + /** + * A PartitionType represents the type of PartitionUpdate. + */ + enum PartitionType { + /** + * a partition-level deletion where partition key = primary key (no clustering key) + */ + PARTITION_KEY_ROW_DELETION, + + /** + * a partition-level deletion where partition key + clustering key = primary key + */ + PARTITION_AND_CLUSTERING_KEY_ROW_DELETION, + + /** + * a row-level modification + */ + ROW_LEVEL_MODIFICATION, + + /** + * an update on materialized view + */ + MATERIALIZED_VIEW, + + /** + * an update on secondary index + */ + SECONDARY_INDEX, + + /** + * an update on a table that contains counter data type + */ + COUNTER; + + static final Set supportedPartitionTypes = new HashSet<>(Arrays.asList(PARTITION_KEY_ROW_DELETION, + PARTITION_AND_CLUSTERING_KEY_ROW_DELETION, + ROW_LEVEL_MODIFICATION)); + + public static PartitionType getPartitionType(PartitionUpdate pu) { + if (pu.metadata().isCounter()) { + return COUNTER; + } + else if (pu.metadata().isView()) { + return MATERIALIZED_VIEW; + } + else if (pu.metadata().isIndex()) { + return SECONDARY_INDEX; + } + else if (isPartitionDeletion(pu) && hasClusteringKeys(pu)) { + return PARTITION_AND_CLUSTERING_KEY_ROW_DELETION; + } + else if (isPartitionDeletion(pu) && !hasClusteringKeys(pu)) { + return PARTITION_KEY_ROW_DELETION; + } + else { + return ROW_LEVEL_MODIFICATION; + } + } + + public static boolean isValid(PartitionType type) { + return supportedPartitionTypes.contains(type); + } + + public static boolean hasClusteringKeys(PartitionUpdate pu) { + return !pu.metadata().clusteringColumns().isEmpty(); + } + + public static boolean isPartitionDeletion(PartitionUpdate pu) { + return pu.partitionLevelDeletion().markedForDeleteAt() > LivenessInfo.NO_TIMESTAMP; + } + } + + /** + * A RowType represents different types of {@link Row}-level modifications in a Cassandra table. + */ + enum RowType { + /** + * Single-row insert + */ + INSERT, + + /** + * Single-row update + */ + UPDATE, + + /** + * Single-row delete + */ + DELETE, + + /** + * A row-level deletion that deletes a range of keys. + * For example: DELETE * FROM table WHERE partition_key = 1 AND clustering_key > 0; + */ + RANGE_TOMBSTONE, + + /** + * Unknown row-level operation + */ + UNKNOWN; + + static final Set supportedRowTypes = new HashSet<>(Arrays.asList(INSERT, UPDATE, DELETE, RANGE_TOMBSTONE)); + + public static RowType getRowType(Unfiltered unfiltered) { + if (unfiltered.isRangeTombstoneMarker()) { + return RANGE_TOMBSTONE; + } + else if (unfiltered.isRow()) { + Row row = (Row) unfiltered; + if (isDelete(row)) { + return DELETE; + } + else if (isInsert(row)) { + return INSERT; + } + else if (isUpdate(row)) { + return UPDATE; + } + } + return UNKNOWN; + } + + public static boolean isValid(RowType rowType) { + return supportedRowTypes.contains(rowType); + } + + public static boolean isDelete(Row row) { + return row.deletion().time().markedForDeleteAt() > LivenessInfo.NO_TIMESTAMP; + } + + public static boolean isInsert(Row row) { + return row.primaryKeyLivenessInfo().timestamp() > LivenessInfo.NO_TIMESTAMP; + } + + public static boolean isUpdate(Row row) { + return row.primaryKeyLivenessInfo().timestamp() == LivenessInfo.NO_TIMESTAMP; + } + } + + @Override + public void handleMutation(Mutation mutation, int size, int entryLocation, CommitLogDescriptor descriptor) { + if (!mutation.trackedByCDC()) { + if (LOGGER.isDebugEnabled()) { + LOGGER.debug("not tracked by cdc {}.{{}}", + mutation.getKeyspaceName(), + mutation.getPartitionUpdates() + .stream() + .map(pa -> pa.metadata().name) + .collect(Collectors.toSet())); + } + return; + } + + metrics.setCommitLogPosition(entryLocation); + + for (PartitionUpdate pu : mutation.getPartitionUpdates()) { + OffsetPosition offsetPosition = new OffsetPosition(descriptor.fileName(), entryLocation); + KeyspaceTable keyspaceTable = new KeyspaceTable(mutation.getKeyspaceName(), pu.metadata().name); + + if (offsetWriter.isOffsetProcessed(keyspaceTable.name(), offsetPosition.serialize(), false)) { + LOGGER.info("Mutation at {} for table {} already processed, skipping...", offsetPosition, keyspaceTable); + return; + } + + try { + process(pu, offsetPosition, keyspaceTable); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to process PartitionUpdate %s at %s for table %s.", + pu, offsetPosition, keyspaceTable.name()), e); + } + } + + metrics.onSuccess(); + } + + @Override + public void handleUnrecoverableError(CommitLogReadException exception) { + LOGGER.error("Unrecoverable error when reading commit log", exception); + metrics.onUnrecoverableError(); + } + + @Override + public boolean shouldSkipSegmentOnError(CommitLogReadException exception) { + if (exception.permissible) { + LOGGER.error("Encountered a permissible exception during log replay", exception); + } + else { + LOGGER.error("Encountered a non-permissible exception during log replay", exception); + } + return false; + } + + /** + * Method which processes a partition update if it's valid (either a single-row partition-level + * deletion or a row-level modification) or throw an exception if it isn't. The valid partition + * update is then converted into a {@link Record} and enqueued to the {@link ChangeEventQueue}. + */ + private void process(PartitionUpdate pu, OffsetPosition offsetPosition, KeyspaceTable keyspaceTable) { + PartitionType partitionType = PartitionType.getPartitionType(pu); + + if (!PartitionType.isValid(partitionType)) { + LOGGER.warn("Encountered an unsupported partition type {}, skipping...", partitionType); + return; + } + + if (partitionType == PARTITION_AND_CLUSTERING_KEY_ROW_DELETION || partitionType == PARTITION_KEY_ROW_DELETION) { + handlePartitionDeletion(pu, offsetPosition, keyspaceTable); + } + handleRowIterator(pu, offsetPosition, keyspaceTable); + } + + private void handleRowIterator(PartitionUpdate pu, OffsetPosition offsetPosition, KeyspaceTable keyspaceTable) { + long maxTimePu = pu.maxTimestamp(); + UnfilteredRowIterator it = pu.unfilteredIterator(); + while (it.hasNext()) { + Unfiltered rowOrRangeTombstone = it.next(); + RowType rowType = RowType.getRowType(rowOrRangeTombstone); + if (!RowType.isValid(rowType)) { + LOGGER.warn("Encountered an unsupported row type {}, skipping...", rowType); + continue; + } + if (rowOrRangeTombstone instanceof Row) { + Row row = (Row) rowOrRangeTombstone; + handleRowModifications(row, rowType, pu, offsetPosition, keyspaceTable, maxTimePu); + } + else if (rowOrRangeTombstone instanceof RangeTombstoneBoundMarker) { + handleRangeTombstoneBoundMarker((RangeTombstoneBoundMarker) rowOrRangeTombstone, + rowType, pu, offsetPosition, keyspaceTable); + } + else { + throw new CassandraConnectorSchemaException("Encountered unsupported Unfiltered type " + rowOrRangeTombstone.getClass()); + } + } + it.close(); + } + + /** + * Handle a valid deletion event resulted from a partition-level deletion by converting Cassandra representation + * of this event into a {@link Record} object and queue the record to {@link ChangeEventQueue}. A valid deletion + * event means a partition only has a single row, this implies there are no clustering keys. + * + * The steps are: + * (1) Populate the "source" field for this event + * (2) Fetch the cached key/value schemas from {@link SchemaHolder} + * (3) Populate the "after" field for this event + * a. populate partition columns + * b. populate clustering columns if any + * b. populate regular columns with null values + * (4) Assemble a {@link Record} object from the populated data and queue the record + */ + private void handlePartitionDeletion(PartitionUpdate pu, OffsetPosition offsetPosition, KeyspaceTable keyspaceTable) { + KeyValueSchema keyValueSchema = schemaHolder.getKeyValueSchema(keyspaceTable); + if (keyValueSchema == null) { + LOGGER.warn("Unable to get KeyValueSchema for table {}. It might have been deleted or CDC disabled.", keyspaceTable.toString()); + return; + } + + Schema keySchema = keyValueSchema.keySchema(); + Schema valueSchema = keyValueSchema.valueSchema(); + TableMetadata tableMetadata = keyValueSchema.tableMetadata(); + + RowData after = schemaFactory.rowData(); + + populatePartitionColumns(after, pu); + + // For partition deletions, the PartitionUpdate only specifies the partition key, it does not + // contain any info on regular (non-partition) columns, as if they were not modified. In order + // to differentiate deleted columns from unmodified columns, we populate the deleted columns + // with null value and timestamps + + // clustering columns if any + + List columns = new ArrayList<>(tableMetadata.getColumns().values()); + Map clusteringColumns = tableMetadata.getClusteringColumns(); + + for (Map.Entry clustering : clusteringColumns.entrySet()) { + ColumnMetadata clusteringKey = clustering.getKey(); + long deletionTs = pu.deletionInfo().getPartitionDeletion().markedForDeleteAt(); + after.addCell(schemaFactory.cellData(clusteringKey.getName().toString(), null, deletionTs, CLUSTERING)); + } + + columns.removeAll(tableMetadata.getPartitionKey()); + columns.removeAll(tableMetadata.getClusteringColumns().keySet()); + + // regular columns if any + + for (ColumnMetadata cm : columns) { + String name = cm.getName().toString(); + long deletionTs = pu.deletionInfo().getPartitionDeletion().markedForDeleteAt(); + CellData cellData = schemaFactory.cellData(name, null, deletionTs, REGULAR); + after.addCell(cellData); + } + + recordMaker.delete(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(pu.maxTimestamp()), after, keySchema, valueSchema, + MARK_OFFSET, queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + } + + /** + * Handle a valid event resulted from a row-level modification by converting Cassandra representation of + * this event into a {@link Record} object and queue the record to {@link ChangeEventQueue}. A valid event + * implies this must be an insert, update, or delete. + * + * The steps are: + * (1) Populate the "source" field for this event + * (2) Fetch the cached key/value schemas from {@link SchemaHolder} + * (3) Populate the "after" field for this event + * a. populate partition columns + * b. populate clustering columns + * c. populate regular columns + * d. for deletions, populate regular columns with null values + * (4) Assemble a {@link Record} object from the populated data and queue the record + */ + private void handleRowModifications(Row row, RowType rowType, PartitionUpdate pu, OffsetPosition offsetPosition, KeyspaceTable keyspaceTable, long maxTimePu) { + KeyValueSchema keyValueSchema = schemaHolder.getKeyValueSchema(keyspaceTable); + if (keyValueSchema == null) { + LOGGER.trace("Unable to get KeyValueSchema for table {}. It might have been deleted or CDC disabled.", keyspaceTable.toString()); + return; + } + Schema keySchema = keyValueSchema.keySchema(); + Schema valueSchema = keyValueSchema.valueSchema(); + + RowData after = schemaFactory.rowData(); + populatePartitionColumns(after, pu); + populateClusteringColumns(after, row, pu); + populateRegularColumns(after, row, rowType, keyValueSchema); + + long ts = rowType == RowType.DELETE ? row.deletion().time().markedForDeleteAt() : maxTimePu; + + switch (rowType) { + case INSERT: + recordMaker.insert(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(ts), after, keySchema, valueSchema, MARK_OFFSET, + queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + break; + + case UPDATE: + recordMaker.update(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(ts), after, keySchema, valueSchema, MARK_OFFSET, + queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + break; + + case DELETE: + recordMaker.delete(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(ts), after, keySchema, valueSchema, MARK_OFFSET, + queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + break; + + case RANGE_TOMBSTONE: + recordMaker.rangeTombstone(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(ts), after, keySchema, valueSchema, MARK_OFFSET, + queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + break; + + default: + throw new CassandraConnectorSchemaException("Unsupported row type " + rowType + " should have been skipped"); + } + } + + private void handleRangeTombstoneBoundMarker(RangeTombstoneBoundMarker rangeTombstoneMarker, + RowType rowType, + PartitionUpdate pu, + OffsetPosition offsetPosition, + KeyspaceTable keyspaceTable) { + if (rowType != RowType.RANGE_TOMBSTONE) { + throw new IllegalStateException("Row type has to be " + RowType.RANGE_TOMBSTONE.name()); + } + KeyValueSchema keyValueSchema = schemaHolder.getKeyValueSchema(keyspaceTable); + if (keyValueSchema == null) { + LOGGER.warn("Unable to get KeyValueSchema for table {}. It might have been deleted or CDC disabled.", keyspaceTable.toString()); + return; + } + + RowData after = rangeTombstoneContext.getOrCreate(pu.metadata()); + + Optional.ofNullable(rangeTombstoneMarker.openBound(false)).ifPresent(cb -> { + after.addStartRange(populateRangeData(cb, RangeData.RANGE_START_NAME, pu.metadata())); + }); + + Optional.ofNullable(rangeTombstoneMarker.closeBound(false)).ifPresent(cb -> { + after.addEndRange(populateRangeData(cb, RangeData.RANGE_END_NAME, pu.metadata())); + }); + + if (RangeTombstoneContext.isComplete(after)) { + try { + populatePartitionColumns(after, pu); + long ts = rangeTombstoneMarker.deletionTime().markedForDeleteAt(); + + recordMaker.rangeTombstone(DatabaseDescriptor.getClusterName(), offsetPosition, keyspaceTable, false, + Conversions.toInstantFromMicros(ts), after, keyValueSchema.keySchema(), keyValueSchema.valueSchema(), MARK_OFFSET, + queues.get(getPartitionQueueIndex(pu, offsetPosition))::enqueue); + } + finally { + rangeTombstoneContext.remove(pu.metadata()); + } + } + } + + private RangeData populateRangeData(ClusteringBound cb, String name, org.apache.cassandra.schema.TableMetadata metaData) { + Map> values = new HashMap<>(); + + for (int i = 0; i < cb.size(); i++) { + String clusteringColumnName = metaData.clusteringColumns().get(i).name.toCQLString(); + String clusteringColumnValue = metaData.comparator.subtype(i).getString(cb.bufferAt(i)); + String clusteringColumnType = metaData.clusteringColumns().get(i).type.toString(); + values.put(clusteringColumnName, Pair.of(clusteringColumnValue, clusteringColumnType)); + } + + return schemaFactory.rangeData(name, cb.kind().toString(), values); + } + + private void populatePartitionColumns(RowData after, PartitionUpdate pu) { + // if it has any cells it was already populated + if (after.hasAnyCell()) { + return; + } + List partitionKeys = getPartitionKeys(pu); + for (org.apache.cassandra.schema.ColumnMetadata cd : pu.metadata().partitionKeyColumns()) { + try { + String name = cd.name.toString(); + Object value = partitionKeys.get(cd.position()); + CellData cellData = schemaFactory.cellData(name, value, null, PARTITION); + after.addCell(cellData); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to populate Column %s with Type %s of Table %s in KeySpace %s.", + cd.name.toString(), cd.type.toString(), cd.cfName, pu.metadata().keyspace), e); + } + } + } + + private void populateClusteringColumns(RowData after, Row row, PartitionUpdate pu) { + for (org.apache.cassandra.schema.ColumnMetadata cd : pu.metadata().clusteringColumns()) { + try { + ByteBuffer bufferAtClustering = row.clustering().bufferAt(cd.position()); + Object value = CassandraTypeDeserializer.deserialize(cd.type, bufferAtClustering); + CellData cellData = schemaFactory.cellData(cd.name.toString(), value, null, CLUSTERING); + after.addCell(cellData); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to populate Column %s with Type %s of Table %s in KeySpace %s.", + cd.name.toString(), cd.type.toString(), cd.cfName, pu.metadata().keyspace), e); + } + } + } + + private void populateRegularColumns(RowData after, Row row, RowType rowType, KeyValueSchema schema) { + if (rowType == RowType.INSERT || rowType == RowType.UPDATE) { + for (org.apache.cassandra.schema.ColumnMetadata cd : row.columns()) { + try { + Object value; + Object deletionTs = null; + AbstractType abstractType = cd.type; + if (abstractType.isCollection() && abstractType.isMultiCell()) { + ComplexColumnData ccd = row.getComplexColumnData(cd); + value = CassandraTypeDeserializer.deserialize((CollectionType) abstractType, getComplexColumnDataByteBufferList(abstractType, ccd)); + } + else { + org.apache.cassandra.db.rows.Cell cell = row.getCell(cd); + value = cell.isTombstone() ? null : CassandraTypeDeserializer.deserialize(abstractType, cell.buffer()); + deletionTs = cell.isExpiring() ? TimeUnit.MICROSECONDS.convert(cell.localDeletionTime(), TimeUnit.SECONDS) : null; + } + String name = cd.name.toString(); + CellData cellData = schemaFactory.cellData(name, value, deletionTs, REGULAR); + after.addCell(cellData); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to populate Column %s with Type %s of Table %s in KeySpace %s.", + cd.name.toString(), cd.type.toString(), cd.cfName, cd.ksName), e); + } + } + + } + else if (rowType == RowType.DELETE) { + // For row-level deletions, row.columns() will result in an empty list and does not contain + // the column definitions for the deleted columns. In order to differentiate deleted columns from + // unmodified columns, we populate the deleted columns with null value and timestamps. + TableMetadata tableMetadata = schema.tableMetadata(); + List columns = new ArrayList<>(tableMetadata.getColumns().values()); + columns.removeAll(tableMetadata.getPrimaryKey()); + for (ColumnMetadata cm : columns) { + String name = cm.getName().toString(); + long deletionTs = row.deletion().time().markedForDeleteAt(); + CellData cellData = schemaFactory.cellData(name, null, deletionTs, REGULAR); + after.addCell(cellData); + } + } + } + + private List getComplexColumnDataByteBufferList(AbstractType abstractType, ComplexColumnData ccd) { + if (abstractType instanceof ListType) { + return ((ListType) abstractType).serializedValues(ccd.iterator()); + } + if (abstractType instanceof SetType) { + return ((SetType) abstractType).serializedValues(ccd.iterator()); + } + if (abstractType instanceof MapType) { + return ((MapType) abstractType).serializedValues(ccd.iterator()); + } + throw new DebeziumException(String.format("Unknow collection type %s", abstractType)); + } + + /** + * Given a PartitionUpdate, deserialize the partition key byte buffer + * into a list of partition key values. + */ + @SuppressWarnings("checkstyle:magicnumber") + private List getPartitionKeys(PartitionUpdate pu) { + List values = new ArrayList<>(); + + List columnDefinitions = pu.metadata().partitionKeyColumns(); + + // simple partition key + if (columnDefinitions.size() == 1) { + ByteBuffer bb = pu.partitionKey().getKey(); + ColumnSpecification cs = columnDefinitions.get(0); + AbstractType type = cs.type; + try { + Object value = CassandraTypeDeserializer.deserialize(type, bb); + values.add(value); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to deserialize Column %s with Type %s in Table %s and KeySpace %s.", + cs.name.toString(), type.toString(), pu.metadata().name, pu.metadata().keyspace), e); + } + + // composite partition key + } + else { + ByteBuffer keyBytes = pu.partitionKey().getKey().duplicate(); + + // 0xFFFF is reserved to encode "static column", skip if it exists at the start + if (keyBytes.remaining() >= 2) { + int header = ByteBufferUtil.getShortLength(keyBytes, keyBytes.position()); + if ((header & 0xFFFF) == 0xFFFF) { + ByteBufferUtil.readShortLength(keyBytes); + } + } + + // the encoding of columns in the partition key byte buffer is + // ... + // where is: + // + // is a 2 bytes unsigned short (excluding 0xFFFF used to encode "static columns") + // should always be 0 for columns (1 for query bounds) + // this section reads the bytes for each column and deserialize into objects based on each column type + int i = 0; + while (keyBytes.remaining() > 0 && i < columnDefinitions.size()) { + ColumnSpecification cs = columnDefinitions.get(i); + AbstractType type = cs.type; + ByteBuffer bb = ByteBufferUtil.readBytesWithShortLength(keyBytes); + try { + Object value = CassandraTypeDeserializer.deserialize(type, bb); + values.add(value); + } + catch (Exception e) { + throw new DebeziumException(String.format("Failed to deserialize Column %s with Type %s in Table %s and KeySpace %s", + cs.name.toString(), cs.type.toString(), cs.cfName, cs.ksName), e); + } + byte b = keyBytes.get(); + if (b != 0) { + break; + } + ++i; + } + } + + return values; + } + + private int getPartitionQueueIndex(PartitionUpdate partitionUpdate, OffsetPosition offsetPosition) { + int hash; + switch (eventOrderGuaranteeMode) { + case COMMITLOG_FILE: + hash = offsetPosition.fileName.hashCode(); + break; + case PARTITION_VALUES: + hash = partitionUpdate.partitionKey().hashCode(); + break; + default: + throw new IllegalStateException(); + } + return ((hash % queues.size()) + queues.size()) % queues.size(); + } +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogSegmentReader.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogSegmentReader.java new file mode 100644 index 00000000..0e021332 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5CommitLogSegmentReader.java @@ -0,0 +1,32 @@ +/* + * 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.CommitLogPosition; +import org.apache.cassandra.db.commitlog.CommitLogReadHandler; +import org.apache.cassandra.db.commitlog.CommitLogReader; + +public class Cassandra5CommitLogSegmentReader implements CommitLogSegmentReader { + + private final CommitLogReader commitLogReader; + + private final CommitLogReadHandler commitLogReadHandler; + + public Cassandra5CommitLogSegmentReader(CassandraConnectorContext context, CommitLogProcessorMetrics metrics) { + this.commitLogReader = new CommitLogReader(); + this.commitLogReadHandler = new Cassandra5CommitLogReadHandlerImpl(context, metrics); + } + + @Override + public void readCommitLogSegment(File file, long segmentId, int position) throws IOException { + commitLogReader.readCommitLogSegment(commitLogReadHandler, new org.apache.cassandra.io.util.File(file), new CommitLogPosition(segmentId, position), + CommitLogReader.ALL_MUTATIONS, false); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5Connector.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5Connector.java new file mode 100644 index 00000000..2a6804cf --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5Connector.java @@ -0,0 +1,17 @@ +/* + * 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 org.apache.kafka.connect.connector.Task; + +public class Cassandra5Connector extends AbstractSourceConnector { + + @Override + public Class taskClass() { + return Cassandra5ConnectorTask.class; + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5ConnectorTask.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5ConnectorTask.java new file mode 100644 index 00000000..398588f1 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5ConnectorTask.java @@ -0,0 +1,15 @@ +/* + * 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; + +public class Cassandra5ConnectorTask extends AbstractConnectorTask { + + @Override + protected CassandraConnectorTaskTemplate init(CassandraConnectorConfig config, ComponentFactory factory) { + return CassandraConnectorTask.init(config, factory); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5SchemaChangeListener.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5SchemaChangeListener.java new file mode 100644 index 00000000..c51b828f --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5SchemaChangeListener.java @@ -0,0 +1,277 @@ +/* + * 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.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; + +import javax.annotation.Nonnull; + +import org.apache.cassandra.cql3.statements.schema.CreateTableStatement; +import org.apache.cassandra.db.ColumnFamilyStore; +import org.apache.cassandra.db.Directories; +import org.apache.cassandra.db.Keyspace; +import org.apache.cassandra.schema.KeyspaceParams; +import org.apache.cassandra.schema.Schema; +import org.apache.cassandra.schema.TableId; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.schema.TableMetadataRef; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.oss.driver.api.core.CqlIdentifier; +import com.datastax.oss.driver.api.core.metadata.schema.KeyspaceMetadata; +import com.datastax.oss.driver.api.core.session.Session; + +import io.debezium.connector.SourceInfoStructMaker; + +public class Cassandra5SchemaChangeListener extends AbstractSchemaChangeListener { + + private static final Logger LOGGER = LoggerFactory.getLogger(Cassandra5SchemaChangeListener.class); + + public Cassandra5SchemaChangeListener(String kafkaTopicPrefix, + SourceInfoStructMaker sourceInfoStructMaker, + SchemaHolder schemaHolder) { + super(kafkaTopicPrefix, sourceInfoStructMaker, schemaHolder); + } + + @Override + public void onSessionReady(@Nonnull Session session) { + LOGGER.info("Initializing SchemaHolder ..."); + List cdcEnabledTableMetadataList = getCdcEnabledTableMetadataList(session); + for (com.datastax.oss.driver.api.core.metadata.schema.TableMetadata tm : cdcEnabledTableMetadataList) { + schemaHolder.addOrUpdateTableSchema(new KeyspaceTable(tm), getKeyValueSchema(tm)); + + Optional keyspace = session.getMetadata().getKeyspace(tm.getKeyspace().toString()); + assert keyspace.isPresent(); + + onKeyspaceCreated(keyspace.get()); + onTableCreated(tm); + } + + Set cdcEnabledEntities = schemaHolder.getCdcEnabledTableMetadataSet() + .stream() + .map(tmd -> tmd.describe(true)) + .collect(Collectors.toSet()); + + LOGGER.info("CDC enabled entities: {}", cdcEnabledEntities); + LOGGER.info("Initialized SchemaHolder."); + } + + @Override + public void onKeyspaceCreated(@Nonnull final KeyspaceMetadata keyspaceMetadata) { + try { + org.apache.cassandra.schema.KeyspaceMetadata existingKMD = Schema.instance.getKeyspaceMetadata(keyspaceMetadata.getName().asInternal()); + if (existingKMD != null) { + return; + } + + org.apache.cassandra.schema.KeyspaceMetadata newKMD = org.apache.cassandra.schema.KeyspaceMetadata.create( + keyspaceMetadata.getName().toString(), + KeyspaceParams.create(keyspaceMetadata.isDurableWrites(), + keyspaceMetadata.getReplication())); + + Schema.instance.updateHandler.apply(schema -> schema.withAddedOrUpdated(newKMD), true); + Keyspace.openWithoutSSTables(keyspaceMetadata.getName().toString()); + LOGGER.info("Added keyspace [{}] to schema instance.", keyspaceMetadata.describe(true)); + } + catch (Exception e) { + LOGGER.warn("Error happened while adding the keyspace {} to schema instance.", keyspaceMetadata.getName(), e); + } + } + + @Override + public void onKeyspaceUpdated(@Nonnull final KeyspaceMetadata current, @Nonnull final KeyspaceMetadata previous) { + try { + org.apache.cassandra.schema.KeyspaceMetadata keyspaceMetadata = org.apache.cassandra.schema.KeyspaceMetadata.create(current.getName().asInternal(), + KeyspaceParams.create(current.isDurableWrites(), + current.getReplication())); + + Schema.instance.updateHandler.apply(schema -> schema.withAddedOrUpdated(keyspaceMetadata), true); + LOGGER.info("Updated keyspace [{}] in schema instance.", current.describe(true)); + } + catch (Exception e) { + LOGGER.warn("Error happened while updating the keyspace {} in schema instance.", current.getName(), e); + } + } + + @Override + public void onKeyspaceDropped(@Nonnull final KeyspaceMetadata keyspaceMetadata) { + try { + for (Map.Entry entries : keyspaceMetadata.getTables().entrySet()) { + onTableDropped(entries.getValue()); + } + Schema.instance.updateHandler.apply(schema -> schema.without(keyspaceMetadata.getName().toString()), true); + LOGGER.info("Removed keyspace [{}] from schema instance.", keyspaceMetadata.describe(true)); + } + catch (Exception e) { + LOGGER.warn("Error happened while removing the keyspace {} from schema instance.", keyspaceMetadata.getName(), e); + } + } + + @Override + public void onTableCreated(com.datastax.oss.driver.api.core.metadata.schema.TableMetadata tableMetadata) { + Object cdc = tableMetadata.getOptions().get(CqlIdentifier.fromInternal("cdc")); + boolean cdcEnabled = cdc.toString().equals("true"); + + if (cdcEnabled) { + schemaHolder.addOrUpdateTableSchema(new KeyspaceTable(tableMetadata), getKeyValueSchema(tableMetadata)); + } + try { + LOGGER.info("Table {}.{} detected to be added!", tableMetadata.getKeyspace(), tableMetadata.getName()); + + Optional optionalId = tableMetadata.getId(); + assert optionalId.isPresent(); + + UUID uuid = optionalId.get(); + + org.apache.cassandra.schema.TableMetadata metadata = CreateTableStatement.parse(tableMetadata.describe(true), + tableMetadata.getKeyspace().toString()) + .id(TableId.fromUUID(uuid)) + .build(); + + final Keyspace keyspace = Keyspace.openWithoutSSTables(tableMetadata.getKeyspace().asInternal()); + if (keyspace.hasColumnFamilyStore(metadata.id)) { + return; + } + keyspace.initCfCustom(ColumnFamilyStore.createColumnFamilyStore(keyspace, + metadata.name, + TableMetadataRef.forOfflineTools(metadata), + new Directories(metadata), + false, + false, + true)); + + final org.apache.cassandra.schema.KeyspaceMetadata current = Schema.instance.getKeyspaceMetadata(metadata.keyspace); + if (current == null) { + LOGGER.warn("Keyspace {} doesn't exist", metadata.keyspace); + return; + } + if (current.tables.get(tableMetadata.getName().toString()).isPresent()) { + LOGGER.debug("Table {}.{} is already added!", tableMetadata.getKeyspace(), tableMetadata.getName()); + return; + } + + Schema.instance.updateHandler.apply(schema -> schema.withAddedOrUpdated(current.withSwapped(current.tables.with(metadata))), true); + LOGGER.info("Added table [{}] to schema instance.", tableMetadata.describe(true)); + } + catch (Exception e) { + LOGGER.warn("Error happened while adding table {}.{} to schema instance.", tableMetadata.getKeyspace(), tableMetadata.getName(), e); + } + } + + @Override + public void onTableDropped(com.datastax.oss.driver.api.core.metadata.schema.TableMetadata tableMetadata) { + Object cdc = tableMetadata.getOptions().get(CqlIdentifier.fromInternal("cdc")); + boolean cdcEnabled = cdc.toString().equals("true"); + + if (cdcEnabled) { + schemaHolder.removeTableSchema(new KeyspaceTable(tableMetadata)); + } + + try { + final String ksName = tableMetadata.getKeyspace().toString(); + final String tableName = tableMetadata.getName().toString(); + LOGGER.debug("Table {}.{} detected to be removed!", ksName, tableName); + final org.apache.cassandra.schema.KeyspaceMetadata oldKsm = Schema.instance.getKeyspaceMetadata(ksName); + if (oldKsm == null) { + LOGGER.warn("KeyspaceMetadata for keyspace {} is not found!", ksName); + return; + } + + Optional optionalId = tableMetadata.getId(); + assert optionalId.isPresent(); + + TableMetadata metadata = Schema.instance.getTableMetadata(TableId.fromUUID(optionalId.get())); + if (metadata == null) { + LOGGER.warn("Metadata for ColumnFamilyStore for {}.{} is not found!", ksName, tableName); + return; + } + + Keyspace instance = Schema.instance.getKeyspaceInstance(metadata.keyspace); + if (instance == null) { + LOGGER.warn("Keyspace instance for ColumnFamilyStore for {}.{} is not found!", ksName, tableName); + return; + } + + final ColumnFamilyStore cfs = instance.hasColumnFamilyStore(metadata.id) + ? instance.getColumnFamilyStore(metadata.id) + : null; + + if (cfs == null) { + LOGGER.warn("ColumnFamilyStore for {}.{} is not found!", ksName, tableName); + return; + } + // make sure all the indexes are dropped, or else. + cfs.indexManager.markAllIndexesRemoved(); + // reinitialize the keyspace. + final Optional cfm = oldKsm.tables.get(tableName); + + if (cfm.isPresent()) { + Schema.instance.updateHandler.apply(schema -> schema.withAddedOrUpdated(oldKsm.withSwapped(oldKsm.tables.without(tableName))), true); + LOGGER.info("Removed table [{}] from schema instance.", tableMetadata.describe(true)); + } + else { + LOGGER.warn("Table {}.{} is not present in old keyspace meta data!", ksName, tableName); + } + + } + catch (Exception e) { + LOGGER.warn("Error happened while removing table {}.{} from schema instance.", tableMetadata.getKeyspace(), tableMetadata.getName(), e); + } + } + + @Override + public void onTableUpdated(final com.datastax.oss.driver.api.core.metadata.schema.TableMetadata newTableMetadata, + final com.datastax.oss.driver.api.core.metadata.schema.TableMetadata oldTableMetaData) { + Object newCdcObject = newTableMetadata.getOptions().get(CqlIdentifier.fromInternal("cdc")); + boolean newCdc = newCdcObject.toString().equals("true"); + Object oldCdcObject = oldTableMetaData.getOptions().get(CqlIdentifier.fromInternal("cdc")); + boolean oldCdc = oldCdcObject.toString().equals("true"); + + if (newCdc) { + // if it was cdc before and now it is too, add it, because its schema might change + // however if it is CDC-enabled but it was not, update it in schema too because its cdc flag has changed + // this basically means we add / update every time if new has cdc flag equals to true + schemaHolder.addOrUpdateTableSchema(new KeyspaceTable(newTableMetadata), getKeyValueSchema(newTableMetadata)); + } + else if (oldCdc) { + // if new table is not on cdc anymore, and we see the old one was, remove it + schemaHolder.removeTableSchema(new KeyspaceTable(newTableMetadata)); + } + try { + LOGGER.debug("Detected alternation in schema of {}.{} (previous cdc = {}, current cdc = {})", + newTableMetadata.getKeyspace(), + newTableMetadata.getName(), + oldCdc, + newCdc); + // else if it was not cdc before nor now, do nothing with schema holder + // but add it to Cassandra for subsequent deserialization path in every case + // we need to use the id of the existing table to correctly replace it + + Optional optionalId = oldTableMetaData.getId(); + assert optionalId.isPresent(); + + org.apache.cassandra.schema.TableMetadata metadata = CreateTableStatement.parse(newTableMetadata.describe(true), + newTableMetadata.getKeyspace().toString()) + .id(TableId.fromUUID(optionalId.get())) + .build(); + + org.apache.cassandra.schema.KeyspaceMetadata current = Schema.instance.getKeyspaceMetadata(metadata.keyspace); + if (current != null) { + Schema.instance.updateHandler.apply(schema -> schema.withAddedOrUpdated(current.withSwapped(current.tables.withSwapped(metadata))), true); + } + LOGGER.info("Updated table [{}] in schema instance.", newTableMetadata.describe(true)); + } + catch (Exception e) { + LOGGER.warn("Error happened while reacting on changed table {}.{} in schema instance.", newTableMetadata.getKeyspace(), newTableMetadata.getName(), e); + } + } +} 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 new file mode 100644 index 00000000..9e144259 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/Cassandra5TypeProvider.java @@ -0,0 +1,119 @@ +/* + * 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.transforms.CassandraTypeKafkaSchemaBuilders.BOOLEAN_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.BYTES_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.BYTE_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.DATE_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.DOUBLE_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.FLOAT_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.INT_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.LONG_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.SHORT_TYPE; +import static io.debezium.connector.cassandra.transforms.CassandraTypeKafkaSchemaBuilders.STRING_TYPE; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collections; +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; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.CounterColumnType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.SimpleDateType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimeUUIDType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.DurationTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.ListTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.MapTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.SetTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.TupleTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.UserDefinedTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.BasicTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.DecimalTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.InetAddressDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.TimeUUIDTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.TimestampTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.UUIDTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.VarIntTypeDeserializer; + +public class Cassandra5TypeProvider implements CassandraTypeProvider { + + @Override + public List deserializers() { + DebeziumTypeDeserializer deserializer = new DebeziumTypeDeserializer() { + @Override + public Object deserialize(Object abstractType, ByteBuffer bb) { + return ((AbstractType) abstractType).getSerializer().deserialize(bb); + } + }; + List deserializers = Arrays.asList( + // Basic Types + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.BOOLEAN, BooleanType.instance, BOOLEAN_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.VARCHAR, UTF8Type.instance, STRING_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.ASCII, AsciiType.instance, STRING_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.TINYINT, ByteType.instance, BYTE_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.BLOB, BytesType.instance, BYTES_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.FLOAT, FloatType.instance, FLOAT_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.DOUBLE, DoubleType.instance, DOUBLE_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.INT, Int32Type.instance, INT_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.SMALLINT, ShortType.instance, SHORT_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.BIGINT, LongType.instance, LONG_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.TIME, TimeType.instance, LONG_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.COUNTER, CounterColumnType.instance, LONG_TYPE), + new BasicTypeDeserializer(deserializer, ProtocolConstants.DataType.DATE, SimpleDateType.instance, DATE_TYPE), + // Logical Types + new InetAddressDeserializer(deserializer, InetAddressType.instance), + new TimestampTypeDeserializer(deserializer, TimestampType.instance), + new UUIDTypeDeserializer(deserializer, UUIDType.instance), + new TimeUUIDTypeDeserializer(deserializer, TimeUUIDType.instance), + new DecimalTypeDeserializer(deserializer, DecimalType.instance), + new VarIntTypeDeserializer(deserializer, IntegerType.instance), + new DurationTypeDeserializer(deserializer), + // Collection Types + new ListTypeDeserializer(deserializer), + new SetTypeDeserializer(deserializer), + new MapTypeDeserializer(deserializer), + // Struct Types + new TupleTypeDeserializer(deserializer), + new UserDefinedTypeDeserializer(deserializer)); + return Collections.unmodifiableList(deserializers); + } + + @Override + 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/main/java/io/debezium/connector/cassandra/CassandraConnectorTask.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTask.java new file mode 100644 index 00000000..8c8d7761 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/CassandraConnectorTask.java @@ -0,0 +1,56 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.cassandra; + +import java.io.File; + +import org.apache.cassandra.config.DatabaseDescriptor; +import org.apache.cassandra.schema.Schema; + +/** + * A task that reads Cassandra commit log in CDC directory and generate corresponding data + * change events which will be emitted to Kafka. If the table has not been bootstrapped, + * this task will also take a snapshot of existing data in the database and convert each row + * into a change event as well. + */ +public class CassandraConnectorTask { + public static class Cassandra5SchemaLoader implements SchemaLoader { + @Override + public void load(String cassandraYaml) { + cassandraYaml = cassandraYaml.startsWith("/") ? cassandraYaml : "/" + cassandraYaml; + System.setProperty("cassandra.config", "file://" + cassandraYaml); + if (!DatabaseDescriptor.isDaemonInitialized() && !DatabaseDescriptor.isToolInitialized()) { + DatabaseDescriptor.toolInitialization(); + Schema.instance.updateHandler.reset(true); + } + } + } + + public static class Cassandra5SchemaChangeListenerProvider implements SchemaChangeListenerProvider { + @Override + public AbstractSchemaChangeListener provide(CassandraConnectorConfig config) { + return new Cassandra5SchemaChangeListener(config.getLogicalName(), + config.getSourceInfoStructMaker(), + new SchemaHolder()); + } + } + + public static void main(String[] args) throws Exception { + CassandraConnectorTaskTemplate.main(args, config -> init(config, new ComponentFactoryStandalone())); + } + + static CassandraConnectorTaskTemplate init(CassandraConnectorConfig config, ComponentFactory factory) { + CommitLogProcessorMetrics metrics = new CommitLogProcessorMetrics(); + return new CassandraConnectorTaskTemplate(config, + new Cassandra5TypeProvider(), + new Cassandra5SchemaLoader(), + new Cassandra5SchemaChangeListenerProvider(), + context -> new AbstractProcessor[]{ new CommitLogIdxProcessor(context, metrics, + new Cassandra5CommitLogSegmentReader(context, metrics), + new File(DatabaseDescriptor.getCDCLogLocation())) }, + factory); + } +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/DurationTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/DurationTypeDeserializer.java new file mode 100644 index 00000000..d44acd75 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/DurationTypeDeserializer.java @@ -0,0 +1,32 @@ +/* + * 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.transforms.type; + +import org.apache.cassandra.cql3.Duration; +import org.apache.cassandra.db.marshal.DurationType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractDurationTypeDeserializer; +import io.debezium.time.NanoDuration; + +public class DurationTypeDeserializer extends AbstractDurationTypeDeserializer { + + public DurationTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.DURATION, DurationType.instance); + } + + @Override + public Object formatDeserializedValue(Object abstractType, Object value) { + Duration duration = (Duration) value; + int months = duration.getMonths(); + int days = duration.getDays(); + long nanoSec = duration.getNanoseconds(); + return NanoDuration.durationNanos(0, months, days, 0, 0, 0, nanoSec); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/ListTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/ListTypeDeserializer.java new file mode 100644 index 00000000..34d521dc --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/ListTypeDeserializer.java @@ -0,0 +1,32 @@ +/* + * 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.transforms.type; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ListType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractListTypeDeserializer; + +public class ListTypeDeserializer extends AbstractListTypeDeserializer { + + public ListTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.LIST, ListType.class); + } + + @Override + protected Object getElementsType(Object abstractType) { + return ((ListType) abstractType).getElementsType(); + } + + @Override + protected Object getAbstractTypeInstance(Object innerAbstractType, boolean isMultiCell) { + return ListType.getInstance((AbstractType) innerAbstractType, isMultiCell); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/MapTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/MapTypeDeserializer.java new file mode 100644 index 00000000..9f5f8bbf --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/MapTypeDeserializer.java @@ -0,0 +1,38 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.cassandra.transforms.type; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.MapType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractMapTypeDeserializer; + +public class MapTypeDeserializer extends AbstractMapTypeDeserializer { + + public MapTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.MAP, MapType.class); + } + + @Override + protected Object getKeysType(Object abstractType) { + return ((MapType) abstractType).getKeysType(); + } + + @Override + protected Object getValuesType(Object abstractType) { + return ((MapType) abstractType).getValuesType(); + } + + @Override + protected Object getAbstractTypeInstance(Object innerKeyAbstractType, Object innerValueAbstractType, + boolean isMultiCell) { + return MapType.getInstance((AbstractType) innerKeyAbstractType, (AbstractType) innerValueAbstractType, isMultiCell); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/SetTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/SetTypeDeserializer.java new file mode 100644 index 00000000..b6abe0bd --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/SetTypeDeserializer.java @@ -0,0 +1,32 @@ +/* + * 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.transforms.type; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.SetType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractSetTypeDeserializer; + +public class SetTypeDeserializer extends AbstractSetTypeDeserializer { + + public SetTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.SET, SetType.class); + } + + @Override + protected Object getElementsType(Object abstractType) { + return ((SetType) abstractType).getElementsType(); + } + + @Override + protected Object getAbstractTypeInstance(Object innerAbstractType, boolean isMultiCell) { + return SetType.getInstance((AbstractType) innerAbstractType, isMultiCell); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/TupleTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/TupleTypeDeserializer.java new file mode 100644 index 00000000..62d68f76 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/TupleTypeDeserializer.java @@ -0,0 +1,42 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.cassandra.transforms.type; + +import java.nio.ByteBuffer; +import java.util.List; + +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.ByteBufferAccessor; +import org.apache.cassandra.db.marshal.TupleType; + +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractTupleTypeDeserializer; + +public class TupleTypeDeserializer extends AbstractTupleTypeDeserializer { + + public TupleTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.TUPLE, TupleType.class); + } + + @Override + protected List allTypes(Object abstractType) { + return ((TupleType) abstractType).allTypes(); + } + + @Override + protected ByteBuffer[] split(Object abstractType, ByteBuffer bb) { + return ((TupleType) abstractType).split(ByteBufferAccessor.instance, bb); + } + + @Override + @SuppressWarnings("unchecked") + protected Object getAbstractTypeInstance(List innerAbstractTypes) { + return new TupleType((List>) innerAbstractTypes); + } + +} diff --git a/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/UserDefinedTypeDeserializer.java b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/UserDefinedTypeDeserializer.java new file mode 100644 index 00000000..6763f6c8 --- /dev/null +++ b/cassandra-5/src/main/java/io/debezium/connector/cassandra/transforms/type/UserDefinedTypeDeserializer.java @@ -0,0 +1,71 @@ +/* + * 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.transforms.type; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +import org.apache.cassandra.cql3.FieldIdentifier; +import org.apache.cassandra.cql3.UserTypes; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UserType; + +import com.datastax.oss.driver.internal.core.type.DefaultUserDefinedType; +import com.datastax.oss.protocol.internal.ProtocolConstants; + +import io.debezium.connector.cassandra.transforms.DebeziumTypeDeserializer; +import io.debezium.connector.cassandra.transforms.type.deserializer.AbstractUserDefinedTypeDeserializer; + +public class UserDefinedTypeDeserializer extends AbstractUserDefinedTypeDeserializer { + + public UserDefinedTypeDeserializer(DebeziumTypeDeserializer deserializer) { + super(deserializer, ProtocolConstants.DataType.UDT, UserType.class); + } + + @Override + protected List fieldNames(Object abstractType) { + UserType userType = (UserType) abstractType; + return userType.fieldNames().stream().map(FieldIdentifier::toString).collect(Collectors.toList()); + } + + @Override + protected List fieldTypes(Object abstractType) { + UserType userType = (UserType) abstractType; + return userType.fieldTypes(); + } + + @Override + protected List bbList(Object abstractType, ByteBuffer bb) { + UserType userType = (UserType) abstractType; + UserTypes.Value value = UserTypes.Value.fromSerialized(bb, userType); + return value.getElements(); + } + + @Override + protected String structName(Object abstractType) { + UserType userType = (UserType) abstractType; + return userType.keyspace + "." + userType.getNameAsString(); + } + + @Override + @SuppressWarnings("unchecked") + protected Object getAbstractTypeInstance(DefaultUserDefinedType userType, List fieldNames, + List innerAbstractTypes) { + ByteBuffer typeNameBuffer = UTF8Type.instance.fromString(userType.getName().toString()); + List fieldIdentifiers = new ArrayList<>(fieldNames.size()); + for (String fieldName : fieldNames) { + fieldIdentifiers.add(FieldIdentifier.forInternalString(fieldName)); + } + return new UserType(userType.getKeyspace().toString(), + typeNameBuffer, + fieldIdentifiers, + (List>) innerAbstractTypes, + !userType.isFrozen()); + } +} diff --git a/cassandra-5/src/main/resources/META-INF/services/org.apache.kafka.connect.source.SourceConnector b/cassandra-5/src/main/resources/META-INF/services/org.apache.kafka.connect.source.SourceConnector new file mode 100644 index 00000000..c700c75a --- /dev/null +++ b/cassandra-5/src/main/resources/META-INF/services/org.apache.kafka.connect.source.SourceConnector @@ -0,0 +1 @@ +io.debezium.connector.cassandra.Cassandra5Connector \ No newline at end of file diff --git a/cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java new file mode 100644 index 00000000..41a9da65 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/AbstractCommitLogProcessorTest.java @@ -0,0 +1,115 @@ +/* + * 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 EmbeddedCassandra5ConnectorTestBase { + 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 Cassandra5CommitLogReadHandlerImpl(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, new org.apache.cassandra.io.util.File(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-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java new file mode 100644 index 00000000..7a09259e --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/BatchTest.java @@ -0,0 +1,97 @@ +/* + * 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/ClusteringPartitionKeyCommitLogProcessingTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java new file mode 100644 index 00000000..d720f720 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/ClusteringPartitionKeyCommitLogProcessingTest.java @@ -0,0 +1,58 @@ +/* + * 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-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java new file mode 100644 index 00000000..8b9b3359 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/CommitLogRealTimeParserTest.java @@ -0,0 +1,109 @@ +/* + * 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 Cassandra5CommitLogSegmentReader(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-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java new file mode 100644 index 00000000..ebf814aa --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/EmbeddedCassandra5ConnectorTestBase.java @@ -0,0 +1,28 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.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 new file mode 100644 index 00000000..bb24ec15 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/MultipleTablesProcessingTest.java @@ -0,0 +1,107 @@ +/* + * 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 new file mode 100644 index 00000000..6ba7edfa --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/PartitionKeyRowDeletionCommitLogProcessingTest.java @@ -0,0 +1,47 @@ +/* + * 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 new file mode 100644 index 00000000..c20c8ecb --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/QueueProcessorTest.java @@ -0,0 +1,22 @@ +/* + * 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 new file mode 100644 index 00000000..835dad49 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/RangeTombstoneCommitLogProcessingTest.java @@ -0,0 +1,96 @@ +/* + * 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/RowInsertionModificationCommitLogProcessingTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java new file mode 100644 index 00000000..ad93e843 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/RowInsertionModificationCommitLogProcessingTest.java @@ -0,0 +1,51 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.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-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java new file mode 100644 index 00000000..88f730a7 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/SchemaChangeListenerTest.java @@ -0,0 +1,63 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package io.debezium.connector.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 new file mode 100644 index 00000000..c1d1d85c --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/SnapshotProcessorTest.java @@ -0,0 +1,177 @@ +/* + * 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/java/io/debezium/connector/cassandra/transforms/CassandraTypeDeserializerTest.java b/cassandra-5/src/test/java/io/debezium/connector/cassandra/transforms/CassandraTypeDeserializerTest.java new file mode 100644 index 00000000..b59f3e97 --- /dev/null +++ b/cassandra-5/src/test/java/io/debezium/connector/cassandra/transforms/CassandraTypeDeserializerTest.java @@ -0,0 +1,662 @@ +/* + * 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.transforms; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.nio.ByteBuffer; +import java.nio.charset.Charset; +import java.time.Instant; +import java.time.temporal.ChronoUnit; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Date; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; + +import org.apache.cassandra.cql3.Duration; +import org.apache.cassandra.cql3.FieldIdentifier; +import org.apache.cassandra.cql3.QueryOptions; +import org.apache.cassandra.cql3.Term; +import org.apache.cassandra.db.marshal.AbstractType; +import org.apache.cassandra.db.marshal.AsciiType; +import org.apache.cassandra.db.marshal.BooleanType; +import org.apache.cassandra.db.marshal.ByteType; +import org.apache.cassandra.db.marshal.BytesType; +import org.apache.cassandra.db.marshal.CounterColumnType; +import org.apache.cassandra.db.marshal.DecimalType; +import org.apache.cassandra.db.marshal.DoubleType; +import org.apache.cassandra.db.marshal.DurationType; +import org.apache.cassandra.db.marshal.FloatType; +import org.apache.cassandra.db.marshal.InetAddressType; +import org.apache.cassandra.db.marshal.Int32Type; +import org.apache.cassandra.db.marshal.IntegerType; +import org.apache.cassandra.db.marshal.ListType; +import org.apache.cassandra.db.marshal.LongType; +import org.apache.cassandra.db.marshal.MapType; +import org.apache.cassandra.db.marshal.ReversedType; +import org.apache.cassandra.db.marshal.SetType; +import org.apache.cassandra.db.marshal.ShortType; +import org.apache.cassandra.db.marshal.SimpleDateType; +import org.apache.cassandra.db.marshal.TimeType; +import org.apache.cassandra.db.marshal.TimeUUIDType; +import org.apache.cassandra.db.marshal.TimestampType; +import org.apache.cassandra.db.marshal.TupleType; +import org.apache.cassandra.db.marshal.UTF8Type; +import org.apache.cassandra.db.marshal.UUIDType; +import org.apache.cassandra.db.marshal.UserType; +import org.apache.cassandra.utils.TimeUUID; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +import com.datastax.oss.driver.api.core.CqlIdentifier; +import com.datastax.oss.driver.api.core.type.DataTypes; +import com.datastax.oss.driver.internal.core.type.DefaultUserDefinedType; + +import io.debezium.connector.cassandra.Cassandra5TypeProvider; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.DecimalMode; +import io.debezium.connector.cassandra.transforms.CassandraTypeDeserializer.VarIntMode; + +public class CassandraTypeDeserializerTest { + + private static final Charset CHARSET = Charset.forName("UTF-8"); + + @BeforeClass + public static void beforeAll() { + Cassandra5TypeProvider provider = new Cassandra5TypeProvider(); + CassandraTypeDeserializer.init(provider.deserializers(), DecimalMode.DOUBLE, VarIntMode.LONG, + provider.baseTypeForReversedType()); + } + + @Test + public void testAscii() { + String expectedAscii = "some text"; + + ByteBuffer serializedAscii = AsciiType.instance.decompose(expectedAscii); + + Object deserializedAscii = CassandraTypeDeserializer.deserialize(AsciiType.instance, serializedAscii); + Assert.assertEquals("some text", deserializedAscii); + + deserializedAscii = CassandraTypeDeserializer.deserialize(DataTypes.ASCII, serializedAscii); + Assert.assertEquals("some text", deserializedAscii); + } + + @Test + public void testBoolean() { + Boolean expectedBoolean = true; + + ByteBuffer serializedBoolean = BooleanType.instance.decompose(expectedBoolean); + + Object deserializedBoolean = CassandraTypeDeserializer.deserialize(BooleanType.instance, serializedBoolean); + Assert.assertEquals(expectedBoolean, deserializedBoolean); + + deserializedBoolean = CassandraTypeDeserializer.deserialize(DataTypes.BOOLEAN, serializedBoolean); + Assert.assertEquals(expectedBoolean, deserializedBoolean); + } + + @Test + public void testBytes() { + ByteBuffer expectedBytes = ByteBuffer.wrap("some random stuff here".getBytes(CHARSET)); + + // Pretty sure this is a no-op, but for consistency... + ByteBuffer serializedBytes = BytesType.instance.decompose(expectedBytes); + + Object deserializedBytes = CassandraTypeDeserializer.deserialize(BytesType.instance, serializedBytes); + Assert.assertEquals(expectedBytes, deserializedBytes); + + deserializedBytes = CassandraTypeDeserializer.deserialize(DataTypes.BLOB, serializedBytes); + Assert.assertEquals(expectedBytes, deserializedBytes); + } + + @Test + public void testByteType() { + Byte expectedByte = Byte.valueOf("9"); + + ByteBuffer serializedByte = ByteType.instance.decompose(expectedByte); + + Object deserializedByte = CassandraTypeDeserializer.deserialize(ByteType.instance, serializedByte); + Assert.assertEquals(expectedByte, deserializedByte); + + deserializedByte = CassandraTypeDeserializer.deserialize(DataTypes.TINYINT, serializedByte); + Assert.assertEquals(expectedByte, deserializedByte); + } + + @Test + public void testCounterColumnType() { + Long expectedCounterColumnType = 42L; + + ByteBuffer serializedCounter = CounterColumnType.instance.decompose(42L); + + Object deserializedCounter = CassandraTypeDeserializer.deserialize(CounterColumnType.instance, serializedCounter); + Assert.assertEquals(expectedCounterColumnType, deserializedCounter); + + deserializedCounter = CassandraTypeDeserializer.deserialize(DataTypes.COUNTER, serializedCounter); + Assert.assertEquals(expectedCounterColumnType, deserializedCounter); + } + + @Test + public void testDecimalType() { + BigDecimal expectedDecimal = BigDecimal.valueOf(Math.PI); + + ByteBuffer serializedDecimal = DecimalType.instance.decompose(expectedDecimal); + + // decimal.handling.mode = DOUBLE (default) + Object deserializedDecimalAsDouble = CassandraTypeDeserializer.deserialize(DecimalType.instance, serializedDecimal); + Assert.assertEquals(expectedDecimal.doubleValue(), deserializedDecimalAsDouble); + + deserializedDecimalAsDouble = CassandraTypeDeserializer.deserialize(DataTypes.DECIMAL, serializedDecimal); + Assert.assertEquals(expectedDecimal.doubleValue(), deserializedDecimalAsDouble); + + // decimal.handling.mode = PRECISE + CassandraTypeDeserializer.setDecimalMode(DecimalMode.PRECISE); + Object deserializedDecimalAsStruct = CassandraTypeDeserializer.deserialize(DecimalType.instance, serializedDecimal); + Schema decimalSchema = CassandraTypeDeserializer.getSchemaBuilder(DecimalType.instance).build(); + Struct expectedDecimalStruct = new Struct(decimalSchema) + .put("value", expectedDecimal.unscaledValue().toByteArray()) + .put("scale", expectedDecimal.scale()); + Assert.assertEquals(expectedDecimalStruct, deserializedDecimalAsStruct); + + deserializedDecimalAsStruct = CassandraTypeDeserializer.deserialize(DataTypes.DECIMAL, serializedDecimal); + Assert.assertEquals(expectedDecimalStruct, deserializedDecimalAsStruct); + + // decimal.handling.mode = STRING + CassandraTypeDeserializer.setDecimalMode(DecimalMode.STRING); + Object deserializedDecimalAsString = CassandraTypeDeserializer.deserialize(DecimalType.instance, serializedDecimal); + Assert.assertEquals(expectedDecimal.toPlainString(), deserializedDecimalAsString); + + deserializedDecimalAsString = CassandraTypeDeserializer.deserialize(DataTypes.DECIMAL, serializedDecimal); + Assert.assertEquals(expectedDecimal.toPlainString(), deserializedDecimalAsString); + } + + @Test + public void testDoubleType() { + Double expectedDouble = 100.5; + + ByteBuffer serializedDouble = DoubleType.instance.decompose(expectedDouble); + + Object deserializedDouble = CassandraTypeDeserializer.deserialize(DoubleType.instance, serializedDouble); + Assert.assertEquals(expectedDouble, deserializedDouble); + + deserializedDouble = CassandraTypeDeserializer.deserialize(DataTypes.DOUBLE, serializedDouble); + Assert.assertEquals(expectedDouble, deserializedDouble); + } + + @Test + public void testDurationType() { + Duration sourceDuration = Duration.newInstance(1, 3, 500); + + long expectedNanoDuration = (30 + 3) * ChronoUnit.DAYS.getDuration().toNanos() + 500; + + ByteBuffer serializedDuration = DurationType.instance.decompose(sourceDuration); + + Object deserializedDuration = CassandraTypeDeserializer.deserialize(DurationType.instance, serializedDuration); + Assert.assertEquals(expectedNanoDuration, deserializedDuration); + + deserializedDuration = CassandraTypeDeserializer.deserialize(DataTypes.DURATION, serializedDuration); + Assert.assertEquals(expectedNanoDuration, deserializedDuration); + } + + @Test + public void testFloatType() { + Float expectedFloat = 66.6F; + + ByteBuffer serializedFloat = FloatType.instance.decompose(expectedFloat); + + Object deserializedFloat = CassandraTypeDeserializer.deserialize(FloatType.instance, serializedFloat); + Assert.assertEquals(expectedFloat, deserializedFloat); + + deserializedFloat = CassandraTypeDeserializer.deserialize(DataTypes.FLOAT, serializedFloat); + Assert.assertEquals(expectedFloat, deserializedFloat); + } + + @Test + public void testInetAddressType() throws UnknownHostException { + InetAddress sourceInetAddress = InetAddress.getLocalHost(); + // the address is the only thing that cassandra will seralize for an inetadress. + String expectedInetAddress = "/" + sourceInetAddress.getHostAddress(); + + ByteBuffer serializedInetAddress = InetAddressType.instance.decompose(sourceInetAddress); + + Object deserializedInetAddress = CassandraTypeDeserializer.deserialize(InetAddressType.instance, serializedInetAddress); + Assert.assertEquals(expectedInetAddress, deserializedInetAddress); + + deserializedInetAddress = CassandraTypeDeserializer.deserialize(DataTypes.INET, serializedInetAddress); + Assert.assertEquals(expectedInetAddress, deserializedInetAddress); + } + + @Test + public void testInt32Type() { + Integer expectedInteger = 8; + + ByteBuffer serializedInt32 = Int32Type.instance.decompose(expectedInteger); + + Object deserializedInt32 = CassandraTypeDeserializer.deserialize(Int32Type.instance, serializedInt32); + Assert.assertEquals(expectedInteger, deserializedInt32); + + deserializedInt32 = CassandraTypeDeserializer.deserialize(DataTypes.INT, serializedInt32); + Assert.assertEquals(expectedInteger, deserializedInt32); + } + + @Test + public void testIntegerType() { + BigInteger expectedInteger = BigInteger.valueOf(8); + + ByteBuffer serializedVarInt = IntegerType.instance.decompose(expectedInteger); + + // varint.handling.mode = LONG (default) + Object deserializedVarIntAsLong = CassandraTypeDeserializer.deserialize(IntegerType.instance, serializedVarInt); + Assert.assertEquals(expectedInteger.longValue(), deserializedVarIntAsLong); + + deserializedVarIntAsLong = CassandraTypeDeserializer.deserialize(DataTypes.VARINT, serializedVarInt); + Assert.assertEquals(expectedInteger.longValue(), deserializedVarIntAsLong); + + // varint.handling.mode = PRECISE + CassandraTypeDeserializer.setVarIntMode(VarIntMode.PRECISE); + Object deserializedVarIntAsBigDecimal = CassandraTypeDeserializer.deserialize(IntegerType.instance, serializedVarInt); + Assert.assertEquals(new BigDecimal(expectedInteger), deserializedVarIntAsBigDecimal); + + deserializedVarIntAsBigDecimal = CassandraTypeDeserializer.deserialize(DataTypes.VARINT, serializedVarInt); + Assert.assertEquals(new BigDecimal(expectedInteger), deserializedVarIntAsBigDecimal); + + // varint.handling.mode = STRING + CassandraTypeDeserializer.setVarIntMode(VarIntMode.STRING); + Object deserializedVarIntAsString = CassandraTypeDeserializer.deserialize(IntegerType.instance, serializedVarInt); + Assert.assertEquals(expectedInteger.toString(), deserializedVarIntAsString); + + deserializedVarIntAsString = CassandraTypeDeserializer.deserialize(DataTypes.VARINT, serializedVarInt); + Assert.assertEquals(expectedInteger.toString(), deserializedVarIntAsString); + } + + @Test + public void testListType() { + List expectedList = new ArrayList<>(); + expectedList.add(1); + expectedList.add(3); + expectedList.add(5); + + // non-frozen + ListType nonFrozenListType = ListType.getInstance(Int32Type.instance, true); + ByteBuffer serializedList = nonFrozenListType.decompose(expectedList); + Object deserializedList = CassandraTypeDeserializer.deserialize(nonFrozenListType, serializedList); + Assert.assertEquals(expectedList, deserializedList); + + deserializedList = CassandraTypeDeserializer.deserialize(DataTypes.listOf(DataTypes.INT), serializedList); + Assert.assertEquals(expectedList, deserializedList); + + // frozen + ListType frozenListType = ListType.getInstance(Int32Type.instance, false); + serializedList = frozenListType.decompose(expectedList); + deserializedList = CassandraTypeDeserializer.deserialize(frozenListType, serializedList); + Assert.assertEquals(expectedList, deserializedList); + + deserializedList = CassandraTypeDeserializer.deserialize(DataTypes.frozenListOf(DataTypes.INT), serializedList); + Assert.assertEquals(expectedList, deserializedList); + } + + @Test + public void testLongType() { + Long expectedLong = 8L; + + ByteBuffer serializedLong = LongType.instance.decompose(expectedLong); + + Object deserializedLong = CassandraTypeDeserializer.deserialize(LongType.instance, serializedLong); + Assert.assertEquals(expectedLong, deserializedLong); + + deserializedLong = CassandraTypeDeserializer.deserialize(DataTypes.BIGINT, serializedLong); + Assert.assertEquals(expectedLong, deserializedLong); + } + + @Test + public void testMapType() { + Map expectedMap = new HashMap<>(); + expectedMap.put("foo", 1D); + expectedMap.put("bar", 50D); + + // non-frozen + MapType nonFrozenMapType = MapType.getInstance(AsciiType.instance, DoubleType.instance, true); + ByteBuffer serializedMap = nonFrozenMapType.decompose(expectedMap); + Object deserializedMap = CassandraTypeDeserializer.deserialize(nonFrozenMapType, serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + + deserializedMap = CassandraTypeDeserializer.deserialize(DataTypes.mapOf(DataTypes.ASCII, DataTypes.DOUBLE), serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + + // frozen + MapType frozenMapType = MapType.getInstance(AsciiType.instance, DoubleType.instance, false); + serializedMap = frozenMapType.decompose(expectedMap); + deserializedMap = CassandraTypeDeserializer.deserialize(frozenMapType, serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + + deserializedMap = CassandraTypeDeserializer.deserialize(DataTypes.frozenMapOf(DataTypes.ASCII, DataTypes.DOUBLE), serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + } + + @Test + public void testMapTypeNonStringKeys() { + Map sourceMap = new HashMap<>(); + sourceMap.put(1, 1.5F); + sourceMap.put(2, 3.1414F); + + Map expectedMap = new HashMap<>(); + expectedMap.put(1, 1.5F); + expectedMap.put(2, 3.1414F); + + MapType mapType = MapType.getInstance(Int32Type.instance, FloatType.instance, true); + ByteBuffer serializedMap = mapType.decompose(sourceMap); + Object deserializedMap = CassandraTypeDeserializer.deserialize(mapType, serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + + deserializedMap = CassandraTypeDeserializer.deserialize(DataTypes.mapOf(DataTypes.INT, DataTypes.FLOAT), serializedMap); + Assert.assertEquals(expectedMap, deserializedMap); + } + + @Test + public void testSetType() { + Set sourceSet = new HashSet<>(); + sourceSet.add(42F); + sourceSet.add(123F); + + // non-frozen + SetType nonFrozenSetType = SetType.getInstance(FloatType.instance, true); + ByteBuffer serializedSet = nonFrozenSetType.decompose(sourceSet); + Collection deserializedSet = (Collection) CassandraTypeDeserializer.deserialize(nonFrozenSetType, serializedSet); + // order may be different in the resulting collection. + Assert.assertTrue(sourceSet.containsAll(deserializedSet)); + Assert.assertTrue(deserializedSet.containsAll(sourceSet)); + + deserializedSet = (Collection) CassandraTypeDeserializer.deserialize(DataTypes.setOf(DataTypes.FLOAT), serializedSet); + Assert.assertTrue(sourceSet.containsAll(deserializedSet)); + Assert.assertTrue(deserializedSet.containsAll(sourceSet)); + + // frozen + SetType frozenSetType = SetType.getInstance(FloatType.instance, false); + serializedSet = frozenSetType.decompose(sourceSet); + deserializedSet = (Collection) CassandraTypeDeserializer.deserialize(frozenSetType, serializedSet); + Assert.assertTrue(sourceSet.containsAll(deserializedSet)); + Assert.assertTrue(deserializedSet.containsAll(sourceSet)); + + deserializedSet = (Collection) CassandraTypeDeserializer.deserialize(DataTypes.frozenSetOf(DataTypes.FLOAT), serializedSet); + Assert.assertTrue(sourceSet.containsAll(deserializedSet)); + Assert.assertTrue(deserializedSet.containsAll(sourceSet)); + } + + @Test + public void testShortType() { + Short expectedShort = (short) 2; + + ByteBuffer serializedShort = ShortType.instance.decompose(expectedShort); + + Object deserializedShort = CassandraTypeDeserializer.deserialize(ShortType.instance, serializedShort); + Assert.assertEquals(expectedShort, deserializedShort); + + deserializedShort = CassandraTypeDeserializer.deserialize(DataTypes.SMALLINT, serializedShort); + Assert.assertEquals(expectedShort, deserializedShort); + } + + @Test + public void testSimpleDateType() { + Integer expectedDate = 17953; + + ByteBuffer serializedDate = SimpleDateType.instance.decompose(expectedDate); + + Object deserializedShort = CassandraTypeDeserializer.deserialize(SimpleDateType.instance, serializedDate); + Assert.assertEquals(expectedDate, deserializedShort); + + deserializedShort = CassandraTypeDeserializer.deserialize(DataTypes.DATE, serializedDate); + Assert.assertEquals(expectedDate, deserializedShort); + } + + @Test + public void testTimeType() { + Long expectedTime = 30L; + + ByteBuffer serializedTime = TimeType.instance.decompose(expectedTime); + + Object deserializedTime = CassandraTypeDeserializer.deserialize(TimeType.instance, serializedTime); + Assert.assertEquals(expectedTime, deserializedTime); + + deserializedTime = CassandraTypeDeserializer.deserialize(DataTypes.TIME, serializedTime); + Assert.assertEquals(expectedTime, deserializedTime); + } + + @Test + public void testTimestampType() { + Date timestamp = new Date(); + Long expectedLongTimestamp = timestamp.getTime(); + + ByteBuffer serializedTimestamp = TimestampType.instance.decompose(timestamp); + + Object deserializedTimestamp = CassandraTypeDeserializer.deserialize(TimestampType.instance, serializedTimestamp); + Assert.assertEquals(expectedLongTimestamp, deserializedTimestamp); + + deserializedTimestamp = CassandraTypeDeserializer.deserialize(DataTypes.TIMESTAMP, serializedTimestamp); + Assert.assertEquals(expectedLongTimestamp, deserializedTimestamp); + } + + @Test + public void testTimeUUIDType() { + TimeUUID timeUUID = TimeUUID.maxAtUnixMillis(Instant.now().toEpochMilli()); + ByteBuffer serializedTimeUUID = TimeUUIDType.instance.decompose(timeUUID); + + Object deserializedTimeUUID = CassandraTypeDeserializer.deserialize(TimeUUIDType.instance, serializedTimeUUID); + Assert.assertEquals(timeUUID.toString(), deserializedTimeUUID); + + deserializedTimeUUID = CassandraTypeDeserializer.deserialize(DataTypes.TIMEUUID, serializedTimeUUID); + Assert.assertEquals(timeUUID.toString(), deserializedTimeUUID); + } + + @Test + public void testTupleType() { + List> innerAbstractTypes = new ArrayList<>(2); + innerAbstractTypes.add(AsciiType.instance); + innerAbstractTypes.add(ShortType.instance); + TupleType tupleType = new TupleType(innerAbstractTypes); + + String sourceTupleString = "foo:1"; + ByteBuffer serializedTuple = tupleType.fromString(sourceTupleString); + + Schema tupleSchema = CassandraTypeDeserializer.getSchemaBuilder(tupleType).build(); + Struct expectedTuple = new Struct(tupleSchema) + .put("field1", "foo") + .put("field2", (short) 1); + + Object deserializedTuple = CassandraTypeDeserializer.deserialize(tupleType, serializedTuple); + Assert.assertEquals(expectedTuple, deserializedTuple); + + deserializedTuple = CassandraTypeDeserializer.deserialize(DataTypes.tupleOf(DataTypes.ASCII, DataTypes.SMALLINT), serializedTuple); + Assert.assertEquals(expectedTuple, deserializedTuple); + } + + @Test + public void testUserType() { + // this is slightly complicated, so we're testing in two parts: + // first, explicitly test for schema correctness + ByteBuffer expectedTypeName = ByteBuffer.wrap("FooType".getBytes(Charset.defaultCharset())); + List expectedFieldIdentifiers = new ArrayList<>(); + expectedFieldIdentifiers.add(new FieldIdentifier(ByteBuffer.wrap("asciiField".getBytes(Charset.defaultCharset())))); + expectedFieldIdentifiers.add(new FieldIdentifier(ByteBuffer.wrap("doubleField".getBytes(Charset.defaultCharset())))); + expectedFieldIdentifiers.add(new FieldIdentifier(ByteBuffer.wrap("durationField".getBytes(Charset.defaultCharset())))); + // testing duration to make sure that recursive deserialization works correctly + List> expectedFieldTypes = new ArrayList<>(); + expectedFieldTypes.add(AsciiType.instance); + expectedFieldTypes.add(DoubleType.instance); + expectedFieldTypes.add(DurationType.instance); + UserType userType = new UserType("barspace", + expectedTypeName, + expectedFieldIdentifiers, + expectedFieldTypes, + true); + + Schema userSchema = CassandraTypeDeserializer.getSchemaBuilder(userType).build(); + + long expectedNanoDuration = (30 + 2) * ChronoUnit.DAYS.getDuration().toNanos() + 3; + + Struct expectedUserTypeData = new Struct(userSchema) + .put("asciiField", "foobar") + .put("doubleField", 1.5d) + .put("durationField", expectedNanoDuration); + + Map jsonObject = new HashMap<>(3); + jsonObject.put("\"asciiField\"", "foobar"); + jsonObject.put("\"doubleField\"", 1.5d); + jsonObject.put("\"durationField\"", DurationType.instance.getSerializer().toString(Duration.newInstance(1, 2, 3))); + Term userTypeObject = userType.fromJSONObject(jsonObject); + + ByteBuffer buffer = userTypeObject.bindAndGet(QueryOptions.DEFAULT); + + ByteBuffer serializedUserTypeObject = userType.decompose(buffer); + + Object deserializedUserTypeObject = CassandraTypeDeserializer.deserialize(userType, serializedUserTypeObject); + Assert.assertEquals(expectedUserTypeData, deserializedUserTypeObject); + + DefaultUserDefinedType userDefinedType = new DefaultUserDefinedType(CqlIdentifier.fromCql("\"barspace\""), + CqlIdentifier.fromCql("\"FooType\""), false, + Arrays.asList(CqlIdentifier.fromCql("\"asciiField\""), CqlIdentifier.fromCql("\"doubleField\""), CqlIdentifier.fromCql("\"durationField\"")), + Arrays.asList(DataTypes.ASCII, DataTypes.DOUBLE, DataTypes.DURATION)); + deserializedUserTypeObject = CassandraTypeDeserializer.deserialize(userDefinedType, serializedUserTypeObject); + Assert.assertEquals(expectedUserTypeData, deserializedUserTypeObject); + } + + @Test + public void testUTF8Type() { + String expectedUTF8 = "Fourscore and seven years ago"; + + ByteBuffer serializedUTF8 = UTF8Type.instance.decompose(expectedUTF8); + + Object deserializedUTF8 = CassandraTypeDeserializer.deserialize(UTF8Type.instance, serializedUTF8); + Assert.assertEquals(expectedUTF8, deserializedUTF8); + + deserializedUTF8 = CassandraTypeDeserializer.deserialize(DataTypes.TEXT, serializedUTF8); + Assert.assertEquals(expectedUTF8, deserializedUTF8); + } + + @Test + public void testUUIDType() { + UUID uuid = UUID.randomUUID(); + + String expectedFixedUUID = uuid.toString(); + + ByteBuffer serializedUUID = UUIDType.instance.decompose(uuid); + + Object deserializedUUID = CassandraTypeDeserializer.deserialize(UUIDType.instance, serializedUUID); + Assert.assertEquals(expectedFixedUUID, deserializedUUID); + + deserializedUUID = CassandraTypeDeserializer.deserialize(DataTypes.UUID, serializedUUID); + Assert.assertEquals(expectedFixedUUID, deserializedUUID); + } + + @Test + public void testReversedType() { + Date timestamp = new Date(); + Long expectedLongTimestamp = timestamp.getTime(); + + ByteBuffer serializedTimestamp = TimestampType.instance.decompose(timestamp); + + AbstractType reversedTimeStampType = ReversedType.getInstance(TimestampType.instance); + + Object deserializedTimestamp = CassandraTypeDeserializer.deserialize(reversedTimeStampType, serializedTimestamp); + Assert.assertEquals(expectedLongTimestamp, deserializedTimestamp); + } + + @Test + public void testListUUIDType() { + + List originalList = new ArrayList<>(); + UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); + UUID uuid3 = UUID.randomUUID(); + originalList.add(uuid1); + originalList.add(uuid2); + originalList.add(uuid3); + + List expectedList = new ArrayList<>(); + String expectedUuidStr1 = uuid1.toString(); + String expectedUuidStr2 = uuid2.toString(); + String expectedUuidStr3 = uuid3.toString(); + expectedList.add(expectedUuidStr1); + expectedList.add(expectedUuidStr2); + expectedList.add(expectedUuidStr3); + + ListType frozenListType = ListType.getInstance(UUIDType.instance, false); + ByteBuffer serializedList = frozenListType.decompose(originalList); + Object deserializedList = CassandraTypeDeserializer.deserialize(frozenListType, serializedList); + Assert.assertEquals(expectedList, deserializedList); + + deserializedList = CassandraTypeDeserializer.deserialize(DataTypes.listOf(DataTypes.UUID), serializedList); + Assert.assertEquals(expectedList, deserializedList); + } + + @Test + public void testListUserType() { + + ByteBuffer userTypeName = ByteBuffer.wrap("FooType".getBytes(Charset.defaultCharset())); + List userTypeFieldIdentifiers = new ArrayList<>(); + userTypeFieldIdentifiers.add(new FieldIdentifier(ByteBuffer.wrap("asciiField".getBytes(Charset.defaultCharset())))); + userTypeFieldIdentifiers.add(new FieldIdentifier(ByteBuffer.wrap("setField".getBytes(Charset.defaultCharset())))); + SetType frozenSetType = SetType.getInstance(AsciiType.instance, false); + List> userFieldTypes = new ArrayList<>(); + userFieldTypes.add(AsciiType.instance); + userFieldTypes.add(frozenSetType); + UserType userType = new UserType("barspace", + userTypeName, + userTypeFieldIdentifiers, + userFieldTypes, + false); + + Schema userTypeSchema = CassandraTypeDeserializer.getSchemaBuilder(userType).build(); + Set sourceSet = new HashSet<>(); + sourceSet.add("text1"); + sourceSet.add("text2"); + Struct expectedUserTypeData1 = new Struct(userTypeSchema) + .put("asciiField", "foobar1") + .put("setField", new ArrayList<>(sourceSet)); + Struct expectedUserTypeData2 = new Struct(userTypeSchema) + .put("asciiField", "foobar2") + .put("setField", new ArrayList<>(sourceSet)); + List expectedList = new ArrayList<>(); + expectedList.add(expectedUserTypeData1); + expectedList.add(expectedUserTypeData2); + + Map jsonObject1 = new HashMap<>(2); + jsonObject1.put("\"asciiField\"", "foobar1"); + jsonObject1.put("\"setField\"", new ArrayList<>(sourceSet)); + Term userTypeObject1 = userType.fromJSONObject(jsonObject1); + ByteBuffer buffer1 = userTypeObject1.bindAndGet(QueryOptions.DEFAULT); + ByteBuffer serializedUserTypeObject1 = userType.decompose(buffer1); + Map jsonObject2 = new HashMap<>(2); + jsonObject2.put("\"asciiField\"", "foobar2"); + jsonObject2.put("\"setField\"", new ArrayList<>(sourceSet)); + Term userTypeObject2 = userType.fromJSONObject(jsonObject2); + ByteBuffer buffer2 = userTypeObject2.bindAndGet(QueryOptions.DEFAULT); + ByteBuffer serializedUserTypeObject2 = userType.decompose(buffer2); + List originalList = new ArrayList<>(); + originalList.add(serializedUserTypeObject1); + originalList.add(serializedUserTypeObject2); + + ListType frozenListType = ListType.getInstance(userType, false); + ByteBuffer serializedList = frozenListType.decompose(originalList); + Object deserializedList = CassandraTypeDeserializer.deserialize(frozenListType, serializedList); + Assert.assertEquals(expectedList, deserializedList); + + DefaultUserDefinedType userDefinedType = new DefaultUserDefinedType(CqlIdentifier.fromCql("\"barspace\""), + CqlIdentifier.fromCql("\"FooType\""), true, + Arrays.asList(CqlIdentifier.fromCql("\"asciiField\""), CqlIdentifier.fromCql("\"setField\"")), + Arrays.asList(DataTypes.ASCII, DataTypes.frozenSetOf(DataTypes.ASCII))); + deserializedList = CassandraTypeDeserializer.deserialize(DataTypes.frozenListOf(userDefinedType), serializedList); + Assert.assertEquals(expectedList, deserializedList); + } + +} diff --git a/cassandra-5/src/test/resources/application.conf b/cassandra-5/src/test/resources/application.conf new file mode 100644 index 00000000..db695391 --- /dev/null +++ b/cassandra-5/src/test/resources/application.conf @@ -0,0 +1,5 @@ +datastax-java-driver { + basic.request { + timeout = 20 seconds + } +} \ No newline at end of file diff --git a/cassandra-5/src/test/resources/cassandra-unit-for-context.yaml b/cassandra-5/src/test/resources/cassandra-unit-for-context.yaml new file mode 100644 index 00000000..80e76969 --- /dev/null +++ b/cassandra-5/src/test/resources/cassandra-unit-for-context.yaml @@ -0,0 +1,1434 @@ + +# Cassandra storage config YAML + +# NOTE: +# See https://cassandra.apache.org/doc/latest/configuration/ for +# full explanations of configuration directives +# /NOTE + +# The name of the cluster. This is mainly used to prevent machines in +# one logical cluster from joining another. +cluster_name: 'Test Cluster' + +# This defines the number of tokens randomly assigned to this node on the ring +# The more tokens, relative to other nodes, the larger the proportion of data +# that this node will store. You probably want all nodes to have the same number +# of tokens assuming they have equal hardware capability. +# +# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, +# and will use the initial_token as described below. +# +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. +# +# See https://cassandra.apache.org/doc/latest/getting_started/production.html#tokens for +# best practice information about num_tokens. +# +num_tokens: 16 + +# Triggers automatic allocation of num_tokens tokens for this node. The allocation +# algorithm attempts to choose tokens in a way that optimizes replicated load over +# the nodes in the datacenter for the replica factor. +# +# The load assigned to each node will be close to proportional to its number of +# vnodes. +# +# Only supported with the Murmur3Partitioner. + +# Replica factor is determined via the replication strategy used by the specified +# keyspace. +# allocate_tokens_for_keyspace: KEYSPACE + +# Replica factor is explicitly set, regardless of keyspace or datacenter. +# This is the replica factor within the datacenter, like NTS. +allocate_tokens_for_local_replication_factor: 3 + +# initial_token allows you to specify tokens manually. While you can use it with +# vnodes (num_tokens > 1, above) -- in which case you should provide a +# comma-separated list -- it's primarily used when adding nodes to legacy clusters +# that do not have vnodes enabled. +# initial_token: + +# May either be "true" or "false" to enable globally +hinted_handoff_enabled: true + +# When hinted_handoff_enabled is true, a black list of data centers that will not +# perform hinted handoff +# hinted_handoff_disabled_datacenters: +# - DC1 +# - DC2 + +# this defines the maximum amount of time a dead host will have hints +# generated. After it has been dead this long, new hints for it will not be +# created until it has been seen alive and gone down again. +max_hint_window_in_ms: 10800000 # 3 hours + +# Maximum throttle in KBs per second, per delivery thread. This will be +# reduced proportionally to the number of nodes in the cluster. (If there +# are two nodes in the cluster, each delivery thread will use the maximum +# rate; if there are three, each will throttle to half of the maximum, +# since we expect two nodes to be delivering hints simultaneously.) +hinted_handoff_throttle_in_kb: 1024 + +# Number of threads with which to deliver hints; +# Consider increasing this number when you have multi-dc deployments, since +# cross-dc handoff tends to be slower +max_hints_delivery_threads: 2 + +# Directory where Cassandra should store hints. +# If not set, the default directory is $CASSANDRA_HOME/data/hints. +hints_directory: target/data/cassandra/hints + +# How often hints should be flushed from the internal buffers to disk. +# Will *not* trigger fsync. +hints_flush_period_in_ms: 10000 + +# Maximum size for a single hints file, in megabytes. +max_hints_file_size_in_mb: 128 + +# Compression to apply to the hint files. If omitted, hints files +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +#hints_compression: +# - class_name: LZ4Compressor +# parameters: +# - + +# Maximum throttle in KBs per second, total. This will be +# reduced proportionally to the number of nodes in the cluster. +batchlog_replay_throttle_in_kb: 1024 + +# Authentication backend, implementing IAuthenticator; used to identify users +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator, +# PasswordAuthenticator}. +# +# - AllowAllAuthenticator performs no checks - set it to disable authentication. +# - PasswordAuthenticator relies on username/password pairs to authenticate +# users. It keeps usernames and hashed passwords in system_auth.roles table. +# Please increase system_auth keyspace replication factor if you use this authenticator. +# If using PasswordAuthenticator, CassandraRoleManager must also be used (see below) +authenticator: AllowAllAuthenticator + +# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer, +# CassandraAuthorizer}. +# +# - AllowAllAuthorizer allows any action to any user - set it to disable authorization. +# - CassandraAuthorizer stores permissions in system_auth.role_permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +authorizer: AllowAllAuthorizer + +# Part of the Authentication & Authorization backend, implementing IRoleManager; used +# to maintain grants and memberships between roles. +# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager, +# which stores role information in the system_auth keyspace. Most functions of the +# IRoleManager require an authenticated login, so unless the configured IAuthenticator +# actually implements authentication, most of this functionality will be unavailable. +# +# - CassandraRoleManager stores role data in the system_auth keyspace. Please +# increase system_auth keyspace replication factor if you use this role manager. +role_manager: CassandraRoleManager + +# Network authorization backend, implementing INetworkAuthorizer; used to restrict user +# access to certain DCs +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllNetworkAuthorizer, +# CassandraNetworkAuthorizer}. +# +# - AllowAllNetworkAuthorizer allows access to any DC to any user - set it to disable authorization. +# - CassandraNetworkAuthorizer stores permissions in system_auth.network_permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +network_authorizer: AllowAllNetworkAuthorizer + +# Validity period for roles cache (fetching granted roles can be an expensive +# operation depending on the role manager, CassandraRoleManager is one example) +# Granted roles are cached for authenticated sessions in AuthenticatedUser and +# after the period specified here, become eligible for (async) reload. +# Defaults to 2000, set to 0 to disable caching entirely. +# Will be disabled automatically for AllowAllAuthenticator. +roles_validity_in_ms: 2000 + +# Refresh interval for roles cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If roles_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as roles_validity_in_ms. +# roles_update_interval_in_ms: 2000 + +# Validity period for permissions cache (fetching permissions can be an +# expensive operation depending on the authorizer, CassandraAuthorizer is +# one example). Defaults to 2000, set to 0 to disable. +# Will be disabled automatically for AllowAllAuthorizer. +permissions_validity_in_ms: 2000 + +# Refresh interval for permissions cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If permissions_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as permissions_validity_in_ms. +# permissions_update_interval_in_ms: 2000 + +# Validity period for credentials cache. This cache is tightly coupled to +# the provided PasswordAuthenticator implementation of IAuthenticator. If +# another IAuthenticator implementation is configured, this cache will not +# be automatically used and so the following settings will have no effect. +# Please note, credentials are cached in their encrypted form, so while +# activating this cache may reduce the number of queries made to the +# underlying table, it may not bring a significant reduction in the +# latency of individual authentication attempts. +# Defaults to 2000, set to 0 to disable credentials caching. +credentials_validity_in_ms: 2000 + +# Refresh interval for credentials cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If credentials_validity_in_ms is non-zero, then this must be +# also. +# Defaults to the same value as credentials_validity_in_ms. +# credentials_update_interval_in_ms: 2000 + +# The partitioner is responsible for distributing groups of rows (by +# partition key) across nodes in the cluster. The partitioner can NOT be +# changed without reloading all data. If you are adding nodes or upgrading, +# you should set this to the same partitioner that you are currently using. +# +# The default partitioner is the Murmur3Partitioner. Older partitioners +# such as the RandomPartitioner, ByteOrderedPartitioner, and +# OrderPreservingPartitioner have been included for backward compatibility only. +# For new clusters, you should NOT change this value. +# +partitioner: org.apache.cassandra.dht.Murmur3Partitioner + +# Directories where Cassandra should store data on disk. Cassandra +# will spread data evenly across them, subject to the granularity of +# the configured compaction strategy. +# If not set, the default directory is $CASSANDRA_HOME/data/data. +data_file_directories: + - target/data/cassandra/data + +# 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: target/data/cassandra/commitlog + +# 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 +# containing a CDC-enabled table if at space limit in cdc_raw_directory). +cdc_enabled: true + +# CommitLogSegments are moved to this directory on flush if cdc_enabled: true and the +# 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 +cdc_raw_directory: target/data/cassandra/cdc_raw + +# Policy for data disk failures: +# +# die +# shut down gossip and client transports and kill the JVM for any fs errors or +# single-sstable errors, so the node can be replaced. +# +# stop_paranoid +# shut down gossip and client transports even for single-sstable errors, +# kill the JVM for errors during startup. +# +# stop +# shut down gossip and client transports, leaving the node effectively dead, but +# can still be inspected via JMX, kill the JVM for errors during startup. +# +# best_effort +# stop using the failed disk and respond to requests based on +# remaining available sstables. This means you WILL see obsolete +# data at CL.ONE! +# +# ignore +# ignore fatal errors and let requests fail, as in pre-1.2 Cassandra +disk_failure_policy: stop + +# Policy for commit disk failures: +# +# die +# shut down the node and kill the JVM, so the node can be replaced. +# +# stop +# shut down the node, leaving the node effectively dead, but +# can still be inspected via JMX. +# +# stop_commit +# shutdown the commit log, letting writes collect but +# continuing to service reads, as in pre-2.0.5 Cassandra +# +# ignore +# ignore fatal errors and let the batches fail +commit_failure_policy: stop + +# Maximum size of the native protocol prepared statement cache +# +# Valid values are either "auto" (omitting the value) or a value greater 0. +# +# Note that specifying a too large value will result in long running GCs and possbily +# out-of-memory errors. Keep the value at a small fraction of the heap. +# +# If you constantly see "prepared statements discarded in the last minute because +# cache limit reached" messages, the first step is to investigate the root cause +# of these messages and check whether prepared statements are used correctly - +# i.e. use bind markers for variable parts. +# +# Do only change the default value, if you really have more prepared statements than +# fit in the cache. In most cases it is not neccessary to change this value. +# Constantly re-preparing statements is a performance penalty. +# +# Default value ("auto") is 1/256th of the heap or 10MB, whichever is greater +prepared_statements_cache_size_mb: + +# Maximum size of the key cache in memory. +# +# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the +# minimum, sometimes more. The key cache is fairly tiny for the amount of +# time it saves, so it's worthwhile to use it at large numbers. +# The row cache saves even more time, but must contain the entire row, +# so it is extremely space-intensive. It's best to only use the +# row cache if you have hot rows or static rows. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(5% of Heap (in MB), 100MB)). Set to 0 to disable key cache. +key_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# save the key cache. Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 14400 or 4 hours. +key_cache_save_period: 14400 + +# Number of keys from the key cache to save +# Disabled by default, meaning all keys are going to be saved +# key_cache_keys_to_save: 100 + +# Row cache implementation class name. Available implementations: +# +# org.apache.cassandra.cache.OHCProvider +# Fully off-heap row cache implementation (default). +# +# org.apache.cassandra.cache.SerializingCacheProvider +# This is the row cache implementation availabile +# in previous releases of Cassandra. +# row_cache_class_name: org.apache.cassandra.cache.OHCProvider + +# Maximum size of the row cache in memory. +# Please note that OHC cache implementation requires some additional off-heap memory to manage +# the map structures and some in-flight memory during operations before/after cache entries can be +# accounted against the cache capacity. This overhead is usually small compared to the whole capacity. +# Do not specify more memory that the system can afford in the worst usual situation and leave some +# headroom for OS block level cache. Do never allow your system to swap. +# +# Default value is 0, to disable row caching. +row_cache_size_in_mb: 0 + +# Duration in seconds after which Cassandra should save the row cache. +# Caches are saved to saved_caches_directory as specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 0 to disable saving the row cache. +row_cache_save_period: 0 + +# Number of keys from the row cache to save. +# Specify 0 (which is the default), meaning all keys are going to be saved +# row_cache_keys_to_save: 100 + +# Maximum size of the counter cache in memory. +# +# Counter cache helps to reduce counter locks' contention for hot counter cells. +# In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before +# write entirely. With RF > 1 a counter cache hit will still help to reduce the duration +# of the lock hold, helping with hot counter cell updates, but will not allow skipping +# the read entirely. Only the local (clock, count) tuple of a counter cell is kept +# in memory, not the whole counter, so it's relatively cheap. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(2.5% of Heap (in MB), 50MB)). Set to 0 to disable counter cache. +# NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache. +counter_cache_size_in_mb: + +# Duration in seconds after which Cassandra should +# save the counter cache (keys only). Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Default is 7200 or 2 hours. +counter_cache_save_period: 7200 + +# Number of keys from the counter cache to save +# Disabled by default, meaning all keys are going to be saved +# counter_cache_keys_to_save: 100 + +# saved caches +# If not set, the default directory is $CASSANDRA_HOME/data/saved_caches. +saved_caches_directory: target/cassandra/data/saved_caches + +# commitlog_sync may be either "periodic", "group", or "batch." +# +# When in batch mode, Cassandra won't ack writes until the commit log +# has been flushed to disk. Each incoming write will trigger the flush task. +# commitlog_sync_batch_window_in_ms is a deprecated value. Previously it had +# almost no value, and is being removed. +# +# commitlog_sync_batch_window_in_ms: 2 +# +# group mode is similar to batch mode, where Cassandra will not ack writes +# until the commit log has been flushed to disk. The difference is group +# mode will wait up to commitlog_sync_group_window_in_ms between flushes. +# +# commitlog_sync_group_window_in_ms: 1000 +# +# the default option is "periodic" where writes may be acked immediately +# and the CommitLog is simply synced every commitlog_sync_period_in_ms +# milliseconds. +commitlog_sync: periodic +commitlog_sync_period_in_ms: 10000 + +# When in periodic commitlog mode, the number of milliseconds to block writes +# while waiting for a slow disk flush to complete. +# periodic_commitlog_sync_lag_block_in_ms: + +# The size of the individual commitlog file segments. A commitlog +# segment may be archived, deleted, or recycled once all the data +# in it (potentially from each columnfamily in the system) has been +# flushed to sstables. +# +# The default size is 32, which is almost always fine, but if you are +# archiving commitlog segments (see commitlog_archiving.properties), +# then you probably want a finer granularity of archiving; 8 or 16 MB +# is reasonable. +# Max mutation size is also configurable via max_mutation_size_in_kb setting in +# cassandra.yaml. The default is half the size commitlog_segment_size_in_mb * 1024. +# This should be positive and less than 2048. +# +# NOTE: If max_mutation_size_in_kb is set explicitly then commitlog_segment_size_in_mb must +# be set to at least twice the size of max_mutation_size_in_kb / 1024 +# +commitlog_segment_size: 1MiB + +# Compression to apply to the commit log. If omitted, the commit log +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +commitlog_compression: + - class_name: LZ4Compressor +# parameters: +# - + +# Compression to apply to SSTables as they flush for compressed tables. +# Note that tables without compression enabled do not respect this flag. +# +# As high ratio compressors like LZ4HC, Zstd, and Deflate can potentially +# block flushes for too long, the default is to flush with a known fast +# compressor in those cases. Options are: +# +# none : Flush without compressing blocks but while still doing checksums. +# fast : Flush with a fast compressor. If the table is already using a +# fast compressor that compressor is used. +# table: Always flush with the same compressor that the table uses. This +# was the pre 4.0 behavior. +# +# flush_compression: fast + +# any class that implements the SeedProvider interface and has a +# constructor that takes a Map of parameters will do. +seed_provider: + # Addresses of hosts that are deemed contact points. + # Cassandra nodes use this list of hosts to find each other and learn + # the topology of the ring. You must change this if you are running + # multiple nodes! + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + # seeds is actually a comma-delimited list of addresses. + # Ex: ",," + - seeds: "127.0.0.1:7000" + +# For workloads with more data than can fit in memory, Cassandra's +# bottleneck will be reads that need to fetch data from +# disk. "concurrent_reads" should be set to (16 * number_of_drives) in +# order to allow the operations to enqueue low enough in the stack +# that the OS and drives can reorder them. Same applies to +# "concurrent_counter_writes", since counter writes read the current +# values before incrementing and writing them back. +# +# On the other hand, since writes are almost never IO bound, the ideal +# number of "concurrent_writes" is dependent on the number of cores in +# your system; (8 * number_of_cores) is a good rule of thumb. +concurrent_reads: 32 +concurrent_writes: 32 +concurrent_counter_writes: 32 + +# For materialized view writes, as there is a read involved, so this should +# be limited by the less of concurrent reads or concurrent writes. +concurrent_materialized_view_writes: 32 + +# Maximum memory to use for inter-node and client-server networking buffers. +# +# Defaults to the smaller of 1/16 of heap or 128MB. This pool is allocated off-heap, +# so is in addition to the memory allocated for heap. The cache also has on-heap +# overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size +# if the default 64k chunk size is used). +# Memory is only allocated when needed. +# networking_cache_size_in_mb: 128 + +# Enable the sstable chunk cache. The chunk cache will store recently accessed +# sections of the sstable in-memory as uncompressed buffers. +# file_cache_enabled: false + +# Maximum memory to use for sstable chunk cache and buffer pooling. +# 32MB of this are reserved for pooling buffers, the rest is used for chunk cache +# that holds uncompressed sstable chunks. +# Defaults to the smaller of 1/4 of heap or 512MB. This pool is allocated off-heap, +# so is in addition to the memory allocated for heap. The cache also has on-heap +# overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size +# if the default 64k chunk size is used). +# Memory is only allocated when needed. +# file_cache_size_in_mb: 512 + +# Flag indicating whether to allocate on or off heap when the sstable buffer +# pool is exhausted, that is when it has exceeded the maximum memory +# file_cache_size_in_mb, beyond which it will not cache buffers but allocate on request. + +# buffer_pool_use_heap_if_exhausted: true + +# The strategy for optimizing disk read +# Possible values are: +# ssd (for solid state disks, the default) +# spinning (for spinning disks) +# disk_optimization_strategy: ssd + +# Total permitted memory to use for memtables. Cassandra will stop +# accepting writes when the limit is exceeded until a flush completes, +# and will trigger a flush based on memtable_cleanup_threshold +# If omitted, Cassandra will set both to 1/4 the size of the heap. +# memtable_heap_space_in_mb: 2048 +# memtable_offheap_space_in_mb: 2048 + +# memtable_cleanup_threshold is deprecated. The default calculation +# is the only reasonable choice. See the comments on memtable_flush_writers +# for more information. +# +# Ratio of occupied non-flushing memtable size to total permitted size +# that will trigger a flush of the largest memtable. Larger mct will +# mean larger flushes and hence less compaction, but also less concurrent +# flush activity which can make it difficult to keep your disks fed +# under heavy write load. +# +# memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1) +# memtable_cleanup_threshold: 0.11 + +# Specify the way Cassandra allocates and manages memtable memory. +# Options are: +# +# heap_buffers +# on heap nio buffers +# +# offheap_buffers +# off heap (direct) nio buffers +# +# offheap_objects +# off heap objects +memtable_allocation_type: heap_buffers + +# Limit memory usage for Merkle tree calculations during repairs. The default +# is 1/16th of the available heap. The main tradeoff is that smaller trees +# have less resolution, which can lead to over-streaming data. If you see heap +# pressure during repairs, consider lowering this, but you cannot go below +# one megabyte. If you see lots of over-streaming, consider raising +# this or using subrange repair. +# +# For more details see https://issues.apache.org/jira/browse/CASSANDRA-14096. +# +# repair_session_space_in_mb: + +# Total space to use for commit logs on disk. +# +# If space gets above this value, Cassandra will flush every dirty CF +# in the oldest segment and remove it. So a small total commitlog space +# will tend to cause more flush activity on less-active columnfamilies. +# +# The default value is the smaller of 8192, and 1/4 of the total space +# of the commitlog volume. +# +# commitlog_total_space_in_mb: 8192 + +# This sets the number of memtable flush writer threads per disk +# as well as the total number of memtables that can be flushed concurrently. +# These are generally a combination of compute and IO bound. +# +# Memtable flushing is more CPU efficient than memtable ingest and a single thread +# can keep up with the ingest rate of a whole server on a single fast disk +# until it temporarily becomes IO bound under contention typically with compaction. +# At that point you need multiple flush threads. At some point in the future +# it may become CPU bound all the time. +# +# You can tell if flushing is falling behind using the MemtablePool.BlockedOnAllocation +# metric which should be 0, but will be non-zero if threads are blocked waiting on flushing +# to free memory. +# +# memtable_flush_writers defaults to two for a single data directory. +# This means that two memtables can be flushed concurrently to the single data directory. +# If you have multiple data directories the default is one memtable flushing at a time +# but the flush will use a thread per data directory so you will get two or more writers. +# +# Two is generally enough to flush on a fast disk [array] mounted as a single data directory. +# Adding more flush writers will result in smaller more frequent flushes that introduce more +# compaction overhead. +# +# There is a direct tradeoff between number of memtables that can be flushed concurrently +# and flush size and frequency. More is not better you just need enough flush writers +# to never stall waiting for flushing to free memory. +# +#memtable_flush_writers: 2 + +# Total space to use for change-data-capture logs on disk. +# +# If space gets above this value, Cassandra will throw WriteTimeoutException +# on Mutations including tables with CDC enabled. A CDCCompactor is responsible +# for parsing the raw CDC logs and deleting them when parsing is completed. +# +# The default value is the min of 4096 mb and 1/8th of the total space +# of the drive where cdc_raw_directory resides. +# cdc_total_space_in_mb: 4096 + +# When we hit our cdc_raw limit and the CDCCompactor is either running behind +# or experiencing backpressure, we check at the following interval to see if any +# new space for cdc-tracked tables has been made available. Default to 250ms +# cdc_free_space_check_interval_ms: 250 + +# A fixed memory pool size in MB for for SSTable index summaries. If left +# empty, this will default to 5% of the heap size. If the memory usage of +# all index summaries exceeds this limit, SSTables with low read rates will +# shrink their index summaries in order to meet this limit. However, this +# is a best-effort process. In extreme conditions Cassandra may need to use +# more than this amount of memory. +index_summary_capacity_in_mb: + +# How frequently index summaries should be resampled. This is done +# periodically to redistribute memory from the fixed-size pool to sstables +# proportional their recent read rates. Setting to -1 will disable this +# process, leaving existing index summaries at their current sampling level. +index_summary_resize_interval_in_minutes: 60 + +# Whether to, when doing sequential writing, fsync() at intervals in +# order to force the operating system to flush the dirty +# buffers. Enable this to avoid sudden dirty buffer flushing from +# impacting read latencies. Almost always a good idea on SSDs; not +# necessarily on platters. +trickle_fsync: false +trickle_fsync_interval_in_kb: 10240 + +# TCP port, for commands and data +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +storage_port: 7000 + +# SSL port, for legacy encrypted communication. This property is unused unless enabled in +# server_encryption_options (see below). As of cassandra 4.0, this property is deprecated +# as a single port can be used for either/both secure and insecure connections. +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +ssl_storage_port: 7001 + +# Address or interface to bind to and tell other Cassandra nodes to connect to. +# You _must_ change this if you want multiple nodes to be able to communicate! +# +# Set listen_address OR listen_interface, not both. +# +# Leaving it blank leaves it up to InetAddress.getLocalHost(). This +# will always do the Right Thing _if_ the node is properly configured +# (hostname, name resolution, etc), and the Right Thing is to use the +# address associated with the hostname (it might not be). If unresolvable +# it will fall back to InetAddress.getLoopbackAddress(), which is wrong for production systems. +# +# Setting listen_address to 0.0.0.0 is always wrong. +# +listen_address: localhost + +# Set listen_address OR listen_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# listen_interface: eth0 + +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using listen_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +# listen_interface_prefer_ipv6: false + +# Address to broadcast to other Cassandra nodes +# Leaving this blank will set it to the same value as listen_address +# broadcast_address: 1.2.3.4 + +# When using multiple physical network interfaces, set this +# to true to listen on broadcast_address in addition to +# the listen_address, allowing nodes to communicate in both +# interfaces. +# Ignore this property if the network configuration automatically +# routes between the public and private networks such as EC2. +# listen_on_broadcast_address: false + +# Internode authentication backend, implementing IInternodeAuthenticator; +# used to allow/disallow connections from peer nodes. +# internode_authenticator: org.apache.cassandra.auth.AllowAllInternodeAuthenticator + +# Whether to start the native transport server. +# The address on which the native transport is bound is defined by rpc_address. +start_native_transport: true +# port for the CQL native transport to listen for clients on +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +native_transport_port: 9042 +# Enabling native transport encryption in client_encryption_options allows you to either use +# encryption for the standard port or to use a dedicated, additional port along with the unencrypted +# standard native_transport_port. +# Enabling client encryption and keeping native_transport_port_ssl disabled will use encryption +# for native_transport_port. Setting native_transport_port_ssl to a different value +# from native_transport_port will use encryption for native_transport_port_ssl while +# keeping native_transport_port unencrypted. +# native_transport_port_ssl: 9142 +# The maximum threads for handling requests (note that idle threads are stopped +# after 30 seconds so there is not corresponding minimum setting). +# native_transport_max_threads: 128 +# +# The maximum size of allowed frame. Frame (requests) larger than this will +# be rejected as invalid. The default is 256MB. If you're changing this parameter, +# you may want to adjust max_value_size_in_mb accordingly. This should be positive and less than 2048. +# native_transport_max_frame_size_in_mb: 256 + +# The maximum number of concurrent client connections. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections: -1 + +# The maximum number of concurrent client connections per source ip. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections_per_ip: -1 + +# Controls whether Cassandra honors older, yet currently supported, protocol versions. +# The default is true, which means all supported protocols will be honored. +native_transport_allow_older_protocols: true + +# Controls when idle client connections are closed. Idle connections are ones that had neither reads +# nor writes for a time period. +# +# Clients may implement heartbeats by sending OPTIONS native protocol message after a timeout, which +# will reset idle timeout timer on the server side. To close idle client connections, corresponding +# values for heartbeat intervals have to be set on the client side. +# +# Idle connection timeouts are disabled by default. +# native_transport_idle_timeout_in_ms: 60000 + +# The address or interface to bind the native transport server to. +# +# Set rpc_address OR rpc_interface, not both. +# +# Leaving rpc_address blank has the same effect as on listen_address +# (i.e. it will be based on the configured hostname of the node). +# +# Note that unlike listen_address, you can specify 0.0.0.0, but you must also +# set broadcast_rpc_address to a value other than 0.0.0.0. +# +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +rpc_address: localhost + +# Set rpc_address OR rpc_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# rpc_interface: eth1 + +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using rpc_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +# rpc_interface_prefer_ipv6: false + +# RPC address to broadcast to drivers and other Cassandra nodes. This cannot +# be set to 0.0.0.0. If left blank, this will be set to the value of +# rpc_address. If rpc_address is set to 0.0.0.0, broadcast_rpc_address must +# be set. +# broadcast_rpc_address: 1.2.3.4 + +# enable or disable keepalive on rpc/native connections +rpc_keepalive: true + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# See also: +# /proc/sys/net/core/wmem_max +# /proc/sys/net/core/rmem_max +# /proc/sys/net/ipv4/tcp_wmem +# /proc/sys/net/ipv4/tcp_wmem +# and 'man tcp' +# internode_send_buff_size_in_bytes: + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# internode_recv_buff_size_in_bytes: + +# Set to true to have Cassandra create a hard link to each sstable +# flushed or streamed locally in a backups/ subdirectory of the +# keyspace data. Removing these links is the operator's +# responsibility. +incremental_backups: false + +# Whether or not to take a snapshot before each compaction. Be +# careful using this option, since Cassandra won't clean up the +# snapshots for you. Mostly useful if you're paranoid when there +# is a data format change. +snapshot_before_compaction: false + +# Whether or not a snapshot is taken of the data before keyspace truncation +# or dropping of column families. The STRONGLY advised default of true +# should be used to provide data safety. If you set this flag to false, you will +# lose data on truncation or drop. +auto_snapshot: true + +# The act of creating or clearing a snapshot involves creating or removing +# potentially tens of thousands of links, which can cause significant performance +# impact, especially on consumer grade SSDs. A non-zero value here can +# be used to throttle these links to avoid negative performance impact of +# taking and clearing snapshots +snapshot_links_per_second: 0 + +# Granularity of the collation index of rows within a partition. +# Increase if your rows are large, or if you have a very large +# number of rows per partition. The competing goals are these: +# +# - a smaller granularity means more index entries are generated +# and looking up rows withing the partition by collation column +# is faster +# - but, Cassandra will keep the collation index in memory for hot +# rows (as part of the key cache), so a larger granularity means +# you can cache more hot rows +column_index_size_in_kb: 64 + +# Per sstable indexed key cache entries (the collation index in memory +# mentioned above) exceeding this size will not be held on heap. +# This means that only partition information is held on heap and the +# index entries are read from disk. +# +# Note that this size refers to the size of the +# serialized index information and not the size of the partition. +column_index_cache_size_in_kb: 2 + +# Number of simultaneous compactions to allow, NOT including +# validation "compactions" for anti-entropy repair. Simultaneous +# compactions can help preserve read performance in a mixed read/write +# workload, by mitigating the tendency of small sstables to accumulate +# during a single long running compactions. The default is usually +# fine and if you experience problems with compaction running too +# slowly or too fast, you should look at +# compaction_throughput_mb_per_sec first. +# +# concurrent_compactors defaults to the smaller of (number of disks, +# number of cores), with a minimum of 2 and a maximum of 8. +# +# If your data directories are backed by SSD, you should increase this +# to the number of cores. +#concurrent_compactors: 1 + +# Number of simultaneous repair validations to allow. If not set or set to +# a value less than 1, it defaults to the value of concurrent_compactors. +# To set a value greeater than concurrent_compactors at startup, the system +# property cassandra.allow_unlimited_concurrent_validations must be set to +# true. To dynamically resize to a value > concurrent_compactors on a running +# node, first call the bypassConcurrentValidatorsLimit method on the +# org.apache.cassandra.db:type=StorageService mbean +# concurrent_validations: 0 + +# Number of simultaneous materialized view builder tasks to allow. +concurrent_materialized_view_builders: 1 + +# Throttles compaction to the given total throughput across the entire +# system. The faster you insert data, the faster you need to compact in +# order to keep the sstable count down, but in general, setting this to +# 16 to 32 times the rate you are inserting data is more than sufficient. +# Setting this to 0 disables throttling. Note that this accounts for all types +# of compaction, including validation compaction (building Merkle trees +# for repairs). +compaction_throughput_mb_per_sec: 64 + +# When compacting, the replacement sstable(s) can be opened before they +# are completely written, and used in place of the prior sstables for +# any range that has been written. This helps to smoothly transfer reads +# between the sstables, reducing page cache churn and keeping hot rows hot +sstable_preemptive_open_interval_in_mb: 50 + +# When enabled, permits Cassandra to zero-copy stream entire eligible +# SSTables between nodes, including every component. +# This speeds up the network transfer significantly subject to +# throttling specified by stream_throughput_outbound_megabits_per_sec. +# Enabling this will reduce the GC pressure on sending and receiving node. +# When unset, the default is enabled. While this feature tries to keep the +# disks balanced, it cannot guarantee it. This feature will be automatically +# disabled if internode encryption is enabled. +# stream_entire_sstables: true + +# Throttles all outbound streaming file transfers on this node to the +# given total throughput in Mbps. This is necessary because Cassandra does +# mostly sequential IO when streaming data during bootstrap or repair, which +# can lead to saturating the network connection and degrading rpc performance. +# When unset, the default is 200 Mbps or 25 MB/s. +# stream_throughput_outbound_megabits_per_sec: 200 + +# Throttles all streaming file transfer between the datacenters, +# this setting allows users to throttle inter dc stream throughput in addition +# to throttling all network stream traffic as configured with +# stream_throughput_outbound_megabits_per_sec +# When unset, the default is 200 Mbps or 25 MB/s +# inter_dc_stream_throughput_outbound_megabits_per_sec: 200 + +# How long the coordinator should wait for read operations to complete. +# Lowest acceptable value is 10 ms. +read_request_timeout_in_ms: 5000 +# How long the coordinator should wait for seq or index scans to complete. +# Lowest acceptable value is 10 ms. +range_request_timeout_in_ms: 10000 +# How long the coordinator should wait for writes to complete. +# Lowest acceptable value is 10 ms. +write_request_timeout_in_ms: 2000 +# How long the coordinator should wait for counter writes to complete. +# Lowest acceptable value is 10 ms. +counter_write_request_timeout_in_ms: 5000 +# How long a coordinator should continue to retry a CAS operation +# that contends with other proposals for the same row. +# Lowest acceptable value is 10 ms. +cas_contention_timeout_in_ms: 1000 +# How long the coordinator should wait for truncates to complete +# (This can be much longer, because unless auto_snapshot is disabled +# we need to flush first so we can snapshot before removing the data.) +# Lowest acceptable value is 10 ms. +truncate_request_timeout_in_ms: 60000 +# The default timeout for other, miscellaneous operations. +# Lowest acceptable value is 10 ms. +request_timeout_in_ms: 10000 + +# Defensive settings for protecting Cassandra from true network partitions. +# See (CASSANDRA-14358) for details. +# +# The amount of time to wait for internode tcp connections to establish. +# internode_tcp_connect_timeout_in_ms = 2000 +# +# The amount of time unacknowledged data is allowed on a connection before we throw out the connection +# Note this is only supported on Linux + epoll, and it appears to behave oddly above a setting of 30000 +# (it takes much longer than 30s) as of Linux 4.12. If you want something that high set this to 0 +# which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8. +# internode_tcp_user_timeout_in_ms = 30000 + +# The amount of time unacknowledged data is allowed on a streaming connection. +# The default is 5 minutes. Increase it or set it to 0 in order to increase the timeout. +# internode_streaming_tcp_user_timeout_in_ms = 300000 + +# The maximum continuous period a connection may be unwritable in application space +# internode_application_timeout_in_ms = 30000 + +# Global, per-endpoint and per-connection limits imposed on messages queued for delivery to other nodes +# and waiting to be processed on arrival from other nodes in the cluster. These limits are applied to the on-wire +# size of the message being sent or received. +# +# The basic per-link limit is consumed in isolation before any endpoint or global limit is imposed. +# Each node-pair has three links: urgent, small and large. So any given node may have a maximum of +# N*3*(internode_application_send_queue_capacity_in_bytes+internode_application_receive_queue_capacity_in_bytes) +# messages queued without any coordination between them although in practice, with token-aware routing, only RF*tokens +# nodes should need to communicate with significant bandwidth. +# +# The per-endpoint limit is imposed on all messages exceeding the per-link limit, simultaneously with the global limit, +# on all links to or from a single node in the cluster. +# The global limit is imposed on all messages exceeding the per-link limit, simultaneously with the per-endpoint limit, +# on all links to or from any node in the cluster. +# +# internode_application_send_queue_capacity_in_bytes: 4194304 #4MiB +# internode_application_send_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB +# internode_application_send_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB +# internode_application_receive_queue_capacity_in_bytes: 4194304 #4MiB +# internode_application_receive_queue_reserve_endpoint_capacity_in_bytes: 134217728 #128MiB +# internode_application_receive_queue_reserve_global_capacity_in_bytes: 536870912 #512MiB + + +# How long before a node logs slow queries. Select queries that take longer than +# this timeout to execute, will generate an aggregated log message, so that slow queries +# can be identified. Set this value to zero to disable slow query logging. +slow_query_log_timeout_in_ms: 500 + +# Enable operation timeout information exchange between nodes to accurately +# measure request timeouts. If disabled, replicas will assume that requests +# were forwarded to them instantly by the coordinator, which means that +# under overload conditions we will waste that much extra time processing +# already-timed-out requests. +# +# Warning: It is generally assumed that users have setup NTP on their clusters, and that clocks are modestly in sync, +# since this is a requirement for general correctness of last write wins. +#cross_node_timeout: true + +# Set keep-alive period for streaming +# This node will send a keep-alive message periodically with this period. +# If the node does not receive a keep-alive message from the peer for +# 2 keep-alive cycles the stream session times out and fail +# Default value is 300s (5 minutes), which means stalled stream +# times out in 10 minutes by default +# streaming_keep_alive_period_in_secs: 300 + +# Limit number of connections per host for streaming +# Increase this when you notice that joins are CPU-bound rather that network +# bound (for example a few nodes with big files). +# streaming_connections_per_host: 1 + + +# phi value that must be reached for a host to be marked down. +# most users should never need to adjust this. +# phi_convict_threshold: 8 + +# endpoint_snitch -- Set this to a class that implements +# IEndpointSnitch. The snitch has two functions: +# +# - it teaches Cassandra enough about your network topology to route +# requests efficiently +# - it allows Cassandra to spread replicas around your cluster to avoid +# correlated failures. It does this by grouping machines into +# "datacenters" and "racks." Cassandra will do its best not to have +# more than one replica on the same "rack" (which may not actually +# be a physical location) +# +# CASSANDRA WILL NOT ALLOW YOU TO SWITCH TO AN INCOMPATIBLE SNITCH +# ONCE DATA IS INSERTED INTO THE CLUSTER. This would cause data loss. +# This means that if you start with the default SimpleSnitch, which +# locates every node on "rack1" in "datacenter1", your only options +# if you need to add another datacenter are GossipingPropertyFileSnitch +# (and the older PFS). From there, if you want to migrate to an +# incompatible snitch like Ec2Snitch you can do it by adding new nodes +# under Ec2Snitch (which will locate them in a new "datacenter") and +# decommissioning the old ones. +# +# Out of the box, Cassandra provides: +# +# SimpleSnitch: +# Treats Strategy order as proximity. This can improve cache +# locality when disabling read repair. Only appropriate for +# single-datacenter deployments. +# +# GossipingPropertyFileSnitch +# This should be your go-to snitch for production use. The rack +# and datacenter for the local node are defined in +# cassandra-rackdc.properties and propagated to other nodes via +# gossip. If cassandra-topology.properties exists, it is used as a +# fallback, allowing migration from the PropertyFileSnitch. +# +# PropertyFileSnitch: +# Proximity is determined by rack and data center, which are +# explicitly configured in cassandra-topology.properties. +# +# Ec2Snitch: +# Appropriate for EC2 deployments in a single Region. Loads Region +# and Availability Zone information from the EC2 API. The Region is +# treated as the datacenter, and the Availability Zone as the rack. +# Only private IPs are used, so this will not work across multiple +# Regions. +# +# Ec2MultiRegionSnitch: +# Uses public IPs as broadcast_address to allow cross-region +# connectivity. (Thus, you should set seed addresses to the public +# IP as well.) You will need to open the storage_port or +# ssl_storage_port on the public IP firewall. (For intra-Region +# traffic, Cassandra will switch to the private IP after +# establishing a connection.) +# +# RackInferringSnitch: +# Proximity is determined by rack and data center, which are +# assumed to correspond to the 3rd and 2nd octet of each node's IP +# address, respectively. Unless this happens to match your +# deployment conventions, this is best used as an example of +# writing a custom Snitch class and is provided in that spirit. +# +# You can use a custom Snitch by setting this to the full class name +# of the snitch, which will be assumed to be on your classpath. +endpoint_snitch: SimpleSnitch + +# controls how often to perform the more expensive part of host score +# calculation +dynamic_snitch_update_interval_in_ms: 100 +# controls how often to reset all host scores, allowing a bad host to +# possibly recover +dynamic_snitch_reset_interval_in_ms: 600000 +# if set greater than zero, this will allow +# 'pinning' of replicas to hosts in order to increase cache capacity. +# The badness threshold will control how much worse the pinned host has to be +# before the dynamic snitch will prefer other replicas over it. This is +# expressed as a double which represents a percentage. Thus, a value of +# 0.2 means Cassandra would continue to prefer the static snitch values +# until the pinned host was 20% worse than the fastest. +dynamic_snitch_badness_threshold: 1.0 + +# Configure server-to-server internode encryption +# +# JVM and netty defaults for supported SSL socket protocols and cipher suites can +# be replaced using custom encryption options. This is not recommended +# unless you have policies in place that dictate certain settings, or +# need to disable vulnerable ciphers or protocols in case the JVM cannot +# be updated. +# +# FIPS compliant settings can be configured at JVM level and should not +# involve changing encryption settings here: +# https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/FIPS.html +# +# **NOTE** this default configuration is an insecure configuration. If you need to +# enable server-to-server encryption generate server keystores (and truststores for mutual +# authentication) per: +# http://download.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# Then perform the following configuration changes: +# +# Step 1: Set internode_encryption= and explicitly set optional=true. Restart all nodes +# +# Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual +# auth set require_client_auth=true. Restart all nodes +server_encryption_options: + # On outbound connections, determine which type of peers to securely connect to. + # The available options are : + # none : Do not encrypt outgoing connections + # dc : Encrypt connections to peers in other datacenters but not within datacenters + # rack : Encrypt connections to peers in other racks but not within racks + # all : Always use encrypted connections + internode_encryption: none + # When set to true, encrypted and unencrypted connections are allowed on the storage_port + # This should _only be true_ while in unencrypted or transitional operation + # optional defaults to true if internode_encryption is none + # optional: true + # If enabled, will open up an encrypted listening socket on ssl_storage_port. Should only be used + # during upgrade to 4.0; otherwise, set to false. + enable_legacy_ssl_storage_port: false + # Set to a valid keystore if internode_encryption is dc, rack or all + keystore: conf/.keystore + keystore_password: cassandra + # Verify peer server certificates + require_client_auth: false + # Set to a valid trustore if require_client_auth is true + truststore: conf/.truststore + truststore_password: cassandra + # Verify that the host name in the certificate matches the connected host + require_endpoint_verification: false + # More advanced defaults: + # protocol: TLS + # store_type: JKS + # cipher_suites: [ + # TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256, + # TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA, + # TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA, + # TLS_RSA_WITH_AES_256_CBC_SHA + # ] + +# Configure client-to-server encryption. +# +# **NOTE** this default configuration is an insecure configuration. If you need to +# enable client-to-server encryption generate server keystores (and truststores for mutual +# authentication) per: +# http://download.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# Then perform the following configuration changes: +# +# Step 1: Set enabled=true and explicitly set optional=true. Restart all nodes +# +# Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual +# auth set require_client_auth=true. Restart all nodes +client_encryption_options: + # Enable client-to-server encryption + enabled: false + # When set to true, encrypted and unencrypted connections are allowed on the native_transport_port + # This should _only be true_ while in unencrypted or transitional operation + # optional defaults to true when enabled is false, and false when enabled is true. + # optional: true + # Set keystore and keystore_password to valid keystores if enabled is true + keystore: conf/.keystore + keystore_password: cassandra + # Verify client certificates + require_client_auth: false + # Set trustore and truststore_password if require_client_auth is true + # truststore: conf/.truststore + # truststore_password: cassandra + # More advanced defaults: + # protocol: TLS + # store_type: JKS + # cipher_suites: [ + # TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256, + # TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA, + # TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA, + # TLS_RSA_WITH_AES_256_CBC_SHA + # ] + +# internode_compression controls whether traffic between nodes is +# compressed. +# Can be: +# +# all +# all traffic is compressed +# +# dc +# traffic between different datacenters is compressed +# +# none +# nothing is compressed. +internode_compression: dc + +# Enable or disable tcp_nodelay for inter-dc communication. +# Disabling it will result in larger (but fewer) network packets being sent, +# reducing overhead from the TCP protocol itself, at the cost of increasing +# latency if you block for cross-datacenter responses. +inter_dc_tcp_nodelay: false + +# TTL for different trace types used during logging of the repair process. +tracetype_query_ttl: 86400 +tracetype_repair_ttl: 604800 + +# If unset, all GC Pauses greater than gc_log_threshold_in_ms will log at +# INFO level +# UDFs (user defined functions) are disabled by default. +# As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code. +enable_user_defined_functions: false + +# Enables scripted UDFs (JavaScript UDFs). +# Java UDFs are always enabled, if enable_user_defined_functions is true. +# Enable this option to be able to use UDFs with "language javascript" or any custom JSR-223 provider. +# This option has no effect, if enable_user_defined_functions is false. +enable_scripted_user_defined_functions: false + +# The default Windows kernel timer and scheduling resolution is 15.6ms for power conservation. +# Lowering this value on Windows can provide much tighter latency and better throughput, however +# some virtualized environments may see a negative performance impact from changing this setting +# below their system default. The sysinternals 'clockres' tool can confirm your system's default +# setting. +windows_timer_interval: 1 + + +# Enables encrypting data at-rest (on disk). Different key providers can be plugged in, but the default reads from +# a JCE-style keystore. A single keystore can hold multiple keys, but the one referenced by +# the "key_alias" is the only key that will be used for encrypt opertaions; previously used keys +# can still (and should!) be in the keystore and will be used on decrypt operations +# (to handle the case of key rotation). +# +# It is strongly recommended to download and install Java Cryptography Extension (JCE) +# Unlimited Strength Jurisdiction Policy Files for your version of the JDK. +# (current link: http://www.oracle.com/technetwork/java/javase/downloads/jce8-download-2133166.html) +# +# Currently, only the following file types are supported for transparent data encryption, although +# more are coming in future cassandra releases: commitlog, hints +transparent_data_encryption_options: + enabled: false + chunk_length_kb: 64 + cipher: AES/CBC/PKCS5Padding + key_alias: testing:1 + # CBC IV length for AES needs to be 16 bytes (which is also the default size) + # iv_length: 16 + key_provider: + - class_name: org.apache.cassandra.security.JKSKeyProvider + parameters: + - keystore: conf/.keystore + keystore_password: cassandra + store_type: JCEKS + key_password: cassandra + + +##################### +# SAFETY THRESHOLDS # +##################### + +# When executing a scan, within or across a partition, we need to keep the +# tombstones seen in memory so we can return them to the coordinator, which +# will use them to make sure other replicas also know about the deleted rows. +# With workloads that generate a lot of tombstones, this can cause performance +# problems and even exaust the server heap. +# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) +# Adjust the thresholds here if you understand the dangers and want to +# scan more tombstones anyway. These thresholds may also be adjusted at runtime +# using the StorageService mbean. +tombstone_warn_threshold: 1000 +tombstone_failure_threshold: 100000 + +# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a +# mechanism called replica filtering protection to ensure that results from stale replicas do +# not violate consistency. (See CASSANDRA-8272 and CASSANDRA-15907 for more details.) This +# mechanism materializes replica results by partition on-heap at the coordinator. The more possibly +# stale results returned by the replicas, the more rows materialized during the query. +replica_filtering_protection: + # These thresholds exist to limit the damage severely out-of-date replicas can cause during these + # queries. They limit the number of rows from all replicas individual index and filtering queries + # can materialize on-heap to return correct results at the desired read consistency level. + # + # "cached_replica_rows_warn_threshold" is the per-query threshold at which a warning will be logged. + # "cached_replica_rows_fail_threshold" is the per-query threshold at which the query will fail. + # + # These thresholds may also be adjusted at runtime using the StorageService mbean. + # + # If the failure threshold is breached, it is likely that either the current page/fetch size + # is too large or one or more replicas is severely out-of-sync and in need of repair. + cached_rows_warn_threshold: 2000 + cached_rows_fail_threshold: 32000 + +# Log WARN on any multiple-partition batch size exceeding this value. 5kb per batch by default. +# Caution should be taken on increasing the size of this threshold as it can lead to node instability. +batch_size_warn_threshold_in_kb: 5 + +# Fail any multiple-partition batch exceeding this value. 50kb (10x warn threshold) by default. +batch_size_fail_threshold_in_kb: 50 + +# Log WARN on any batches not of type LOGGED than span across more partitions than this limit +unlogged_batch_across_partitions_warn_threshold: 10 + +# Log a warning when compacting partitions larger than this value +compaction_large_partition_warning_threshold_mb: 100 + +# GC Pauses greater than 200 ms will be logged at INFO level +# This threshold can be adjusted to minimize logging if necessary +# gc_log_threshold_in_ms: 200 + +# GC Pauses greater than gc_warn_threshold_in_ms will be logged at WARN level +# Adjust the threshold based on your application throughput requirement. Setting to 0 +# will deactivate the feature. +# gc_warn_threshold_in_ms: 1000 + +# Maximum size of any value in SSTables. Safety measure to detect SSTable corruption +# early. Any value size larger than this threshold will result into marking an SSTable +# as corrupted. This should be positive and less than 2048. +# max_value_size_in_mb: 256 + +# Coalescing Strategies # +# Coalescing multiples messages turns out to significantly boost message processing throughput (think doubling or more). +# On bare metal, the floor for packet processing throughput is high enough that many applications won't notice, but in +# virtualized environments, the point at which an application can be bound by network packet processing can be +# surprisingly low compared to the throughput of task processing that is possible inside a VM. It's not that bare metal +# doesn't benefit from coalescing messages, it's that the number of packets a bare metal network interface can process +# is sufficient for many applications such that no load starvation is experienced even without coalescing. +# There are other benefits to coalescing network messages that are harder to isolate with a simple metric like messages +# per second. By coalescing multiple tasks together, a network thread can process multiple messages for the cost of one +# trip to read from a socket, and all the task submission work can be done at the same time reducing context switching +# and increasing cache friendliness of network message processing. +# See CASSANDRA-8692 for details. + +# Strategy to use for coalescing messages in OutboundTcpConnection. +# Can be fixed, movingaverage, timehorizon, disabled (default). +# You can also specify a subclass of CoalescingStrategies.CoalescingStrategy by name. +# otc_coalescing_strategy: DISABLED + +# How many microseconds to wait for coalescing. For fixed strategy this is the amount of time after the first +# message is received before it will be sent with any accompanying messages. For moving average this is the +# maximum amount of time that will be waited as well as the interval at which messages must arrive on average +# for coalescing to be enabled. +# otc_coalescing_window_us: 200 + +# Do not try to coalesce messages if we already got that many messages. This should be more than 2 and less than 128. +# otc_coalescing_enough_coalesced_messages: 8 + +# How many milliseconds to wait between two expiration runs on the backlog (queue) of the OutboundTcpConnection. +# Expiration is done if messages are piling up in the backlog. Droppable messages are expired to free the memory +# taken by expired messages. The interval should be between 0 and 1000, and in most installations the default value +# will be appropriate. A smaller value could potentially expire messages slightly sooner at the expense of more CPU +# time and queue contention while iterating the backlog of messages. +# An interval of 0 disables any wait time, which is the behavior of former Cassandra versions. +# +# otc_backlog_expiration_interval_ms: 200 + +# Track a metric per keyspace indicating whether replication achieved the ideal consistency +# level for writes without timing out. This is different from the consistency level requested by +# each write which may be lower in order to facilitate availability. +# ideal_consistency_level: EACH_QUORUM + +# Automatically upgrade sstables after upgrade - if there is no ordinary compaction to do, the +# oldest non-upgraded sstable will get upgraded to the latest version +# automatic_sstable_upgrade: false +# Limit the number of concurrent sstable upgrades +# max_concurrent_automatic_sstable_upgrades: 1 + +# Audit logging - Logs every incoming CQL command request, authentication to a node. See the docs +# on audit_logging for full details about the various configuration options. +audit_logging_options: + enabled: false + logger: + - class_name: BinAuditLogger + # audit_logs_dir: + # included_keyspaces: + # excluded_keyspaces: system, system_schema, system_virtual_schema + # included_categories: + # excluded_categories: + # included_users: + # excluded_users: + # roll_cycle: HOURLY + # block: true + # max_queue_weight: 268435456 # 256 MiB + # max_log_size: 17179869184 # 16 GiB + ## archive command is "/path/to/script.sh %path" where %path is replaced with the file being rolled: + # archive_command: + # max_archive_retries: 10 + + + # default options for full query logging - these can be overridden from command line when executing + # nodetool enablefullquerylog + #full_query_logging_options: + # log_dir: + # roll_cycle: HOURLY + # block: true + # max_queue_weight: 268435456 # 256 MiB + # max_log_size: 17179869184 # 16 GiB + ## archive command is "/path/to/script.sh %path" where %path is replaced with the file being rolled: + # archive_command: + # max_archive_retries: 10 + +# validate tombstones on reads and compaction +# can be either "disabled", "warn" or "exception" +# corrupted_tombstone_strategy: disabled + +# Diagnostic Events # +# If enabled, diagnostic events can be helpful for troubleshooting operational issues. Emitted events contain details +# on internal state and temporal relationships across events, accessible by clients via JMX. +diagnostic_events_enabled: false + +# Use native transport TCP message coalescing. If on upgrade to 4.0 you found your throughput decreasing, and in +# particular you run an old kernel or have very fewer client connections, this option might be worth evaluating. +#native_transport_flush_in_batches_legacy: false + +# Enable tracking of repaired state of data during reads and comparison between replicas +# Mismatches between the repaired sets of replicas can be characterized as either confirmed +# or unconfirmed. In this context, unconfirmed indicates that the presence of pending repair +# sessions, unrepaired partition tombstones, or some other condition means that the disparity +# cannot be considered conclusive. Confirmed mismatches should be a trigger for investigation +# as they may be indicative of corruption or data loss. +# There are separate flags for range vs partition reads as single partition reads are only tracked +# when CL > 1 and a digest mismatch occurs. Currently, range queries don't use digests so if +# enabled for range reads, all range reads will include repaired data tracking. As this adds +# some overhead, operators may wish to disable it whilst still enabling it for partition reads +repaired_data_tracking_for_range_reads_enabled: false +repaired_data_tracking_for_partition_reads_enabled: false +# If false, only confirmed mismatches will be reported. If true, a separate metric for unconfirmed +# mismatches will also be recorded. This is to avoid potential signal:noise issues are unconfirmed +# mismatches are less actionable than confirmed ones. +report_unconfirmed_repaired_data_mismatches: false + +# Having many tables and/or keyspaces negatively affects performance of many operations in the +# cluster. When the number of tables/keyspaces in the cluster exceeds the following thresholds +# a client warning will be sent back to the user when creating a table or keyspace. +# table_count_warn_threshold: 150 +# keyspace_count_warn_threshold: 40 + +######################### +# EXPERIMENTAL FEATURES # +######################### + +# Enables materialized view creation on this node. +# Materialized views are considered experimental and are not recommended for production use. +enable_materialized_views: false + +# Enables SASI index creation on this node. +# SASI indexes are considered experimental and are not recommended for production use. +enable_sasi_indexes: false + +# Enables creation of transiently replicated keyspaces on this node. +# Transient replication is experimental and is not recommended for production use. +enable_transient_replication: false + +# Enables the used of 'ALTER ... DROP COMPACT STORAGE' statements on this node. +# 'ALTER ... DROP COMPACT STORAGE' is considered experimental and is not recommended for production use. +enable_drop_compact_storage: false diff --git a/cassandra-5/src/test/resources/docker/Dockerfile b/cassandra-5/src/test/resources/docker/Dockerfile new file mode 100644 index 00000000..28e9aa79 --- /dev/null +++ b/cassandra-5/src/test/resources/docker/Dockerfile @@ -0,0 +1,12 @@ +FROM cassandra:5.0.2 + +ENV CASSANDRA_YAML=/opt/cassandra/conf + +COPY cassandra.yaml $CASSANDRA_YAML + +RUN mkdir -p /var/lib/cassandra/data && \ + chown -R cassandra:cassandra $CASSANDRA_YAML/cassandra.yaml /var/lib/cassandra/data && \ + chmod 777 /var/lib/cassandra/data + +USER cassandra + diff --git a/cassandra-5/src/test/resources/docker/cassandra.yaml b/cassandra-5/src/test/resources/docker/cassandra.yaml new file mode 100644 index 00000000..b4e4ca3e --- /dev/null +++ b/cassandra-5/src/test/resources/docker/cassandra.yaml @@ -0,0 +1,2275 @@ + +# Cassandra storage config YAML + +# NOTE: +# See https://cassandra.apache.org/doc/latest/configuration/ for +# full explanations of configuration directives +# /NOTE + +# NOTE: +# This file is provided in two versions: +# - cassandra.yaml: Contains configuration defaults for a "compatible" +# configuration that operates using settings that are backwards-compatible +# and interoperable with machines running older versions of Cassandra. +# This version is provided to facilitate pain-free upgrades for existing +# users of Cassandra running in production who want to gradually and +# carefully introduce new features. +# - cassandra_latest.yaml: Contains configuration defaults that enable +# the latest features of Cassandra, including improved functionality as +# well as higher performance. This version is provided for new users of +# Cassandra who want to get the most out of their cluster, and for users +# evaluating the technology. +# /NOTE + +# The name of the cluster. This is mainly used to prevent machines in +# one logical cluster from joining another. +cluster_name: 'Test Cluster' + +# This defines the number of tokens randomly assigned to this node on the ring +# The more tokens, relative to other nodes, the larger the proportion of data +# that this node will store. You probably want all nodes to have the same number +# of tokens assuming they have equal hardware capability. +# +# If you leave this unspecified, Cassandra will use the default of 1 token for legacy compatibility, +# and will use the initial_token as described below. +# +# Specifying initial_token will override this setting on the node's initial start, +# on subsequent starts, this setting will apply even if initial token is set. +# +# See https://cassandra.apache.org/doc/latest/getting-started/production.html#tokens for +# best practice information about num_tokens. +# +num_tokens: 16 + +# Triggers automatic allocation of num_tokens tokens for this node. The allocation +# algorithm attempts to choose tokens in a way that optimizes replicated load over +# the nodes in the datacenter for the replica factor. +# +# The load assigned to each node will be close to proportional to its number of +# vnodes. +# +# Only supported with the Murmur3Partitioner. + +# Replica factor is determined via the replication strategy used by the specified +# keyspace. +# allocate_tokens_for_keyspace: KEYSPACE + +# Replica factor is explicitly set, regardless of keyspace or datacenter. +# This is the replica factor within the datacenter, like NTS. +allocate_tokens_for_local_replication_factor: 3 + +# initial_token allows you to specify tokens manually. While you can use it with +# vnodes (num_tokens > 1, above) -- in which case you should provide a +# comma-separated list -- it's primarily used when adding nodes to legacy clusters +# that do not have vnodes enabled. +# initial_token: + +# May either be "true" or "false" to enable globally +hinted_handoff_enabled: true + +# When hinted_handoff_enabled is true, a black list of data centers that will not +# perform hinted handoff +# hinted_handoff_disabled_datacenters: +# - DC1 +# - DC2 + +# this defines the maximum amount of time a dead host will have hints +# generated. After it has been dead this long, new hints for it will not be +# created until it has been seen alive and gone down again. +# Min unit: ms +max_hint_window: 3h + +# Maximum throttle in KiBs per second, per delivery thread. This will be +# reduced proportionally to the number of nodes in the cluster. (If there +# are two nodes in the cluster, each delivery thread will use the maximum +# rate; if there are three, each will throttle to half of the maximum, +# since we expect two nodes to be delivering hints simultaneously.) +# Min unit: KiB +hinted_handoff_throttle: 1024KiB + +# Number of threads with which to deliver hints; +# Consider increasing this number when you have multi-dc deployments, since +# cross-dc handoff tends to be slower +max_hints_delivery_threads: 2 + +# Directory where Cassandra should store hints. +# If not set, the default directory is $CASSANDRA_HOME/data/hints. +hints_directory: /var/lib/cassandra/hints + +# How often hints should be flushed from the internal buffers to disk. +# Will *not* trigger fsync. +# Min unit: ms +hints_flush_period: 10000ms + +# Maximum size for a single hints file, in mebibytes. +# Min unit: MiB +max_hints_file_size: 128MiB + +# The file size limit to store hints for an unreachable host, in mebibytes. +# Once the local hints files have reached the limit, no more new hints will be created. +# Set a non-positive value will disable the size limit. +# max_hints_size_per_host: 0MiB + +# Enable / disable automatic cleanup for the expired and orphaned hints file. +# Disable the option in order to preserve those hints on the disk. +auto_hints_cleanup_enabled: false + +# Enable/disable transfering hints to a peer during decommission. Even when enabled, this does not guarantee +# consistency for logged batches, and it may delay decommission when coupled with a strict hinted_handoff_throttle. +# Default: true +# transfer_hints_on_decommission: true + +# Compression to apply to the hint files. If omitted, hints files +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +#hints_compression: +# - class_name: LZ4Compressor +# parameters: +# - + +# Directory where Cassandra should store results of a One-Shot troubleshooting heapdump for uncaught exceptions. +# Note: this value can be overridden by the -XX:HeapDumpPath JVM env param with a relative local path for testing if +# so desired. +# If not set, the default directory is $CASSANDRA_HOME/heapdump +# heap_dump_path: /var/lib/cassandra/heapdump + +# Enable / disable automatic dump of heap on first uncaught exception +# If not set, the default value is false +# dump_heap_on_uncaught_exception: true + +# Enable / disable persistent hint windows. +# +# If set to false, a hint will be stored only in case a respective node +# that hint is for is down less than or equal to max_hint_window. +# +# If set to true, a hint will be stored in case there is not any +# hint which was stored earlier than max_hint_window. This is for cases +# when a node keeps to restart and hints are not delivered yet, we would be saving +# hints for that node indefinitely. +# +# Defaults to true. +# +# hint_window_persistent_enabled: true + +# Maximum throttle in KiBs per second, total. This will be +# reduced proportionally to the number of nodes in the cluster. +# Min unit: KiB +batchlog_replay_throttle: 1024KiB + +# Strategy to choose the batchlog storage endpoints. +# +# Available options: +# +# - random_remote +# Default, purely random, prevents the local rack, if possible. +# +# - prefer_local +# Similar to random_remote. Random, except that one of the replications will go to the local rack, +# which mean it offers lower availability guarantee than random_remote or dynamic_remote. +# +# - dynamic_remote +# Using DynamicEndpointSnitch to select batchlog storage endpoints, prevents the +# local rack, if possible. This strategy offers the same availability guarantees +# as random_remote but selects the fastest endpoints according to the DynamicEndpointSnitch. +# (DynamicEndpointSnitch currently only tracks reads and not writes - i.e. write-only +# (or mostly-write) workloads might not benefit from this strategy.) +# Note: this strategy will fall back to random_remote, if dynamic_snitch is not enabled. +# +# - dynamic +# Mostly the same as dynamic_remote, except that local rack is not excluded, which mean it offers lower +# availability guarantee than random_remote or dynamic_remote. +# Note: this strategy will fall back to random_remote, if dynamic_snitch is not enabled. +# +# batchlog_endpoint_strategy: random_remote + +# Authentication backend, implementing IAuthenticator; used to identify users +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthenticator, +# PasswordAuthenticator}. +# Optional parameters can be specified in the form of: +# parameters: +# param_key1: param_value1 +# ... +# +# - AllowAllAuthenticator performs no checks - set it to disable authentication. +# - PasswordAuthenticator relies on username/password pairs to authenticate +# users. It keeps usernames and hashed passwords in system_auth.roles table. +# Please increase system_auth keyspace replication factor if you use this authenticator. +# If using PasswordAuthenticator, CassandraRoleManager must also be used (see below) +authenticator: AllowAllAuthenticator +# MutualTlsAuthenticator can be configured using the following configuration. One can add their own validator +# which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates +# and validating certificates. +# class_name : org.apache.cassandra.auth.MutualTlsAuthenticator +# parameters : +# validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator + +# Authorization backend, implementing IAuthorizer; used to limit access/provide permissions +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllAuthorizer, +# CassandraAuthorizer}. +# Optional parameters can be specified in the form of: +# parameters: +# param_key1: param_value1 +# ... +# +# - AllowAllAuthorizer allows any action to any user - set it to disable authorization. +# - CassandraAuthorizer stores permissions in system_auth.role_permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +authorizer: AllowAllAuthorizer + +# Part of the Authentication & Authorization backend, implementing IRoleManager; used +# to maintain grants and memberships between roles. +# Out of the box, Cassandra provides org.apache.cassandra.auth.CassandraRoleManager, +# which stores role information in the system_auth keyspace. Most functions of the +# IRoleManager require an authenticated login, so unless the configured IAuthenticator +# actually implements authentication, most of this functionality will be unavailable. +# Optional parameters can be specified in the form of: +# parameters: +# param_key1: param_value1 +# ... +# +# - CassandraRoleManager stores role data in the system_auth keyspace. Please +# increase system_auth keyspace replication factor if you use this role manager. +role_manager: CassandraRoleManager + +# Network authorization backend, implementing INetworkAuthorizer; used to restrict user +# access to certain DCs +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllNetworkAuthorizer, +# CassandraNetworkAuthorizer}. +# Optional parameters can be specified in the form of: +# parameters: +# param_key1: param_value1 +# ... +# +# - AllowAllNetworkAuthorizer allows access to any DC to any user - set it to disable authorization. +# - CassandraNetworkAuthorizer stores permissions in system_auth.network_permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer. +network_authorizer: AllowAllNetworkAuthorizer + +# CIDR authorization backend, implementing ICIDRAuthorizer; used to restrict user +# access from certain CIDRs +# Out of the box, Cassandra provides org.apache.cassandra.auth.{AllowAllCIDRAuthorizer, +# CassandraCIDRAuthorizer}. +# Optional parameters can be specified in the form of: +# parameters: +# param_key1: param_value1 +# ... +# +# - AllowAllCIDRAuthorizer allows access from any CIDR to any user - set it to disable CIDR authorization. +# - CassandraCIDRAuthorizer stores user's CIDR permissions in system_auth.cidr_permissions table. Please +# increase system_auth keyspace replication factor if you use this authorizer, otherwise any changes to +# system_auth tables being used by this feature may be lost when a host goes down. +cidr_authorizer: + class_name: AllowAllCIDRAuthorizer + # Below parameters are used only when CIDR authorizer is enabled + # parameters: + # CIDR authorizer when enabled, i.e, CassandraCIDRAuthorizer, is applicable for non-superusers only by default. + # Set this setting to true, to enable CIDR authorization for superusers as well. + # Note: CIDR checks cannot be performed for JMX calls + # cidr_checks_for_superusers: true + + # CIDR authorizer when enabled, supports MONITOR and ENFORCE modes. Default mode is MONITOR + # In MONITOR mode, CIDR checks are NOT enforced. Instead, CIDR groups of users accesses are logged using + # nospamlogger. A warning message would be logged if a user accesses from unauthorized CIDR group (but access won't + # be rejected). An info message would be logged otherwise. + # In ENFORCE mode, CIDR checks are enforced, i.e, users accesses would be rejected if attempted from unauthorized + # CIDR groups. + # cidr_authorizer_mode: MONITOR + + # Refresh interval for CIDR groups cache, this value is considered in minutes + # cidr_groups_cache_refresh_interval: 5 + + # Maximum number of entries an IP to CIDR groups cache can accommodate + # ip_cache_max_size: 100 + +# Depending on the auth strategy of the cluster, it can be beneficial to iterate +# from root to table (root -> ks -> table) instead of table to root (table -> ks -> root). +# As the auth entries are whitelisting, once a permission is found you know it to be +# valid. We default to false as the legacy behavior is to query at the table level then +# move back up to the root. See CASSANDRA-17016 for details. +# traverse_auth_from_root: false + +# Validity period for roles cache (fetching granted roles can be an expensive +# operation depending on the role manager, CassandraRoleManager is one example) +# Granted roles are cached for authenticated sessions in AuthenticatedUser and +# after the period specified here, become eligible for (async) reload. +# Defaults to 2000, set to 0 to disable caching entirely. +# Will be disabled automatically for AllowAllAuthenticator. +# For a long-running cache using roles_cache_active_update, consider +# setting to something longer such as a daily validation: 86400000 +# Min unit: ms +roles_validity: 2000ms + +# Refresh interval for roles cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If roles_validity is non-zero, then this must be +# also. +# This setting is also used to inform the interval of auto-updating if +# using roles_cache_active_update. +# Defaults to the same value as roles_validity. +# For a long-running cache, consider setting this to 60000 (1 hour) etc. +# Min unit: ms +# roles_update_interval: 2000ms + +# If true, cache contents are actively updated by a background task at the +# interval set by roles_update_interval. If false, cache entries +# become eligible for refresh after their update interval. Upon next access, +# an async reload is scheduled and the old value returned until it completes. +# roles_cache_active_update: false + +# Validity period for permissions cache (fetching permissions can be an +# expensive operation depending on the authorizer, CassandraAuthorizer is +# one example). Defaults to 2000, set to 0 to disable. +# Will be disabled automatically for AllowAllAuthorizer. +# For a long-running cache using permissions_cache_active_update, consider +# setting to something longer such as a daily validation: 86400000ms +# Min unit: ms +permissions_validity: 2000ms + +# Refresh interval for permissions cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If permissions_validity is non-zero, then this must be +# also. +# This setting is also used to inform the interval of auto-updating if +# using permissions_cache_active_update. +# Defaults to the same value as permissions_validity. +# For a longer-running permissions cache, consider setting to update hourly (60000) +# Min unit: ms +# permissions_update_interval: 2000ms + +# If true, cache contents are actively updated by a background task at the +# interval set by permissions_update_interval. If false, cache entries +# become eligible for refresh after their update interval. Upon next access, +# an async reload is scheduled and the old value returned until it completes. +# permissions_cache_active_update: false + +# Validity period for credentials cache. This cache is tightly coupled to +# the provided PasswordAuthenticator implementation of IAuthenticator. If +# another IAuthenticator implementation is configured, this cache will not +# be automatically used and so the following settings will have no effect. +# Please note, credentials are cached in their encrypted form, so while +# activating this cache may reduce the number of queries made to the +# underlying table, it may not bring a significant reduction in the +# latency of individual authentication attempts. +# Defaults to 2000, set to 0 to disable credentials caching. +# For a long-running cache using credentials_cache_active_update, consider +# setting to something longer such as a daily validation: 86400000 +# Min unit: ms +credentials_validity: 2000ms + +# Refresh interval for credentials cache (if enabled). +# After this interval, cache entries become eligible for refresh. Upon next +# access, an async reload is scheduled and the old value returned until it +# completes. If credentials_validity is non-zero, then this must be +# also. +# This setting is also used to inform the interval of auto-updating if +# using credentials_cache_active_update. +# Defaults to the same value as credentials_validity. +# For a longer-running permissions cache, consider setting to update hourly (60000) +# Min unit: ms +# credentials_update_interval: 2000ms + +# If true, cache contents are actively updated by a background task at the +# interval set by credentials_update_interval. If false (default), cache entries +# become eligible for refresh after their update interval. Upon next access, +# an async reload is scheduled and the old value returned until it completes. +# credentials_cache_active_update: false + +# The partitioner is responsible for distributing groups of rows (by +# partition key) across nodes in the cluster. The partitioner can NOT be +# changed without reloading all data. If you are adding nodes or upgrading, +# you should set this to the same partitioner that you are currently using. +# +# The default partitioner is the Murmur3Partitioner. Older partitioners +# such as the RandomPartitioner, ByteOrderedPartitioner, and +# OrderPreservingPartitioner have been included for backward compatibility only. +# For new clusters, you should NOT change this value. +# +partitioner: org.apache.cassandra.dht.Murmur3Partitioner + +# Directories where Cassandra should store data on disk. If multiple +# directories are specified, Cassandra will spread data evenly across +# them by partitioning the token ranges. +# If not set, the default directory is $CASSANDRA_HOME/data/data. +data_file_directories: + - /var/lib/cassandra/data + +# Directory were Cassandra should store the data of the local system keyspaces. +# By default Cassandra will store the data of the local system keyspaces in the first of the data directories specified +# by data_file_directories. +# This approach ensures that if one of the other disks is lost Cassandra can continue to operate. For extra security +# this setting allows to store those data on a different directory that provides redundancy. +# local_system_data_file_directory: + +# 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 + +# 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 +# containing a CDC-enabled table if at space limit in cdc_raw_directory). +cdc_enabled: true + +# Specify whether writes to the CDC-enabled tables should be blocked when CDC data on disk has reached to the limit. +# When setting to false, the writes will not be blocked and the oldest CDC data on disk will be deleted to +# ensure the size constraint. The default is true. +# cdc_block_writes: true + +# Specify whether CDC mutations are replayed through the write path on streaming, e.g. repair. +# When enabled, CDC data streamed to the destination node will be written into commit log first. When setting to false, +# the streamed CDC data is written into SSTables just the same as normal streaming. The default is true. +# If this is set to false, streaming will be considerably faster however it's possible that, in extreme situations +# (losing > quorum # nodes in a replica set), you may have data in your SSTables that never makes it to the CDC log. +# cdc_on_repair_enabled: true + +# CommitLogSegments are moved to this directory on flush if cdc_enabled: true and the +# 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 + +# Policy for accessing disk: +# +# auto +# Enable mmap on both data and index files on a 64-bit JVM. +# +# standard +# Disable mmap entirely. +# +# mmap +# Map index and data files. mmap can cause excessive paging if all actively read SSTables do not fit into RAM. +# +# mmap_index_only +# Similar to mmap but maps only index files. Using this setting might also help if you observe high number of page +# faults or steals along with increased latencies. This setting is default. +# +# disk_access_mode: mmap_index_only + +# Policy for data disk failures: +# +# die +# shut down gossip and client transports and kill the JVM for any fs errors or +# single-sstable errors, so the node can be replaced. +# +# stop_paranoid +# shut down gossip and client transports even for single-sstable errors, +# kill the JVM for errors during startup. +# +# stop +# shut down gossip and client transports, leaving the node effectively dead, but +# can still be inspected via JMX, kill the JVM for errors during startup. +# +# best_effort +# stop using the failed disk and respond to requests based on +# remaining available sstables. This means you WILL see obsolete +# data at CL.ONE! +# +# ignore +# ignore fatal errors and let requests fail, as in pre-1.2 Cassandra +disk_failure_policy: stop + +# Policy for commit disk failures: +# +# die +# shut down the node and kill the JVM, so the node can be replaced. +# +# stop +# shut down the node, leaving the node effectively dead, but +# can still be inspected via JMX. +# +# stop_commit +# shutdown the commit log, letting writes collect but +# continuing to service reads, as in pre-2.0.5 Cassandra +# +# ignore +# ignore fatal errors and let the batches fail +commit_failure_policy: stop + +# Maximum size of the native protocol prepared statement cache +# +# Valid values are either "auto" (omitting the value) or a value greater 0. +# +# Note that specifying a too large value will result in long running GCs and possbily +# out-of-memory errors. Keep the value at a small fraction of the heap. +# +# If you constantly see "prepared statements discarded in the last minute because +# cache limit reached" messages, the first step is to investigate the root cause +# of these messages and check whether prepared statements are used correctly - +# i.e. use bind markers for variable parts. +# +# Do only change the default value, if you really have more prepared statements than +# fit in the cache. In most cases it is not neccessary to change this value. +# Constantly re-preparing statements is a performance penalty. +# +# Default value ("auto") is 1/256th of the heap or 10MiB, whichever is greater +# Min unit: MiB +prepared_statements_cache_size: + +# Maximum size of the key cache in memory. +# +# Each key cache hit saves 1 seek and each row cache hit saves 2 seeks at the +# minimum, sometimes more. The key cache is fairly tiny for the amount of +# time it saves, so it's worthwhile to use it at large numbers. +# The row cache saves even more time, but must contain the entire row, +# so it is extremely space-intensive. It's best to only use the +# row cache if you have hot rows or static rows. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(5% of Heap (in MiB), 100MiB)). Set to 0 to disable key cache. +# +# This is only relevant to SSTable formats that use key cache, e.g. BIG. +# Min unit: MiB +key_cache_size: + +# Duration in seconds after which Cassandra should +# save the key cache. Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# This is only relevant to SSTable formats that use key cache, e.g. BIG. +# Default is 14400 or 4 hours. +# Min unit: s +key_cache_save_period: 4h + +# Number of keys from the key cache to save +# Disabled by default, meaning all keys are going to be saved +# This is only relevant to SSTable formats that use key cache, e.g. BIG. +# key_cache_keys_to_save: 100 + +# Row cache implementation class name. Available implementations: +# +# org.apache.cassandra.cache.OHCProvider +# Fully off-heap row cache implementation (default). +# +# org.apache.cassandra.cache.SerializingCacheProvider +# This is the row cache implementation available +# in previous releases of Cassandra. +# row_cache_class_name: org.apache.cassandra.cache.OHCProvider + +# Maximum size of the row cache in memory. +# Please note that OHC cache implementation requires some additional off-heap memory to manage +# the map structures and some in-flight memory during operations before/after cache entries can be +# accounted against the cache capacity. This overhead is usually small compared to the whole capacity. +# Do not specify more memory that the system can afford in the worst usual situation and leave some +# headroom for OS block level cache. Do never allow your system to swap. +# +# Default value is 0, to disable row caching. +# Min unit: MiB +row_cache_size: 0MiB + +# Duration in seconds after which Cassandra should save the row cache. +# Caches are saved to saved_caches_directory as specified in this configuration file. +# +# Saved caches greatly improve cold-start speeds, and is relatively cheap in +# terms of I/O for the key cache. Row cache saving is much more expensive and +# has limited use. +# +# Default is 0 to disable saving the row cache. +# Min unit: s +row_cache_save_period: 0s + +# Number of keys from the row cache to save. +# Specify 0 (which is the default), meaning all keys are going to be saved +# row_cache_keys_to_save: 100 + +# Maximum size of the counter cache in memory. +# +# Counter cache helps to reduce counter locks' contention for hot counter cells. +# In case of RF = 1 a counter cache hit will cause Cassandra to skip the read before +# write entirely. With RF > 1 a counter cache hit will still help to reduce the duration +# of the lock hold, helping with hot counter cell updates, but will not allow skipping +# the read entirely. Only the local (clock, count) tuple of a counter cell is kept +# in memory, not the whole counter, so it's relatively cheap. +# +# NOTE: if you reduce the size, you may not get you hottest keys loaded on startup. +# +# Default value is empty to make it "auto" (min(2.5% of Heap (in MiB), 50MiB)). Set to 0 to disable counter cache. +# NOTE: if you perform counter deletes and rely on low gcgs, you should disable the counter cache. +# Min unit: MiB +counter_cache_size: + +# Duration in seconds after which Cassandra should +# save the counter cache (keys only). Caches are saved to saved_caches_directory as +# specified in this configuration file. +# +# Default is 7200 or 2 hours. +# Min unit: s +counter_cache_save_period: 7200s + +# Number of keys from the counter cache to save +# Disabled by default, meaning all keys are going to be saved +# counter_cache_keys_to_save: 100 + +# saved caches +# If not set, the default directory is $CASSANDRA_HOME/data/saved_caches. +saved_caches_directory: /var/lib/cassandra/saved_caches + +# Number of seconds the server will wait for each cache (row, key, etc ...) to load while starting +# the Cassandra process. Setting this to zero is equivalent to disabling all cache loading on startup +# while still having the cache during runtime. +# Min unit: s +# cache_load_timeout: 30s + +# commitlog_sync may be either "periodic", "group", or "batch." +# +# When in batch mode, Cassandra won't ack writes until the commit log +# has been flushed to disk. Each incoming write will trigger the flush task. +# +# group mode is similar to batch mode, where Cassandra will not ack writes +# until the commit log has been flushed to disk. The difference is group +# mode will wait up to commitlog_sync_group_window between flushes. +# +# Min unit: ms +# commitlog_sync_group_window: 1000ms +# +# the default option is "periodic" where writes may be acked immediately +# and the CommitLog is simply synced every commitlog_sync_period +# milliseconds. +commitlog_sync: periodic +# Min unit: ms +commitlog_sync_period: 1000ms + +# When in periodic commitlog mode, the number of milliseconds to block writes +# while waiting for a slow disk flush to complete. +# Min unit: ms +# periodic_commitlog_sync_lag_block: + +# The size of the individual commitlog file segments. A commitlog +# segment may be archived, deleted, or recycled once all the data +# in it (potentially from each columnfamily in the system) has been +# flushed to sstables. +# +# The default size is 32, which is almost always fine, but if you are +# archiving commitlog segments (see commitlog_archiving.properties), +# then you probably want a finer granularity of archiving; 8 or 16 MB +# is reasonable. +# Max mutation size is also configurable via max_mutation_size setting in +# cassandra.yaml. The default is half the size commitlog_segment_size in bytes. +# This should be positive and less than 2048. +# +# NOTE: If max_mutation_size is set explicitly then commitlog_segment_size must +# be set to at least twice the size of max_mutation_size +# +# Min unit: MiB +commitlog_segment_size: 1MiB + +# Compression to apply to the commit log. If omitted, the commit log +# will be written uncompressed. LZ4, Snappy, and Deflate compressors +# are supported. +# commitlog_compression: +# - class_name: LZ4Compressor +# parameters: +# - + +# Set the disk access mode for writing commitlog segments. The allowed values are: +# - auto: version dependent optimal setting +# - legacy: the default mode as used in Cassandra 4.x and earlier (standard I/O when the commitlog is either +# compressed or encrypted or mmap otherwise) +# - mmap: use memory mapped I/O - available only when the commitlog is neither compressed nor encrypted +# - direct: use direct I/O - available only when the commitlog is neither compressed nor encrypted +# - standard: use standard I/O - available only when the commitlog is compressed or encrypted +# The default setting is legacy when the storage compatibility is set to 4 or auto otherwise. +commitlog_disk_access_mode: legacy + +# Compression to apply to SSTables as they flush for compressed tables. +# Note that tables without compression enabled do not respect this flag. +# +# As high ratio compressors like LZ4HC, Zstd, and Deflate can potentially +# block flushes for too long, the default is to flush with a known fast +# compressor in those cases. Options are: +# +# none : Flush without compressing blocks but while still doing checksums. +# fast : Flush with a fast compressor. If the table is already using a +# fast compressor that compressor is used. +# table: Always flush with the same compressor that the table uses. This +# was the pre 4.0 behavior. +# +# flush_compression: fast + +# any class that implements the SeedProvider interface and has a +# constructor that takes a Map of parameters will do. +seed_provider: + # Addresses of hosts that are deemed contact points. + # Cassandra nodes use this list of hosts to find each other and learn + # the topology of the ring. You must change this if you are running + # multiple nodes! + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + # seeds is actually a comma-delimited list of addresses. + # Ex: ",," + - seeds: "127.0.0.1:7000" + # If set to "true", SimpleSeedProvider will return all IP addresses for a DNS name, + # based on the configured name service on the system. Defaults to "false". + # resolve_multiple_ip_addresses_per_dns_record: "false" + +# For workloads with more data than can fit in memory, Cassandra's +# bottleneck will be reads that need to fetch data from +# disk. "concurrent_reads" should be set to (16 * number_of_drives) in +# order to allow the operations to enqueue low enough in the stack +# that the OS and drives can reorder them. Same applies to +# "concurrent_counter_writes", since counter writes read the current +# values before incrementing and writing them back. +# +# On the other hand, since writes are almost never IO bound, the ideal +# number of "concurrent_writes" is dependent on the number of cores in +# your system; (8 * number_of_cores) is a good rule of thumb. +concurrent_reads: 32 +concurrent_writes: 32 +concurrent_counter_writes: 32 + +# For materialized view writes, as there is a read involved, so this should +# be limited by the less of concurrent reads or concurrent writes. +concurrent_materialized_view_writes: 32 + +# Maximum memory to use for inter-node and client-server networking buffers. +# +# Defaults to the smaller of 1/16 of heap or 128MB. This pool is allocated off-heap, +# so is in addition to the memory allocated for heap. The cache also has on-heap +# overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size +# if the default 64k chunk size is used). +# Memory is only allocated when needed. +# Min unit: MiB +# networking_cache_size: 128MiB + +# Enable the sstable chunk cache. The chunk cache will store recently accessed +# sections of the sstable in-memory as uncompressed buffers. +# file_cache_enabled: false + +# Maximum memory to use for sstable chunk cache and buffer pooling. +# 32MB of this are reserved for pooling buffers, the rest is used for chunk cache +# that holds uncompressed sstable chunks. +# Defaults to the smaller of 1/4 of heap or 512MB. This pool is allocated off-heap, +# so is in addition to the memory allocated for heap. The cache also has on-heap +# overhead which is roughly 128 bytes per chunk (i.e. 0.2% of the reserved size +# if the default 64k chunk size is used). +# Memory is only allocated when needed. +# Min unit: MiB +# file_cache_size: 512MiB + +# Flag indicating whether to allocate on or off heap when the sstable buffer +# pool is exhausted, that is when it has exceeded the maximum memory +# file_cache_size, beyond which it will not cache buffers but allocate on request. + +# buffer_pool_use_heap_if_exhausted: true + +# The strategy for optimizing disk read +# Possible values are: +# ssd (for solid state disks, the default) +# spinning (for spinning disks) +# disk_optimization_strategy: ssd + +# Supported memtable implementations and selected default. +# Currently Cassandra offers two memtable implementations: +# - SkipListMemtable is the legacy memtable implementation provided by earlier +# versions of Cassandra. +# - TrieMemtable is a new memtable that utilizes a trie data structure. This +# implementation significantly reduces garbage collection load by moving +# more of the sstable metadata off-heap, fits more data in the same allocation +# and can reliably handle higher write throughput. +# Because the trie memtable is a sharded single-writer solution, it can perform +# worse when the load is very unevenly distributed, e.g. when most of the writes +# access a very small number of partitions or with legacy secondary indexes. +# The memtable implementation can be selected per table by setting memtable +# property in the table definition to one of the configurations specified below. +# If the memtable property is not set, the "default" configuration will be used. +# See src/java/org/apache/cassandra/db/memtable/Memtable_API.md for further +# information. +memtable: + configurations: + skiplist: + class_name: SkipListMemtable + trie: + class_name: TrieMemtable + default: + inherits: skiplist + +# Total permitted memory to use for memtables. Cassandra will stop +# accepting writes when the limit is exceeded until a flush completes, +# and will trigger a flush based on memtable_cleanup_threshold +# If omitted, Cassandra will set both to 1/4 the size of the heap. +# Min unit: MiB +# memtable_heap_space: 2048MiB +# Min unit: MiB +# memtable_offheap_space: 2048MiB + +# memtable_cleanup_threshold is deprecated. The default calculation +# is the only reasonable choice. See the comments on memtable_flush_writers +# for more information. +# +# Ratio of occupied non-flushing memtable size to total permitted size +# that will trigger a flush of the largest memtable. Larger mct will +# mean larger flushes and hence less compaction, but also less concurrent +# flush activity which can make it difficult to keep your disks fed +# under heavy write load. +# +# memtable_cleanup_threshold defaults to 1 / (memtable_flush_writers + 1) +# memtable_cleanup_threshold: 0.11 + +# Specify the way Cassandra allocates and manages memtable memory. +# Options are: +# +# heap_buffers +# on heap nio buffers +# +# offheap_buffers +# off heap (direct) nio buffers +# +# offheap_objects +# off heap objects +memtable_allocation_type: heap_buffers + +# Limit memory usage for Merkle tree calculations during repairs of a certain +# table and common token range. Repair commands targetting multiple tables or +# virtual nodes can exceed this limit depending on concurrent_merkle_tree_requests. +# +# The default is 1/16th of the available heap. The main tradeoff is that +# smaller trees have less resolution, which can lead to over-streaming data. +# If you see heap pressure during repairs, consider lowering this, but you +# cannot go below one mebibyte. If you see lots of over-streaming, consider +# raising this or using subrange repair. +# +# For more details see https://issues.apache.org/jira/browse/CASSANDRA-14096. +# +# Min unit: MiB +# repair_session_space: + +# The number of simultaneous Merkle tree requests during repairs that can +# be performed by a repair command. The size of each validation request is +# limited by the repair_session_space property, so setting this to 1 will make +# sure that a repair command doesn't exceed that limit, even if the repair +# command is repairing multiple tables or multiple virtual nodes. +# +# There isn't a limit by default for backwards compatibility, but this can +# produce OOM for commands repairing multiple tables or multiple virtual nodes. +# A limit of just 1 simultaneous Merkle tree request is generally recommended +# with no virtual nodes so repair_session_space, and thereof the Merkle tree +# resolution, can be high. For virtual nodes a value of 1 with the default +# repair_session_space value will produce higher resolution Merkle trees +# at the expense of speed. Alternatively, when working with virtual nodes it +# can make sense to reduce the repair_session_space and increase the value of +# concurrent_merkle_tree_requests because each range will contain fewer data. +# +# For more details see https://issues.apache.org/jira/browse/CASSANDRA-19336. +# +# A zero value means no limit. +# concurrent_merkle_tree_requests: 0 + +# repair: +# # Configure the retries for each of the repair messages that support it. As of this moment retries use an exponential algorithm where each attempt sleeps longer based off the base_sleep_time and attempt. +# retries: +# max_attempts: 10 +# base_sleep_time: 200ms +# max_sleep_time: 1s +# # Increase the timeout of validation responses due to them containing the merkle tree +# merkle_tree_response: +# base_sleep_time: 30s +# max_sleep_time: 1m + +# Total space to use for commit logs on disk. +# +# If space gets above this value, Cassandra will flush every dirty CF +# in the oldest segment and remove it. So a small total commitlog space +# will tend to cause more flush activity on less-active columnfamilies. +# +# The default value is the smaller of 8192, and 1/4 of the total space +# of the commitlog volume. +# +# commitlog_total_space: 8192MiB + +# This sets the number of memtable flush writer threads per disk +# as well as the total number of memtables that can be flushed concurrently. +# These are generally a combination of compute and IO bound. +# +# Memtable flushing is more CPU efficient than memtable ingest and a single thread +# can keep up with the ingest rate of a whole server on a single fast disk +# until it temporarily becomes IO bound under contention typically with compaction. +# At that point you need multiple flush threads. At some point in the future +# it may become CPU bound all the time. +# +# You can tell if flushing is falling behind using the MemtablePool.BlockedOnAllocation +# metric which should be 0, but will be non-zero if threads are blocked waiting on flushing +# to free memory. +# +# memtable_flush_writers defaults to two for a single data directory. +# This means that two memtables can be flushed concurrently to the single data directory. +# If you have multiple data directories the default is one memtable flushing at a time +# but the flush will use a thread per data directory so you will get two or more writers. +# +# Two is generally enough to flush on a fast disk [array] mounted as a single data directory. +# Adding more flush writers will result in smaller more frequent flushes that introduce more +# compaction overhead. +# +# There is a direct tradeoff between number of memtables that can be flushed concurrently +# and flush size and frequency. More is not better you just need enough flush writers +# to never stall waiting for flushing to free memory. +# +# memtable_flush_writers: 2 + +# Total space to use for change-data-capture logs on disk. +# +# If space gets above this value, Cassandra will throw WriteTimeoutException +# on Mutations including tables with CDC enabled. A CDCCompactor is responsible +# for parsing the raw CDC logs and deleting them when parsing is completed. +# +# The default value is the min of 4096 MiB and 1/8th of the total space +# of the drive where cdc_raw_directory resides. +# Min unit: MiB +# cdc_total_space: 4096MiB + +# When we hit our cdc_raw limit and the CDCCompactor is either running behind +# or experiencing backpressure, we check at the following interval to see if any +# new space for cdc-tracked tables has been made available. Default to 250ms +# Min unit: ms +# cdc_free_space_check_interval: 250ms + +# A fixed memory pool size in MB for for SSTable index summaries. If left +# empty, this will default to 5% of the heap size. If the memory usage of +# all index summaries exceeds this limit, SSTables with low read rates will +# shrink their index summaries in order to meet this limit. However, this +# is a best-effort process. In extreme conditions Cassandra may need to use +# more than this amount of memory. +# Only relevant to formats that use an index summary, e.g. BIG. +# Min unit: KiB +index_summary_capacity: + +# How frequently index summaries should be resampled. This is done +# periodically to redistribute memory from the fixed-size pool to sstables +# proportional their recent read rates. Setting to null value will disable this +# process, leaving existing index summaries at their current sampling level. +# Only relevant to formats that use an index summary, e.g. BIG. +# Min unit: m +index_summary_resize_interval: 60m + +# Whether to, when doing sequential writing, fsync() at intervals in +# order to force the operating system to flush the dirty +# buffers. Enable this to avoid sudden dirty buffer flushing from +# impacting read latencies. Almost always a good idea on SSDs; not +# necessarily on platters. +trickle_fsync: false +# Min unit: KiB +trickle_fsync_interval: 10240KiB + +# TCP port, for commands and data +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +storage_port: 7000 + +# SSL port, for legacy encrypted communication. This property is unused unless enabled in +# server_encryption_options (see below). As of cassandra 4.0, this property is deprecated +# as a single port can be used for either/both secure and insecure connections. +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +ssl_storage_port: 7001 + +# Address or interface to bind to and tell other Cassandra nodes to connect to. +# You _must_ change this if you want multiple nodes to be able to communicate! +# +# Set listen_address OR listen_interface, not both. +# +# Leaving it blank leaves it up to InetAddress.getLocalHost(). This +# will always do the Right Thing _if_ the node is properly configured +# (hostname, name resolution, etc), and the Right Thing is to use the +# address associated with the hostname (it might not be). If unresolvable +# it will fall back to InetAddress.getLoopbackAddress(), which is wrong for production systems. +# +# Setting listen_address to 0.0.0.0 is always wrong. +# +listen_address: 127.0.0.1 + +# Set listen_address OR listen_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# listen_interface: eth0 + +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using listen_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +# listen_interface_prefer_ipv6: false + +# Address to broadcast to other Cassandra nodes +# Leaving this blank will set it to the same value as listen_address +# broadcast_address: 1.2.3.4 + +# When using multiple physical network interfaces, set this +# to true to listen on broadcast_address in addition to +# the listen_address, allowing nodes to communicate in both +# interfaces. +# Ignore this property if the network configuration automatically +# routes between the public and private networks such as EC2. +# listen_on_broadcast_address: false + +# Internode authentication backend, implementing IInternodeAuthenticator; +# used to allow/disallow connections from peer nodes. +#internode_authenticator: +# class_name : org.apache.cassandra.auth.AllowAllInternodeAuthenticator +# parameters : +# MutualTlsInternodeAuthenticator can be configured using the following configuration.One can add their own validator +# which implements MutualTlsCertificateValidator class and provide logic for extracting identity out of certificates +# and validating certificates. +# class_name : org.apache.cassandra.auth.MutualTlsInternodeAuthenticator +# parameters : +# validator_class_name: org.apache.cassandra.auth.SpiffeCertificateValidator +# trusted_peer_identities: "spiffe1,spiffe2" +# node_identity: "spiffe1" +# Whether to start the native transport server. +# The address on which the native transport is bound is defined by rpc_address. +start_native_transport: true +# port for the CQL native transport to listen for clients on +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +native_transport_port: 9042 +# Enabling native transport encryption in client_encryption_options allows you to either use +# encryption for the standard port or to use a dedicated, additional port along with the unencrypted +# standard native_transport_port. +# Enabling client encryption and keeping native_transport_port_ssl disabled will use encryption +# for native_transport_port. Setting native_transport_port_ssl to a different value +# from native_transport_port will use encryption for native_transport_port_ssl while +# keeping native_transport_port unencrypted. +# This feature is deprecated since Cassandra 5.0 and will be removed. Please consult deprecation section in NEWS.txt. +# native_transport_port_ssl: 9142 +# The maximum threads for handling requests (note that idle threads are stopped +# after 30 seconds so there is not corresponding minimum setting). +# native_transport_max_threads: 128 +# +# The maximum size of allowed frame. Frame (requests) larger than this will +# be rejected as invalid. The default is 16MiB. If you're changing this parameter, +# you may want to adjust max_value_size accordingly. This should be positive and less than 2048. +# Min unit: MiB +# native_transport_max_frame_size: 16MiB + +# The maximum number of concurrent client connections. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections: -1 + +# The maximum number of concurrent client connections per source ip. +# The default is -1, which means unlimited. +# native_transport_max_concurrent_connections_per_ip: -1 + +# Controls whether Cassandra honors older, yet currently supported, protocol versions. +# The default is true, which means all supported protocols will be honored. +native_transport_allow_older_protocols: true + +# Controls when idle client connections are closed. Idle connections are ones that had neither reads +# nor writes for a time period. +# +# Clients may implement heartbeats by sending OPTIONS native protocol message after a timeout, which +# will reset idle timeout timer on the server side. To close idle client connections, corresponding +# values for heartbeat intervals have to be set on the client side. +# +# Idle connection timeouts are disabled by default. +# Min unit: ms +# native_transport_idle_timeout: 60000ms + +# When enabled, limits the number of native transport requests dispatched for processing per second. +# Behavior once the limit has been breached depends on the value of THROW_ON_OVERLOAD specified in +# the STARTUP message sent by the client during connection establishment. (See section "4.1.1. STARTUP" +# in "CQL BINARY PROTOCOL v5".) With the THROW_ON_OVERLOAD flag enabled, messages that breach the limit +# are dropped, and an OverloadedException is thrown for the client to handle. When the flag is not +# enabled, the server will stop consuming messages from the channel/socket, putting backpressure on +# the client while already dispatched messages are processed. +# native_transport_rate_limiting_enabled: false +# native_transport_max_requests_per_second: 1000000 + +# The address or interface to bind the native transport server to. +# +# Set rpc_address OR rpc_interface, not both. +# +# Leaving rpc_address blank has the same effect as on listen_address +# (i.e. it will be based on the configured hostname of the node). +# +# Note that unlike listen_address, you can specify 0.0.0.0, but you must also +# set broadcast_rpc_address to a value other than 0.0.0.0. +# +# For security reasons, you should not expose this port to the internet. Firewall it if needed. +rpc_address: 0.0.0.0 + +# Set rpc_address OR rpc_interface, not both. Interfaces must correspond +# to a single address, IP aliasing is not supported. +# rpc_interface: eth1 + +# If you choose to specify the interface by name and the interface has an ipv4 and an ipv6 address +# you can specify which should be chosen using rpc_interface_prefer_ipv6. If false the first ipv4 +# address will be used. If true the first ipv6 address will be used. Defaults to false preferring +# ipv4. If there is only one address it will be selected regardless of ipv4/ipv6. +# rpc_interface_prefer_ipv6: false + +# RPC address to broadcast to drivers and other Cassandra nodes. This cannot +# be set to 0.0.0.0. If left blank, this will be set to the value of +# rpc_address. If rpc_address is set to 0.0.0.0, broadcast_rpc_address must +# be set. +broadcast_rpc_address: 127.0.0.1 + +# enable or disable keepalive on rpc/native connections +rpc_keepalive: true + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# See also: +# /proc/sys/net/core/wmem_max +# /proc/sys/net/core/rmem_max +# /proc/sys/net/ipv4/tcp_wmem +# /proc/sys/net/ipv4/tcp_wmem +# and 'man tcp' +# Min unit: B +# internode_socket_send_buffer_size: + +# Uncomment to set socket buffer size for internode communication +# Note that when setting this, the buffer size is limited by net.core.wmem_max +# and when not setting it it is defined by net.ipv4.tcp_wmem +# Min unit: B +# internode_socket_receive_buffer_size: + +# Set to true to have Cassandra create a hard link to each sstable +# flushed or streamed locally in a backups/ subdirectory of all the +# keyspace data in this node. Removing these links is the operator's +# responsibility. The operator can also turn off incremental backups +# for specified table by setting table parameter incremental_backups to +# false, which is set to true by default. See CASSANDRA-15402 +incremental_backups: false + +# Whether or not to take a snapshot before each compaction. Be +# careful using this option, since Cassandra won't clean up the +# snapshots for you. Mostly useful if you're paranoid when there +# is a data format change. +snapshot_before_compaction: false + +# Whether or not a snapshot is taken of the data before keyspace truncation +# or dropping of column families. The STRONGLY advised default of true +# should be used to provide data safety. If you set this flag to false, you will +# lose data on truncation or drop. +auto_snapshot: true + +# Adds a time-to-live (TTL) to auto snapshots generated by table +# truncation or drop (when enabled). +# After the TTL is elapsed, the snapshot is automatically cleared. +# By default, auto snapshots *do not* have TTL, uncomment the property below +# to enable TTL on auto snapshots. +# Accepted units: d (days), h (hours) or m (minutes) +# auto_snapshot_ttl: 30d + +# The act of creating or clearing a snapshot involves creating or removing +# potentially tens of thousands of links, which can cause significant performance +# impact, especially on consumer grade SSDs. A non-zero value here can +# be used to throttle these links to avoid negative performance impact of +# taking and clearing snapshots +snapshot_links_per_second: 0 + +# The sstable formats configuration. SSTable formats implementations are +# loaded using the service loader mechanism. In this section, one can select +# the format for created sstables and pass additional parameters for the formats +# available on the classpath. +# The default format is "big", the legacy SSTable format in use since Cassandra 3.0. +# Cassandra versions 5.0 and later also support the trie-indexed "bti" format, +# which offers better performance. +#sstable: +# selected_format: big + +# Granularity of the collation index of rows within a partition. +# Applies to both BIG and BTI SSTable formats. In both formats, +# a smaller granularity results in faster lookup of rows within +# a partition, but a bigger index file size. +# Using smaller granularities with the BIG format is not recommended +# because bigger collation indexes cannot be cached efficiently +# or at all if they become sufficiently large. Further, if +# large rows, or a very large number of rows per partition are +# present, it is recommended to increase the index granularity +# or switch to the BTI SSTable format. +# +# Leave undefined to use a default suitable for the SSTable format +# in use (64 KiB for BIG, 16KiB for BTI). +# Min unit: KiB +# column_index_size: 4KiB + +# Per sstable indexed key cache entries (the collation index in memory +# mentioned above) exceeding this size will not be held on heap. +# This means that only partition information is held on heap and the +# index entries are read from disk. +# +# Note that this size refers to the size of the +# serialized index information and not the size of the partition. +# +# This is only relevant to SSTable formats that use key cache, e.g. BIG. +# Min unit: KiB +column_index_cache_size: 2KiB + +# Default compaction strategy, applied when a table's parameters do not +# specify compaction. +# The selected compaction strategy will also apply to system tables. +# +# If no value is specified, the default is to use SizeTieredCompactionStrategy, +# with its default compaction parameters. +# +# default_compaction: +# class_name: SizeTieredCompactionStrategy +# parameters: +# min_threshold: 4 +# max_threshold: 32 + + +# Number of simultaneous compactions to allow, NOT including +# validation "compactions" for anti-entropy repair. Simultaneous +# compactions can help preserve read performance in a mixed read/write +# workload, by mitigating the tendency of small sstables to accumulate +# during a single long running compactions. The default is usually +# fine and if you experience problems with compaction running too +# slowly or too fast, you should look at +# compaction_throughput first. +# +# concurrent_compactors defaults to the smaller of (number of disks, +# number of cores), with a minimum of 2 and a maximum of 8. +# +# If your data directories are backed by SSD, you should increase this +# to the number of cores. +# concurrent_compactors: 1 + +# Number of simultaneous repair validations to allow. If not set or set to +# a value less than 1, it defaults to the value of concurrent_compactors. +# To set a value greeater than concurrent_compactors at startup, the system +# property cassandra.allow_unlimited_concurrent_validations must be set to +# true. To dynamically resize to a value > concurrent_compactors on a running +# node, first call the bypassConcurrentValidatorsLimit method on the +# org.apache.cassandra.db:type=StorageService mbean +# concurrent_validations: 0 + +# Number of simultaneous materialized view builder tasks to allow. +concurrent_materialized_view_builders: 1 + +# Throttles compaction to the given total throughput across the entire +# system. The faster you insert data, the faster you need to compact in +# order to keep the sstable count down, but in general, setting this to +# 16 to 32 times the rate you are inserting data is more than sufficient. +# Setting this to 0 disables throttling. Note that this accounts for all types +# of compaction, including validation compaction (building Merkle trees +# for repairs). +compaction_throughput: 64MiB/s + +# When compacting, the replacement sstable(s) can be opened before they +# are completely written, and used in place of the prior sstables for +# any range that has been written. This helps to smoothly transfer reads +# between the sstables, reducing page cache churn and keeping hot rows hot +# Set sstable_preemptive_open_interval to null for disabled which is equivalent to +# sstable_preemptive_open_interval_in_mb being negative +# Min unit: MiB +sstable_preemptive_open_interval: 50MiB + +# Starting from 4.1 sstables support UUID based generation identifiers. They are disabled by default +# because once enabled, there is no easy way to downgrade. When the node is restarted with this option +# set to true, each newly created sstable will have a UUID based generation identifier and such files are +# not readable by previous Cassandra versions. At some point, this option will become true by default +# and eventually get removed from the configuration. +uuid_sstable_identifiers_enabled: false + +# When enabled, permits Cassandra to zero-copy stream entire eligible +# SSTables between nodes, including every component. +# This speeds up the network transfer significantly subject to +# throttling specified by entire_sstable_stream_throughput_outbound, +# and entire_sstable_inter_dc_stream_throughput_outbound +# for inter-DC transfers. +# Enabling this will reduce the GC pressure on sending and receiving node. +# When unset, the default is enabled. While this feature tries to keep the +# disks balanced, it cannot guarantee it. This feature will be automatically +# disabled if internode encryption is enabled. +# stream_entire_sstables: true + +# Throttles entire SSTable outbound streaming file transfers on +# this node to the given total throughput in Mbps. +# Setting this value to 0 it disables throttling. +# When unset, the default is 200 Mbps or 24 MiB/s. +# entire_sstable_stream_throughput_outbound: 24MiB/s + +# Throttles entire SSTable file streaming between datacenters. +# Setting this value to 0 disables throttling for entire SSTable inter-DC file streaming. +# When unset, the default is 200 Mbps or 24 MiB/s. +# entire_sstable_inter_dc_stream_throughput_outbound: 24MiB/s + +# Throttles all outbound streaming file transfers on this node to the +# given total throughput in Mbps. This is necessary because Cassandra does +# mostly sequential IO when streaming data during bootstrap or repair, which +# can lead to saturating the network connection and degrading rpc performance. +# When unset, the default is 200 Mbps or 24 MiB/s. +# stream_throughput_outbound: 24MiB/s + +# Throttles all streaming file transfer between the datacenters, +# this setting allows users to throttle inter dc stream throughput in addition +# to throttling all network stream traffic as configured with +# stream_throughput_outbound_megabits_per_sec +# When unset, the default is 200 Mbps or 24 MiB/s. +# inter_dc_stream_throughput_outbound: 24MiB/s + +# Server side timeouts for requests. The server will return a timeout exception +# to the client if it can't complete an operation within the corresponding +# timeout. Those settings are a protection against: +# 1) having client wait on an operation that might never terminate due to some +# failures. +# 2) operations that use too much CPU/read too much data (leading to memory build +# up) by putting a limit to how long an operation will execute. +# For this reason, you should avoid putting these settings too high. In other words, +# if you are timing out requests because of underlying resource constraints then +# increasing the timeout will just cause more problems. Of course putting them too +# low is equally ill-advised since clients could get timeouts even for successful +# operations just because the timeout setting is too tight. + +# How long the coordinator should wait for read operations to complete. +# Lowest acceptable value is 10 ms. +# Min unit: ms +read_request_timeout: 5000ms +# How long the coordinator should wait for seq or index scans to complete. +# Lowest acceptable value is 10 ms. +# Min unit: ms +range_request_timeout: 10000ms +# How long the coordinator should wait for writes to complete. +# Lowest acceptable value is 10 ms. +# Min unit: ms +write_request_timeout: 2000ms +# How long the coordinator should wait for counter writes to complete. +# Lowest acceptable value is 10 ms. +# Min unit: ms +counter_write_request_timeout: 5000ms +# How long a coordinator should continue to retry a CAS operation +# that contends with other proposals for the same row. +# Lowest acceptable value is 10 ms. +# Min unit: ms +cas_contention_timeout: 1000ms +# How long the coordinator should wait for truncates to complete +# (This can be much longer, because unless auto_snapshot is disabled +# we need to flush first so we can snapshot before removing the data.) +# Lowest acceptable value is 10 ms. +# Min unit: ms +truncate_request_timeout: 60000ms +# The default timeout for other, miscellaneous operations. +# Lowest acceptable value is 10 ms. +# Min unit: ms +request_timeout: 10000ms + +# Defensive settings for protecting Cassandra from true network partitions. +# See (CASSANDRA-14358) for details. +# +# The amount of time to wait for internode tcp connections to establish. +# Min unit: ms +# internode_tcp_connect_timeout: 2000ms +# +# The amount of time unacknowledged data is allowed on a connection before we throw out the connection +# Note this is only supported on Linux + epoll, and it appears to behave oddly above a setting of 30000 +# (it takes much longer than 30s) as of Linux 4.12. If you want something that high set this to 0 +# which picks up the OS default and configure the net.ipv4.tcp_retries2 sysctl to be ~8. +# Min unit: ms +# internode_tcp_user_timeout: 30000ms + +# The amount of time unacknowledged data is allowed on a streaming connection. +# The default is 5 minutes. Increase it or set it to 0 in order to increase the timeout. +# Min unit: ms +# internode_streaming_tcp_user_timeout: 300000ms + +# Global, per-endpoint and per-connection limits imposed on messages queued for delivery to other nodes +# and waiting to be processed on arrival from other nodes in the cluster. These limits are applied to the on-wire +# size of the message being sent or received. +# +# The basic per-link limit is consumed in isolation before any endpoint or global limit is imposed. +# Each node-pair has three links: urgent, small and large. So any given node may have a maximum of +# N*3*(internode_application_send_queue_capacity+internode_application_receive_queue_capacity) +# messages queued without any coordination between them although in practice, with token-aware routing, only RF*tokens +# nodes should need to communicate with significant bandwidth. +# +# The per-endpoint limit is imposed on all messages exceeding the per-link limit, simultaneously with the global limit, +# on all links to or from a single node in the cluster. +# The global limit is imposed on all messages exceeding the per-link limit, simultaneously with the per-endpoint limit, +# on all links to or from any node in the cluster. +# +# Min unit: B +# internode_application_send_queue_capacity: 4MiB +# internode_application_send_queue_reserve_endpoint_capacity: 128MiB +# internode_application_send_queue_reserve_global_capacity: 512MiB +# internode_application_receive_queue_capacity: 4MiB +# internode_application_receive_queue_reserve_endpoint_capacity: 128MiB +# internode_application_receive_queue_reserve_global_capacity: 512MiB + + +# How long before a node logs slow queries. Select queries that take longer than +# this timeout to execute, will generate an aggregated log message, so that slow queries +# can be identified. Set this value to zero to disable slow query logging. +# Min unit: ms +slow_query_log_timeout: 500ms + +# Enable operation timeout information exchange between nodes to accurately +# measure request timeouts. If disabled, replicas will assume that requests +# were forwarded to them instantly by the coordinator, which means that +# under overload conditions we will waste that much extra time processing +# already-timed-out requests. +# +# Warning: It is generally assumed that users have setup NTP on their clusters, and that clocks are modestly in sync, +# since this is a requirement for general correctness of last write wins. +# internode_timeout: true + +# Set period for idle state control messages for earlier detection of failed streams +# This node will send a keep-alive message periodically on the streaming's control channel. +# This ensures that any eventual SocketTimeoutException will occur within 2 keep-alive cycles +# If the node cannot send, or timeouts sending, the keep-alive message on the netty control channel +# the stream session is closed. +# Default value is 300s (5 minutes), which means stalled streams +# are detected within 10 minutes +# Specify 0 to disable. +# Min unit: s +# streaming_keep_alive_period: 300s + +# Limit number of connections per host for streaming +# Increase this when you notice that joins are CPU-bound rather that network +# bound (for example a few nodes with big files). +# streaming_connections_per_host: 1 + +# Settings for stream stats tracking; used by system_views.streaming table +# How long before a stream is evicted from tracking; this impacts both historic and currently running +# streams. +# streaming_state_expires: 3d +# How much memory may be used for tracking before evicting session from tracking; once crossed +# historic and currently running streams maybe impacted. +# streaming_state_size: 40MiB +# Enable/Disable tracking of streaming stats +# streaming_stats_enabled: true + +# Allows denying configurable access (rw/rr) to operations on configured ks, table, and partitions, intended for use by +# operators to manage cluster health vs application access. See CASSANDRA-12106 and CEP-13 for more details. +# partition_denylist_enabled: false + +# denylist_writes_enabled: true +# denylist_reads_enabled: true +# denylist_range_reads_enabled: true + +# The interval at which keys in the cache for denylisting will "expire" and async refresh from the backing DB. +# Note: this serves only as a fail-safe, as the usage pattern is expected to be "mutate state, refresh cache" on any +# changes to the underlying denylist entries. See documentation for details. +# Min unit: s +# denylist_refresh: 600s + +# In the event of errors on attempting to load the denylist cache, retry on this interval. +# Min unit: s +# denylist_initial_load_retry: 5s + +# We cap the number of denylisted keys allowed per table to keep things from growing unbounded. Nodes will warn above +# this limit while allowing new denylisted keys to be inserted. Denied keys are loaded in natural query / clustering +# ordering by partition key in case of overflow. +# denylist_max_keys_per_table: 1000 + +# We cap the total number of denylisted keys allowed in the cluster to keep things from growing unbounded. +# Nodes will warn on initial cache load that there are too many keys and be direct the operator to trim down excess +# entries to within the configured limits. +# denylist_max_keys_total: 10000 + +# Since the denylist in many ways serves to protect the health of the cluster from partitions operators have identified +# as being in a bad state, we usually want more robustness than just CL.ONE on operations to/from these tables to +# ensure that these safeguards are in place. That said, we allow users to configure this if they're so inclined. +# denylist_consistency_level: QUORUM + +# phi value that must be reached for a host to be marked down. +# most users should never need to adjust this. +# phi_convict_threshold: 8 + +# endpoint_snitch -- Set this to a class that implements +# IEndpointSnitch. The snitch has two functions: +# +# - it teaches Cassandra enough about your network topology to route +# requests efficiently +# - it allows Cassandra to spread replicas around your cluster to avoid +# correlated failures. It does this by grouping machines into +# "datacenters" and "racks." Cassandra will do its best not to have +# more than one replica on the same "rack" (which may not actually +# be a physical location) +# +# CASSANDRA WILL NOT ALLOW YOU TO SWITCH TO AN INCOMPATIBLE SNITCH +# ONCE DATA IS INSERTED INTO THE CLUSTER. This would cause data loss. +# This means that if you start with the default SimpleSnitch, which +# locates every node on "rack1" in "datacenter1", your only options +# if you need to add another datacenter are GossipingPropertyFileSnitch +# (and the older PFS). From there, if you want to migrate to an +# incompatible snitch like Ec2Snitch you can do it by adding new nodes +# under Ec2Snitch (which will locate them in a new "datacenter") and +# decommissioning the old ones. +# +# Out of the box, Cassandra provides: +# +# SimpleSnitch: +# Treats Strategy order as proximity. This can improve cache +# locality when disabling read repair. Only appropriate for +# single-datacenter deployments. +# +# GossipingPropertyFileSnitch +# This should be your go-to snitch for production use. The rack +# and datacenter for the local node are defined in +# cassandra-rackdc.properties and propagated to other nodes via +# gossip. If cassandra-topology.properties exists, it is used as a +# fallback, allowing migration from the PropertyFileSnitch. +# +# PropertyFileSnitch: +# Proximity is determined by rack and data center, which are +# explicitly configured in cassandra-topology.properties. +# +# AlibabaCloudSnitch: +# Snitch for getting dc and rack of a node from metadata service of Alibaba cloud. +# This snitch that assumes an ECS region is a DC and an ECS availability_zone is a rack. +# +# AzureSnitch: +# Gets datacenter from 'location' and rack from 'zone' fields of 'compute' object +# from instance metadata service. If the availability zone is not enabled, it will use the fault +# domain and get its respective value. +# +# CloudstackSnitch: +# A snitch that assumes a Cloudstack Zone follows the typical convention +# country-location-az and uses a country/location tuple as a datacenter +# and the availability zone as a rack. +# WARNING: This snitch is deprecated and it is scheduled to be removed +# in the next major version of Cassandra. +# +# Ec2Snitch: +# Appropriate for EC2 deployments in a single Region. Loads Region +# and Availability Zone information from the EC2 API. The Region is +# treated as the datacenter, and the Availability Zone as the rack. +# Only private IPs are used, so this will not work across multiple +# Regions. +# +# Ec2MultiRegionSnitch: +# Uses public IPs as broadcast_address to allow cross-region +# connectivity. (Thus, you should set seed addresses to the public +# IP as well.) You will need to open the storage_port or +# ssl_storage_port on the public IP firewall. (For intra-Region +# traffic, Cassandra will switch to the private IP after +# establishing a connection.) +# +# GoogleCloudSnitch: +# Snitch for getting dc and rack of a node from metadata service of Google cloud. +# This snitch that assumes an GCE region is a DC and an GCE availability_zone is a rack. +# +# RackInferringSnitch: +# Proximity is determined by rack and data center, which are +# assumed to correspond to the 3rd and 2nd octet of each node's IP +# address, respectively. Unless this happens to match your +# deployment conventions, this is best used as an example of +# writing a custom Snitch class and is provided in that spirit. +# +# You can use a custom Snitch by setting this to the full class name +# of the snitch, which will be assumed to be on your classpath. +endpoint_snitch: SimpleSnitch + +# controls how often to perform the more expensive part of host score +# calculation +# Min unit: ms +dynamic_snitch_update_interval: 100ms +# controls how often to reset all host scores, allowing a bad host to +# possibly recover +# Min unit: ms +dynamic_snitch_reset_interval: 600000ms +# if set greater than zero, this will allow +# 'pinning' of replicas to hosts in order to increase cache capacity. +# The badness threshold will control how much worse the pinned host has to be +# before the dynamic snitch will prefer other replicas over it. This is +# expressed as a double which represents a percentage. Thus, a value of +# 0.2 means Cassandra would continue to prefer the static snitch values +# until the pinned host was 20% worse than the fastest. +dynamic_snitch_badness_threshold: 1.0 + +# Configures Java crypto provider. By default, it will use DefaultCryptoProvider +# which will install Amazon Correto Crypto Provider. +# +# Amazon Correto Crypto Provider works currently for x86_64 and aarch_64 platforms. +# If this provider fails it will fall back to the default crypto provider in the JRE. +# +# To force failure when the provider was not installed properly, set the property "fail_on_missing_provider" to "true". +# +# To bypass the installation of a crypto provider use class 'org.apache.cassandra.security.JREProvider' +# +crypto_provider: + - class_name: org.apache.cassandra.security.DefaultCryptoProvider + parameters: + - fail_on_missing_provider: "false" + +# Configure server-to-server internode encryption +# +# JVM and netty defaults for supported SSL socket protocols and cipher suites can +# be replaced using custom encryption options. This is not recommended +# unless you have policies in place that dictate certain settings, or +# need to disable vulnerable ciphers or protocols in case the JVM cannot +# be updated. +# +# FIPS compliant settings can be configured at JVM level and should not +# involve changing encryption settings here: +# https://docs.oracle.com/javase/8/docs/technotes/guides/security/jsse/FIPS.html +# +# **NOTE** this default configuration is an insecure configuration. If you need to +# enable server-to-server encryption generate server keystores (and truststores for mutual +# authentication) per: +# http://download.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# Then perform the following configuration changes: +# +# Step 1: Set internode_encryption= and explicitly set optional=true. Restart all nodes +# +# Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual +# auth set require_client_auth=true. Restart all nodes +server_encryption_options: + # On outbound connections, determine which type of peers to securely connect to. + # The available options are : + # none : Do not encrypt outgoing connections + # dc : Encrypt connections to peers in other datacenters but not within datacenters + # rack : Encrypt connections to peers in other racks but not within racks + # all : Always use encrypted connections + internode_encryption: none + # When set to true, encrypted and unencrypted connections are allowed on the storage_port + # This should _only be true_ while in unencrypted or transitional operation + # optional defaults to true if internode_encryption is none + # optional: true + # If enabled, will open up an encrypted listening socket on ssl_storage_port. Should only be used + # during upgrade to 4.0; otherwise, set to false. + legacy_ssl_storage_port_enabled: false + # Set to a valid keystore if internode_encryption is dc, rack or all + keystore: conf/.keystore + #keystore_password: cassandra + # Configure the way Cassandra creates SSL contexts. + # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory + # ssl_context_factory: + # # Must be an instance of org.apache.cassandra.security.ISslContextFactory + # class_name: org.apache.cassandra.security.DefaultSslContextFactory + # During internode mTLS authentication, inbound connections (acting as servers) use keystore, keystore_password + # containing server certificate to create SSLContext and + # outbound connections (acting as clients) use outbound_keystore & outbound_keystore_password with client certificates + # to create SSLContext. By default, outbound_keystore is the same as keystore indicating mTLS is not enabled. + # outbound_keystore: conf/.keystore + # outbound_keystore_password: cassandra + # Verify peer server certificates + require_client_auth: false + # Set to a valid trustore if require_client_auth is true + truststore: conf/.truststore + #truststore_password: cassandra + # Verify that the host name in the certificate matches the connected host + require_endpoint_verification: false + # More advanced defaults: + # protocol: TLS + # store_type: JKS + # cipher_suites: [ + # TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256, + # TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA, + # TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA, + # TLS_RSA_WITH_AES_256_CBC_SHA + # ] + +# Configure client-to-server encryption. +# +# **NOTE** this default configuration is an insecure configuration. If you need to +# enable client-to-server encryption generate server keystores (and truststores for mutual +# authentication) per: +# http://download.oracle.com/javase/8/docs/technotes/guides/security/jsse/JSSERefGuide.html#CreateKeystore +# Then perform the following configuration changes: +# +# Step 1: Set enabled=true and explicitly set optional=true. Restart all nodes +# +# Step 2: Set optional=false (or remove it) and if you generated truststores and want to use mutual +# auth set require_client_auth=true. Restart all nodes +client_encryption_options: + # Enable client-to-server encryption + enabled: false + # When set to true, encrypted and unencrypted connections are allowed on the native_transport_port + # This should _only be true_ while in unencrypted or transitional operation + # optional defaults to true when enabled is false, and false when enabled is true. + # optional: true + # Set keystore and keystore_password to valid keystores if enabled is true + keystore: conf/.keystore + #keystore_password: cassandra + # Configure the way Cassandra creates SSL contexts. + # To use PEM-based key material, see org.apache.cassandra.security.PEMBasedSslContextFactory + # ssl_context_factory: + # # Must be an instance of org.apache.cassandra.security.ISslContextFactory + # class_name: org.apache.cassandra.security.DefaultSslContextFactory + # Verify client certificates + require_client_auth: false + # require_endpoint_verification: false + # Set trustore and truststore_password if require_client_auth is true + # truststore: conf/.truststore + # truststore_password: cassandra + # More advanced defaults: + # protocol: TLS + # store_type: JKS + # cipher_suites: [ + # TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384, TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256, + # TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256, TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA, + # TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA, TLS_RSA_WITH_AES_128_GCM_SHA256, TLS_RSA_WITH_AES_128_CBC_SHA, + # TLS_RSA_WITH_AES_256_CBC_SHA + # ] + +# internode_compression controls whether traffic between nodes is +# compressed. +# Can be: +# +# all +# all traffic is compressed +# +# dc +# traffic between different datacenters is compressed +# +# none +# nothing is compressed. +internode_compression: dc + +# Enable or disable tcp_nodelay for inter-dc communication. +# Disabling it will result in larger (but fewer) network packets being sent, +# reducing overhead from the TCP protocol itself, at the cost of increasing +# latency if you block for cross-datacenter responses. +inter_dc_tcp_nodelay: false + +# TTL for different trace types used during logging of the repair process. +# Min unit: s +trace_type_query_ttl: 1d +# Min unit: s +trace_type_repair_ttl: 7d + +# If unset, all GC Pauses greater than gc_log_threshold will log at +# INFO level +# UDFs (user defined functions) are disabled by default. +# As of Cassandra 3.0 there is a sandbox in place that should prevent execution of evil code. +user_defined_functions_enabled: false + +# Enables encrypting data at-rest (on disk). Different key providers can be plugged in, but the default reads from +# a JCE-style keystore. A single keystore can hold multiple keys, but the one referenced by +# the "key_alias" is the only key that will be used for encrypt opertaions; previously used keys +# can still (and should!) be in the keystore and will be used on decrypt operations +# (to handle the case of key rotation). +# +# It is strongly recommended to download and install Java Cryptography Extension (JCE) +# Unlimited Strength Jurisdiction Policy Files for your version of the JDK. +# (current link: http://www.oracle.com/technetwork/java/javase/downloads/jce8-download-2133166.html) +# +# Currently, only the following file types are supported for transparent data encryption, although +# more are coming in future cassandra releases: commitlog, hints +transparent_data_encryption_options: + enabled: false + chunk_length_kb: 64 + cipher: AES/CBC/PKCS5Padding + key_alias: testing:1 + # CBC IV length for AES needs to be 16 bytes (which is also the default size) + # iv_length: 16 + key_provider: + - class_name: org.apache.cassandra.security.JKSKeyProvider + parameters: + - keystore: conf/.keystore + keystore_password: cassandra + store_type: JCEKS + key_password: cassandra + + # Storage Attached Indexing options. + # sai_options: + ## Total permitted memory allowed for writing SAI index segments. This memory + ## is split between all SAI indexes being built so more indexes will mean smaller + ## segment sizes. + # segment_write_buffer_size: 1024MiB + +##################### +# SAFETY THRESHOLDS # +##################### + +# When executing a scan, within or across a partition, we need to keep the +# tombstones seen in memory so we can return them to the coordinator, which +# will use them to make sure other replicas also know about the deleted rows. +# With workloads that generate a lot of tombstones, this can cause performance +# problems and even exaust the server heap. +# (http://www.datastax.com/dev/blog/cassandra-anti-patterns-queues-and-queue-like-datasets) +# Adjust the thresholds here if you understand the dangers and want to +# scan more tombstones anyway. These thresholds may also be adjusted at runtime +# using the StorageService mbean. +tombstone_warn_threshold: 1000 +tombstone_failure_threshold: 100000 + +# Filtering and secondary index queries at read consistency levels above ONE/LOCAL_ONE use a +# mechanism called replica filtering protection to ensure that results from stale replicas do +# not violate consistency. (See CASSANDRA-8272 and CASSANDRA-15907 for more details.) This +# mechanism materializes replica results by partition on-heap at the coordinator. The more possibly +# stale results returned by the replicas, the more rows materialized during the query. +replica_filtering_protection: + # These thresholds exist to limit the damage severely out-of-date replicas can cause during these + # queries. They limit the number of rows from all replicas individual index and filtering queries + # can materialize on-heap to return correct results at the desired read consistency level. + # + # "cached_replica_rows_warn_threshold" is the per-query threshold at which a warning will be logged. + # "cached_replica_rows_fail_threshold" is the per-query threshold at which the query will fail. + # + # These thresholds may also be adjusted at runtime using the StorageService mbean. + # + # If the failure threshold is breached, it is likely that either the current page/fetch size + # is too large or one or more replicas is severely out-of-sync and in need of repair. + cached_rows_warn_threshold: 2000 + cached_rows_fail_threshold: 32000 + +# Log WARN on any multiple-partition batch size exceeding this value. 5KiB per batch by default. +# Caution should be taken on increasing the size of this threshold as it can lead to node instability. +# Min unit: KiB +batch_size_warn_threshold: 5KiB + +# Fail any multiple-partition batch exceeding this value. 50KiB (10x warn threshold) by default. +# Min unit: KiB +batch_size_fail_threshold: 50KiB + +# Log WARN on any batches not of type LOGGED than span across more partitions than this limit +unlogged_batch_across_partitions_warn_threshold: 10 + +# GC Pauses greater than 200 ms will be logged at INFO level +# This threshold can be adjusted to minimize logging if necessary +# Min unit: ms +# gc_log_threshold: 200ms + +# GC Pauses greater than gc_warn_threshold will be logged at WARN level +# Adjust the threshold based on your application throughput requirement. Setting to 0 +# will deactivate the feature. +# Min unit: ms +# gc_warn_threshold: 1000ms + +# Maximum size of any value in SSTables. Safety measure to detect SSTable corruption +# early. Any value size larger than this threshold will result into marking an SSTable +# as corrupted. This should be positive and less than 2GiB. +# Min unit: MiB +# max_value_size: 256MiB + +# ** Impact on keyspace creation ** +# If replication factor is not mentioned as part of keyspace creation, default_keyspace_rf would apply. +# Changing this configuration would only take effect for keyspaces created after the change, but does not impact +# existing keyspaces created prior to the change. +# ** Impact on keyspace alter ** +# When altering a keyspace from NetworkTopologyStrategy to SimpleStrategy, default_keyspace_rf is applied if rf is not +# explicitly mentioned. +# ** Impact on system keyspaces ** +# This would also apply for any system keyspaces that need replication factor. +# A further note about system keyspaces - system_traces and system_distributed keyspaces take RF of 2 or default, +# whichever is higher, and system_auth keyspace takes RF of 1 or default, whichever is higher. +# Suggested value for use in production: 3 +# default_keyspace_rf: 1 + +# Track a metric per keyspace indicating whether replication achieved the ideal consistency +# level for writes without timing out. This is different from the consistency level requested by +# each write which may be lower in order to facilitate availability. +# ideal_consistency_level: EACH_QUORUM + +# Automatically upgrade sstables after upgrade - if there is no ordinary compaction to do, the +# oldest non-upgraded sstable will get upgraded to the latest version +# automatic_sstable_upgrade: false +# Limit the number of concurrent sstable upgrades +# max_concurrent_automatic_sstable_upgrades: 1 + +# Audit logging - Logs every incoming CQL command request, authentication to a node. See the docs +# on audit_logging for full details about the various configuration options and production tips. +audit_logging_options: + enabled: false + logger: + - class_name: BinAuditLogger + # audit_logs_dir: + # included_keyspaces: + # excluded_keyspaces: system, system_schema, system_virtual_schema + # included_categories: + # excluded_categories: + # included_users: + # excluded_users: + # roll_cycle: HOURLY + # block: true + # max_queue_weight: 268435456 # 256 MiB + # max_log_size: 17179869184 # 16 GiB + # + ## If archive_command is empty or unset, Cassandra uses a built-in DeletingArchiver that deletes the oldest files if ``max_log_size`` is reached. + ## If archive_command is set, Cassandra does not use DeletingArchiver, so it is the responsibility of the script to make any required cleanup. + ## Example: "/path/to/script.sh %path" where %path is replaced with the file being rolled. + # archive_command: + # max_archive_retries: 10 + + # default options for full query logging - these can be overridden from command line when executing + # nodetool enablefullquerylog + # full_query_logging_options: + # log_dir: + # roll_cycle: HOURLY + # block: true + # max_queue_weight: 268435456 # 256 MiB + # max_log_size: 17179869184 # 16 GiB + ## archive command is "/path/to/script.sh %path" where %path is replaced with the file being rolled: + # archive_command: + ## note that enabling this allows anyone with JMX/nodetool access to run local shell commands as the user running cassandra + # allow_nodetool_archive_command: false + # max_archive_retries: 10 + +# validate tombstones on reads and compaction +# can be either "disabled", "warn" or "exception" +# corrupted_tombstone_strategy: disabled + +# Diagnostic Events # +# If enabled, diagnostic events can be helpful for troubleshooting operational issues. Emitted events contain details +# on internal state and temporal relationships across events, accessible by clients via JMX. +diagnostic_events_enabled: false + +# Use native transport TCP message coalescing. If on upgrade to 4.0 you found your throughput decreasing, and in +# particular you run an old kernel or have very fewer client connections, this option might be worth evaluating. +#native_transport_flush_in_batches_legacy: false + +# Enable tracking of repaired state of data during reads and comparison between replicas +# Mismatches between the repaired sets of replicas can be characterized as either confirmed +# or unconfirmed. In this context, unconfirmed indicates that the presence of pending repair +# sessions, unrepaired partition tombstones, or some other condition means that the disparity +# cannot be considered conclusive. Confirmed mismatches should be a trigger for investigation +# as they may be indicative of corruption or data loss. +# There are separate flags for range vs partition reads as single partition reads are only tracked +# when CL > 1 and a digest mismatch occurs. Currently, range queries don't use digests so if +# enabled for range reads, all range reads will include repaired data tracking. As this adds +# some overhead, operators may wish to disable it whilst still enabling it for partition reads +repaired_data_tracking_for_range_reads_enabled: false +repaired_data_tracking_for_partition_reads_enabled: false +# If false, only confirmed mismatches will be reported. If true, a separate metric for unconfirmed +# mismatches will also be recorded. This is to avoid potential signal:noise issues are unconfirmed +# mismatches are less actionable than confirmed ones. +report_unconfirmed_repaired_data_mismatches: false + +# configure the read and write consistency levels for modifications to auth tables +# auth_read_consistency_level: LOCAL_QUORUM +# auth_write_consistency_level: EACH_QUORUM + +# Delays on auth resolution can lead to a thundering herd problem on reconnects; this option will enable +# warming of auth caches prior to node completing startup. See CASSANDRA-16958 +# auth_cache_warming_enabled: false + +# If enabled, dynamic data masking allows to attach CQL masking functions to the columns of a table. +# Users without the UNMASK permission will see an obscured version of the values of the columns with an attached mask. +# If dynamic data masking is disabled it won't be allowed to create new column masks, although it will still be possible +# to drop any previously existing masks. Also, any existing mask will be ignored at query time, so all users will see +# the clear values of the masked columns. +# Defaults to false to disable dynamic data masking. +# dynamic_data_masking_enabled: false + +######################### +# EXPERIMENTAL FEATURES # +######################### + +# Enables materialized view creation on this node. +# Materialized views are considered experimental and are not recommended for production use. +materialized_views_enabled: false + +# Enables SASI index creation on this node. +# SASI indexes are considered experimental and are not recommended for production use. +sasi_indexes_enabled: false + +# Enables creation of transiently replicated keyspaces on this node. +# Transient replication is experimental and is not recommended for production use. +transient_replication_enabled: false + +# Enables the used of 'ALTER ... DROP COMPACT STORAGE' statements on this node. +# 'ALTER ... DROP COMPACT STORAGE' is considered experimental and is not recommended for production use. +drop_compact_storage_enabled: false + +# Whether or not USE is allowed. This is enabled by default to avoid failure on upgrade. +#use_statements_enabled: true + +# When the client triggers a protocol exception or unknown issue (Cassandra bug) we increment +# a client metric showing this; this logic will exclude specific subnets from updating these +# metrics +#client_error_reporting_exclusions: +# subnets: +# - 127.0.0.1 +# - 127.0.0.0/31 + +# Enables read thresholds (warn/fail) across all replicas for reporting back to the client. +# See: CASSANDRA-16850 +# read_thresholds_enabled: false # scheduled to be set true in 4.2 +# When read_thresholds_enabled: true, this tracks the materialized size of a query on the +# coordinator. If coordinator_read_size_warn_threshold is defined, this will emit a warning +# to clients with details on what query triggered this as well as the size of the result set; if +# coordinator_read_size_fail_threshold is defined, this will fail the query after it +# has exceeded this threshold, returning a read error to the user. +# coordinator_read_size_warn_threshold: +# coordinator_read_size_fail_threshold: +# When read_thresholds_enabled: true, this tracks the size of the local read (as defined by +# heap size), and will warn/fail based off these thresholds; undefined disables these checks. +# local_read_size_warn_threshold: +# local_read_size_fail_threshold: +# When read_thresholds_enabled: true, this tracks the expected memory size of the RowIndexEntry +# and will warn/fail based off these thresholds; undefined disables these checks +# row_index_read_size_warn_threshold: +# row_index_read_size_fail_threshold: + +# Guardrail to warn or fail when creating more user keyspaces than threshold. +# The two thresholds default to -1 to disable. +# keyspaces_warn_threshold: -1 +# keyspaces_fail_threshold: -1 +# +# Guardrail to warn or fail when creating more user tables than threshold. +# The two thresholds default to -1 to disable. +# tables_warn_threshold: -1 +# tables_fail_threshold: -1 +# +# Guardrail to enable or disable the ability to create uncompressed tables +# uncompressed_tables_enabled: true +# +# Guardrail to warn or fail when creating/altering a table with more columns per table than threshold. +# The two thresholds default to -1 to disable. +# columns_per_table_warn_threshold: -1 +# columns_per_table_fail_threshold: -1 +# +# Guardrail to warn or fail when creating more secondary indexes per table than threshold. +# The two thresholds default to -1 to disable. +# secondary_indexes_per_table_warn_threshold: -1 +# secondary_indexes_per_table_fail_threshold: -1 +# +# Guardrail to enable or disable the creation of secondary indexes +# secondary_indexes_enabled: true +# +# Guardrail to warn or fail when creating more materialized views per table than threshold. +# The two thresholds default to -1 to disable. +# materialized_views_per_table_warn_threshold: -1 +# materialized_views_per_table_fail_threshold: -1 +# +# Guardrail to warn about, ignore or reject properties when creating tables. By default all properties are allowed. +# table_properties_warned: [] +# table_properties_ignored: [] +# table_properties_disallowed: [] +# +# Guardrail to allow/disallow user-provided timestamps. Defaults to true. +# user_timestamps_enabled: true +# +# Guardrail to bound user-provided timestamps within a given range. Default is infinite (denoted by null). +# Accepted values are durations of the form 12h, 24h, etc. +# maximum_timestamp_warn_threshold: +# maximum_timestamp_fail_threshold: +# minimum_timestamp_warn_threshold: +# minimum_timestamp_fail_threshold: +# +# Guardrail to allow/disallow GROUP BY functionality. +# group_by_enabled: true +# +# Guardrail to allow/disallow TRUNCATE and DROP TABLE statements +# drop_truncate_table_enabled: true +# +# Guardrail to allow/disallow DROP KEYSPACE statements +# drop_keyspace_enabled: true +# +# Guardrail to warn or fail when using a page size greater than threshold. +# The two thresholds default to -1 to disable. +# page_size_warn_threshold: -1 +# page_size_fail_threshold: -1 +# +# Guardrail to allow/disallow list operations that require read before write, i.e. setting list element by index and +# removing list elements by either index or value. Defaults to true. +# read_before_write_list_operations_enabled: true +# +# Guardrail to warn or fail when querying with an IN restriction selecting more partition keys than threshold. +# The two thresholds default to -1 to disable. +# partition_keys_in_select_warn_threshold: -1 +# partition_keys_in_select_fail_threshold: -1 +# +# Guardrail to warn or fail when an IN query creates a cartesian product with a size exceeding threshold, +# eg. "a in (1,2,...10) and b in (1,2...10)" results in cartesian product of 100. +# The two thresholds default to -1 to disable. +# in_select_cartesian_product_warn_threshold: -1 +# in_select_cartesian_product_fail_threshold: -1 +# +# Guardrail to warn about or reject read consistency levels. By default, all consistency levels are allowed. +# read_consistency_levels_warned: [] +# read_consistency_levels_disallowed: [] +# +# Guardrail to warn about or reject write consistency levels. By default, all consistency levels are allowed. +# write_consistency_levels_warned: [] +# write_consistency_levels_disallowed: [] +# +# Guardrail to warn or fail when writing partitions larger than threshold, expressed as 100MiB, 1GiB, etc. +# The guardrail is only checked when writing sstables (flush and compaction), and exceeding the fail threshold on that +# moment will only log an error message, without interrupting the operation. +# This operates on a per-sstable basis, so it won't detect a large partition if it is spread across multiple sstables. +# The warning threshold replaces the deprecated config property compaction_large_partition_warning_threshold. +# The two thresholds default to null to disable. +# partition_size_warn_threshold: +# partition_size_fail_threshold: +# +# Guardrail to warn or fail when writing partitions with more tombstones than threshold. +# The guardrail is only checked when writing sstables (flush and compaction), and exceeding the fail threshold on that +# moment will only log an error message, without interrupting the operation. +# This operates on a per-sstable basis, so it won't detect a large partition if it is spread across multiple sstables. +# The warning threshold replaces the deprecated config property compaction_tombstone_warning_threshold. +# The two thresholds default to -1 to disable. +# partition_tombstones_warn_threshold: -1 +# partition_tombstones_fail_threshold: -1 +# +# Guardrail to warn or fail when writing column values larger than threshold. +# This guardrail is only applied to the values of regular columns because both the serialized partitions keys and the +# values of the components of the clustering key already have a fixed, relatively small size limit of 65535 bytes, which +# is probably lesser than the thresholds defined here. +# Deleting individual elements of non-frozen sets and maps involves creating tombstones that contain the value of the +# deleted element, independently on whether the element existed or not. That tombstone value is also guarded by this +# guardrail, to prevent the insertion of tombstones over the threshold. The downside is that enabling or raising this +# threshold can prevent users from deleting set/map elements that were written when the guardrail was disabled or with a +# lower value. Deleting the entire column, row or partition is always allowed, since the tombstones created for those +# operations don't contain the CQL column values. +# This guardrail is different to max_value_size. max_value_size is checked when deserializing any value to detect +# sstable corruption, whereas this guardrail is checked on the CQL layer at write time to reject regular user queries +# inserting too large columns. +# The two thresholds default to null to disable. +# Min unit: B +# column_value_size_warn_threshold: +# column_value_size_fail_threshold: +# +# Guardrail to warn or fail when encountering larger size of collection data than threshold. +# At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case +# of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to +# prevent read-before-write. The guardrail is also checked at sstable write time to detect large non-frozen collections, +# although in that case exceeding the fail threshold will only log an error message, without interrupting the operation. +# The two thresholds default to null to disable. +# Min unit: B +# collection_size_warn_threshold: +# Min unit: B +# collection_size_fail_threshold: +# +# Guardrail to warn or fail when encountering more elements in collection than threshold. +# At query time this guardrail is applied only to the collection fragment that is being writen, even though in the case +# of non-frozen collections there could be unaccounted parts of the collection on the sstables. This is done this way to +# prevent read-before-write. The guardrail is also checked at sstable write time to detect large non-frozen collections, +# although in that case exceeding the fail threshold will only log an error message, without interrupting the operation. +# The two thresholds default to -1 to disable. +# items_per_collection_warn_threshold: -1 +# items_per_collection_fail_threshold: -1 +# +# Guardrail to allow/disallow querying with ALLOW FILTERING. Defaults to true. +# ALLOW FILTERING can potentially visit all the data in the table and have unpredictable performance. +# allow_filtering_enabled: true +# +# Guardrail to allow/disallow setting SimpleStrategy via keyspace creation or alteration. Defaults to true. +# simplestrategy_enabled: true +# +# Guardrail to warn or fail when creating a user-defined-type with more fields in than threshold. +# Default -1 to disable. +# fields_per_udt_warn_threshold: -1 +# fields_per_udt_fail_threshold: -1 +# +# Guardrail to warn or fail when creating a vector column with more dimensions than threshold. +# Default -1 to disable. +# vector_dimensions_warn_threshold: -1 +# vector_dimensions_fail_threshold: -1 +# +# Guardrail to indicate whether or not users are allowed to use ALTER TABLE commands to make column changes to tables +# alter_table_enabled: true +# +# Guardrail to warn or fail when local data disk usage percentage exceeds threshold. Valid values are in [1, 100]. +# This is only used for the disks storing data directories, so it won't count any separate disks used for storing +# the commitlog, hints nor saved caches. The disk usage is the ratio between the amount of space used by the data +# directories and the addition of that same space and the remaining free space on disk. The main purpose of this +# guardrail is rejecting user writes when the disks are over the defined usage percentage, so the writes done by +# background processes such as compaction and streaming don't fail due to a full disk. The limits should be defined +# accordingly to the expected data growth due to those background processes, so for example a compaction strategy +# doubling the size of the data would require to keep the disk usage under 50%. +# The two thresholds default to -1 to disable. +# data_disk_usage_percentage_warn_threshold: -1 +# data_disk_usage_percentage_fail_threshold: -1 +# +# Guardrail that allows users to define the max disk size of the data directories when calculating thresholds for +# disk_usage_percentage_warn_threshold and disk_usage_percentage_fail_threshold, so if this is greater than zero they +# become percentages of a fixed size on disk instead of percentages of the physically available disk size. This should +# be useful when we have a large disk and we only want to use a part of it for Cassandra's data directories. +# Valid values are in [1, max available disk size of all data directories]. +# Defaults to null to disable and use the physically available disk size of data directories during calculations. +# Min unit: B +# data_disk_usage_max_disk_size: +# +# Guardrail to warn or fail when the minimum replication factor is lesser than threshold. +# This would also apply to system keyspaces. +# Suggested value for use in production: 2 or higher +# minimum_replication_factor_warn_threshold: -1 +# minimum_replication_factor_fail_threshold: -1 +# +# Guardrail to warn or fail when the maximum replication factor is greater than threshold. +# This would also apply to system keyspaces. +# maximum_replication_factor_warn_threshold: -1 +# maximum_replication_factor_fail_threshold: -1 + +# Guardrail to enable a CREATE or ALTER TABLE statement when default_time_to_live is set to 0 +# and the table is using TimeWindowCompactionStrategy compaction or a subclass of it. +# It is suspicious to use default_time_to_live set to 0 with such compaction strategy. +# Please keep in mind that data will not start to automatically expire after they are older than +# a respective compaction window unit of a certain size. Please set TTL for your INSERT or UPDATE +# statements if you expect data to be expired as table settings will not do it. +# Defaults to true. If set to false, such statements fail and zero_ttl_on_twcs_warned flag is irrelevant. +#zero_ttl_on_twcs_enabled: true +# Guardrail to warn a user upon executing CREATE or ALTER TABLE statement when default_time_to_live is set to 0 +# and the table is using TimeWindowCompactionStrategy compaction or a subclass of it. Defaults to true. +# if zero_ttl_on_twcs_enabled is set to false, this property is irrelevant as such statements will fail. +#zero_ttl_on_twcs_warned: true + +# Guardrail enabling secondary index queries that do not restrict on partition key (defaults to true) +#non_partition_restricted_index_query_enabled: true +# Maximum number of referenced SAI indexes on a replica when executing a SELECT query +# before emitting a warning (defaults to 32) +#sai_sstable_indexes_per_query_warn_threshold: 32 +# Maximum number of referenced SAI indexes on a replica when executing a SELECT query +# before emitting a failure (defaults to -1 to disable) +#sai_sstable_indexes_per_query_fail_threshold: -1 + +# Guardrail specifying warn/fail thresholds for the size of string terms written to an SAI index +# sai_string_term_size_warn_threshold: 1KiB +# sai_string_term_size_fail_threshold: 8KiB + +# Guardrail specifying warn/fail thresholds for the size of frozen terms written to an SAI index +# sai_frozen_term_size_warn_threshold: 1KiB +# sai_frozen_term_size_fail_threshold: 8KiB + +# Guardrail specifying warn/fail thresholds for the size of vector terms written to an SAI index +# sai_vector_term_size_warn_threshold: 16KiB +# sai_vector_term_size_fail_threshold: 32KiB + +# The default secondary index implementation when CREATE INDEX does not specify one via USING. +# ex. "legacy_local_table" - (default) legacy secondary index, implemented as a hidden table +# ex. "sai" - "storage-attched" index, implemented via optimized SSTable/Memtable-attached indexes +#default_secondary_index: legacy_local_table + +# Whether a default secondary index implementation is allowed. If this is "false", CREATE INDEX must +# specify an index implementation via USING. +#default_secondary_index_enabled: true + +# Startup Checks are executed as part of Cassandra startup process, not all of them +# are configurable (so you can disable them) but these which are enumerated bellow. +# Uncomment the startup checks and configure them appropriately to cover your needs. +# +#startup_checks: +# Verifies correct ownership of attached locations on disk at startup. See CASSANDRA-16879 for more details. +# check_filesystem_ownership: +# enabled: false +# ownership_token: "sometoken" # (overriden by "CassandraOwnershipToken" system property) +# ownership_filename: ".cassandra_fs_ownership" # (overriden by "cassandra.fs_ownership_filename") +# Prevents a node from starting if snitch's data center differs from previous data center. +# check_dc: +# enabled: true # (overriden by cassandra.ignore_dc system property) +# Prevents a node from starting if snitch's rack differs from previous rack. +# check_rack: +# enabled: true # (overriden by cassandra.ignore_rack system property) +# Enable this property to fail startup if the node is down for longer than gc_grace_seconds, to potentially +# prevent data resurrection on tables with deletes. By default, this will run against all keyspaces and tables +# except the ones specified on excluded_keyspaces and excluded_tables. +# check_data_resurrection: +# enabled: false +# file where Cassandra periodically writes the last time it was known to run +# heartbeat_file: /var/lib/cassandra/data/cassandra-heartbeat +# excluded_keyspaces: # comma separated list of keyspaces to exclude from the check +# excluded_tables: # comma separated list of keyspace.table pairs to exclude from the check + +# This property indicates with what Cassandra major version the storage format will be compatible with. +# +# The chosen storage compatibility mode will determine the versions of the written sstables, commitlogs, hints, etc. +# For example, if we're going to remain compatible with Cassandra 4.x, the value of this property should be 4, which +# will make us use sstables in the latest N version of the BIG format. +# +# This will also determine if certain features that depend on newer formats are available. For example, extended TTL +# (up to 2106) depends on the sstable, commit-log, hints, and messaging versions introduced by Cassandra 5.0, so that +# feature won't be available if this property is set to CASSANDRA_4. See the upgrade guide for more details. +# +# Possible values are: +# +# ** CASSANDRA_4: Stays compatible with the 4.x line in features, formats and component versions. +# ** UPGRADING: The cluster monitors the version of each node during this interim stage. This has a cost but ensures +# all new features, formats, versions, etc. are enabled safely. +# ** NONE: Start with all the new features and formats enabled. +# +# A typical upgrade would be: +# +# . Do a rolling upgrade, starting all nodes in CASSANDRA_X compatibility mode. +# . Once the new binary is rendered stable, do a rolling restart with the UPGRADING mode. The cluster will keep new +# features disabled until all nodes are started in the UPGRADING mode; when that happens, new features controlled by +# the storage compatibility mode are enabled. +# . Do a rolling restart with all nodes starting with the NONE mode. This eliminates the cost of checking node versions +# and ensures stability. If Cassandra was started at the previous version by accident, a node with disabled +# compatibility mode would no longer toggle behaviors as when it was running in the UPGRADING mode. +# +storage_compatibility_mode: CASSANDRA_4 diff --git a/cassandra-5/src/test/resources/logback-test.xml b/cassandra-5/src/test/resources/logback-test.xml new file mode 100644 index 00000000..82e90f2f --- /dev/null +++ b/cassandra-5/src/test/resources/logback-test.xml @@ -0,0 +1,19 @@ + + + + %d{ISO8601} %-5p %X{dbz.connectorType}|%X{dbz.connectorName}|%X{dbz.connectorContext} %m [%c]%n + + + + + + + + + + + + + + + \ No newline at end of file diff --git a/pom.xml b/pom.xml index ee392d3c..085beadb 100644 --- a/pom.xml +++ b/pom.xml @@ -25,6 +25,7 @@ core cassandra-3 cassandra-4 + cassandra-5 dse