From e9c0b0891ec9fc8c9bb58c770c1d27b5efe4a600 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Thu, 18 Sep 2025 15:06:40 -0700 Subject: [PATCH 01/20] CASSSIDECAR-242: Implementation of CachingSchemaStore CASSSIDECAR-242: Implementation of CachingSchemaStore House cleanup WIP commit Working CDC on Sidecar Latest changes Working CDC Working Working --- build.gradle | 3 + conf/sidecar.yaml | 20 +- gradle.properties | 5 +- server/build.gradle | 15 +- .../sidecar/cdc/CachingSchemaStore.java | 260 ++++++++++++ .../sidecar/cdc/CdcAvroSerializer.java | 24 ++ .../cassandra/sidecar/cdc/CdcConfigImpl.java | 145 ++----- .../CdcDynamicSidecarInstancesProvider.java | 57 +++ .../sidecar/cdc/CdcEventConsumer.java | 31 ++ .../cassandra/sidecar/cdc/CdcManager.java | 247 +++++++++++ .../cassandra/sidecar/cdc/CdcPublisher.java | 383 +++++++++++++++++ .../sidecar/cdc/CdcSchemaSupplier.java | 105 +++++ .../sidecar/cdc/CdcSidecarInstanceImpl.java | 106 +++++ .../sidecar/cdc/ExecutorPoolsExecutor.java | 76 ++++ .../sidecar/cdc/SidecarCdcOptions.java | 35 ++ .../cdc/SidecarCqlToAvroSchemaConverter.java | 42 ++ .../cdc/StateSidecarCdcCassandraClient.java | 39 ++ .../sidecar/codecs/BigIntegerCodec.java | 66 +++ .../codecs/CdcConfigMappingsCodec.java | 78 ++++ .../sidecar/codecs/CommonCodecs.java | 24 ++ .../DcLocalTopologyChangeEventCodec.java | 134 ++++++ .../sidecar/codecs/RangeChangeEventCodec.java | 57 +++ .../cassandra/sidecar/codecs/RangeCodec.java | 83 ++++ .../sidecar/codecs/RangeMapCodec.java | 109 +++++ .../config/yaml/CdcConfigurationImpl.java | 2 +- .../yaml/SchemaKeyspaceConfigurationImpl.java | 4 +- .../CassandraClientTokenRingProvider.java | 25 +- .../ContentionFreeRangeManager.java | 34 ++ .../sidecar/coordination/RangeManager.java | 398 ++++++++++++++++++ .../coordination/TokenRingProvider.java | 9 +- .../sidecar/db/CdcDatabaseAccessor.java | 82 ++-- .../db/TableHistoryDatabaseAccessor.java | 78 ++++ .../db/VirtualTablesDatabaseAccessor.java | 43 ++ .../sidecar/db/schema/TableHistorySchema.java | 2 +- .../sidecar/handlers/AbstractHandler.java | 4 +- .../cassandra/sidecar/modules/CdcModule.java | 231 +++++++++- .../multibindings/PeriodicTaskMapKeys.java | 2 + .../multibindings/TableSchemaMapKeys.java | 2 + .../sidecar/server/SidecarServerEvents.java | 4 + .../tasks/CassandraClusterSchemaMonitor.java | 9 +- .../tasks/CdcConfigRefresherNotifierTask.java | 145 +++++++ .../sidecar/tasks/ClusterTopologyMonitor.java | 296 +++++++++++++ .../sidecar/cdc/CachingSchemaStoreTest.java | 334 +++++++++++++++ .../sidecar/cdc/CdcConfigImplTest.java | 36 +- .../cassandra/sidecar/cdc/CdcManagerTest.java | 297 +++++++++++++ .../sidecar/cdc/CdcPublisherTests.java | 291 +++++++++++++ .../sidecar/cdc/CdcSchemaSupplierTest.java | 169 ++++++++ .../CassandraClientTokenRingProviderTest.java | 112 ++++- .../sidecar/db/CdcDatabaseAccessorTests.java | 18 +- 49 files changed, 4515 insertions(+), 256 deletions(-) create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcDynamicSidecarInstancesProvider.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSidecarInstanceImpl.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java create mode 100644 server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java create mode 100644 server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java create mode 100644 server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java create mode 100644 server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java diff --git a/build.gradle b/build.gradle index 2f4d36679..0eda7323f 100644 --- a/build.gradle +++ b/build.gradle @@ -194,6 +194,8 @@ startScripts { } } +apply from: "${project.rootDir}/gradle/common/java11Options.gradle" + run { def confFile = System.getProperty("sidecar.config", "file:" + File.separator + File.separator + "$projectDir/conf/sidecar.yaml") println "Sidecar configuration file $confFile" @@ -201,6 +203,7 @@ run { "-Dsidecar.config=" + confFile, "-Dlogback.configurationFile=./conf/logback.xml", "-Dvertx.logger-delegate-factory-class-name=io.vertx.core.logging.SLF4JLogDelegateFactory"] + jvmArgs += project.ext.JDK11_OPTIONS } distributions { diff --git a/conf/sidecar.yaml b/conf/sidecar.yaml index 57be2e787..d270bf277 100644 --- a/conf/sidecar.yaml +++ b/conf/sidecar.yaml @@ -21,7 +21,7 @@ # cassandra_instances: - id: 1 - host: localhost1 + host: localhost port: 9042 # The instance's storage directory as defined per the cassandra.storagedir property @@ -42,11 +42,13 @@ cassandra_instances: # Directory where Cassandra stores mutations. If not set, the default directory is # /cdc_raw. - #cdc_dir: /var/lib/cassandra/cdc_raw + cdc_dir: /Users/bernardobotellacorbi/Documents/dev/cassandra_trunk/data/cdc_raw +# cdc_dir: /var/lib/cassandra/cdc_raw # Directory where Cassandra stores the commit logs. If not set, the default directory # is /commitlog - #commitlog_dir: /var/lib/cassandra/commitlog + commitlog_dir: /Users/bernardobotellacorbi/Documents/dev/cassandra_trunk/data/commitlog +# commitlog_dir: /var/lib/cassandra/commitlog # Directory where Cassandra stores hints. If not set, the default directory is # /hints. @@ -61,14 +63,14 @@ cassandra_instances: # first of the data directories specified by data_file_directories. # local_system_data_file_dir: - jmx_host: 127.0.0.1 + jmx_host: localhost jmx_port: 7199 jmx_ssl_enabled: false # jmx_role: # jmx_role_password: sidecar: - host: 0.0.0.0 + host: localhost port: 9043 request_idle_timeout: 5m request_timeout: 5m @@ -121,7 +123,7 @@ sidecar: max_retries: 3 retry_delay: 200ms schema: - is_enabled: false + is_enabled: true keyspace: sidecar_internal replication_strategy: SimpleStrategy replication_factor: 1 @@ -302,7 +304,7 @@ access_control: driver_parameters: contact_points: - - "127.0.0.1:9042" + - "localhost:9042" username: cassandra password: cassandra ssl: @@ -348,8 +350,8 @@ sidecar_client: #max_retries: 5 # The amount of retries the client will attempt a request #retry_delay: 500ms # The initial delay between the retries the client will attempt a request #max_retry_delay: 10s # The max delay between the retries the client will attempt a request - #ssl: # if ssl is enabled, this is the ssl configuration used for the sidecar client - # enabled: false + ssl: # if ssl is enabled, this is the ssl configuration used for the sidecar client + enabled: false # keystore: # type: PKCS12 # Keystore type # path: path/to/client/keystore.p12 # Path to the client keystore file diff --git a/gradle.properties b/gradle.properties index 463fcb93f..e731b195d 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ # limitations under the License. # -version=0.3.0 +version=0.3.99 junitVersion=5.9.2 vertxVersion=4.5.23 nettyVersion=4.1.118.Final @@ -43,6 +43,7 @@ jakartaWsRsVersion=3.1.0 swaggerVersion=2.2.21 # Cdc dependencies kryoVersion=4.0.2 -analyticsVersion=0.1.0 # OSHI dependencies oshiVersion=6.9.0 +analyticsVersion=0.2.0 +kafkaClientVersion=3.7.0 diff --git a/server/build.gradle b/server/build.gradle index 53419b0b1..a4a1418db 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -80,6 +80,8 @@ configurations { containerTestImplementation.extendsFrom testImplementation } +apply from: "${project.rootDir}/gradle/common/java11Options.gradle" + dependencies { compileOnly('org.jetbrains:annotations:23.0.0') testCompileOnly('org.jetbrains:annotations:23.0.0') @@ -144,7 +146,13 @@ dependencies { implementation("jakarta.ws.rs:jakarta.ws.rs-api:${project.jakartaWsRsVersion}") implementation(group: "org.apache.cassandra", name: "cassandra-analytics-common", version: "${[project.analyticsVersion]}") + implementation(group: "org.apache.cassandra", name: "cassandra-analytics-core_spark3_2.12", version: "${[project.analyticsVersion]}") + implementation(group: "org.apache.cassandra", name: "cassandra-analytics-cdc-codec_spark3_2.12", version: "${[project.analyticsVersion]}") + implementation(group: "org.apache.cassandra", name: "cassandra-avro-converter_spark3_2.12", version: "${[project.analyticsVersion]}") implementation(group: "org.apache.cassandra", name: "cassandra-analytics-cdc_spark3_2.12", version: "${[project.analyticsVersion]}") + implementation(group: "org.apache.cassandra", name: "cassandra-analytics-cdc-sidecar_spark3_2.12", version: "${[project.analyticsVersion]}") + + implementation "org.apache.kafka:kafka-clients:${project.kafkaClientVersion}" implementation "com.esotericsoftware:kryo-shaded:${kryoVersion}" // OSHI core library for fetching system information @@ -197,10 +205,16 @@ dependencies { } test { + if (JavaVersion.current().isJava11Compatible()) { + jvmArgs(project.ext.JDK11_OPTIONS) + println("JVM arguments for $project.name are $allJvmArgs") + } + systemProperty "vertxweb.environment", "dev" systemProperty "vertx.logger-delegate-factory-class-name", "io.vertx.core.logging.SLF4JLogDelegateFactory" // There is no native lib (JNR) for getting time for testing systemProperty "com.datastax.driver.USE_NATIVE_CLOCK", "false" + // ordinarily we don't need integration tests // see the integrationTest task useJUnitPlatform() @@ -219,7 +233,6 @@ test { } apply from: "${project.rootDir}/gradle/common/integrationTestTask.gradle" -apply from: "${project.rootDir}/gradle/common/java11Options.gradle" tasks.register("containerTest", Test) { if (JavaVersion.current().isJava11Compatible()) { diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java new file mode 100644 index 000000000..911d5c846 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -0,0 +1,260 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.nio.charset.StandardCharsets; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; +import javax.annotation.Nullable; + +import com.google.common.annotations.VisibleForTesting; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.Vertx; +import io.vertx.core.eventbus.EventBus; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; + +import org.apache.cassandra.cdc.avro.AvroSchemas; +import org.apache.cassandra.cdc.avro.CqlToAvroSchemaConverter; +import org.apache.cassandra.cdc.kafka.KafkaOptions; +import org.apache.cassandra.cdc.schemastore.SchemaStore; +import org.apache.cassandra.cdc.schemastore.SchemaStorePublisherFactory; +import org.apache.cassandra.cdc.schemastore.TableSchemaPublisher; +import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; +import org.apache.cassandra.sidecar.db.TableHistoryDatabaseAccessor; +import org.apache.cassandra.sidecar.db.schema.SidecarSchema; +import org.apache.cassandra.sidecar.tasks.CassandraClusterSchemaMonitor; +import org.apache.cassandra.spark.data.CqlTable; +import org.apache.cassandra.spark.utils.TableIdentifier; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CACHE_WARMED_UP; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SERVER_START; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_SCHEMA_INITIALIZED; + +/** + * Schemas cache to be used by CDC event serialization. + */ +@Singleton +public class CachingSchemaStore implements SchemaStore +{ + private static final Logger LOGGER = LoggerFactory.getLogger(CachingSchemaStore.class); + private final Map avroSchemasCache = new ConcurrentHashMap<>(); + private final CassandraClusterSchemaMonitor cassandraClusterSchemaMonitor; + private final SidecarSchema sidecarSchema; + private final TableHistoryDatabaseAccessor tableHistoryDatabaseAccessor; + private final Vertx vertx; + private final CdcConfigImpl cdcConfig; + @Nullable volatile TableSchemaPublisher publisher; + private final CqlToAvroSchemaConverter cqlToAvroSchemaConverter; + private final SidecarCdcStats sidecarCdcStats; + + @Inject + CachingSchemaStore(Vertx vertx, + CassandraClusterSchemaMonitor cassandraClusterSchemaMonitor, + TableHistoryDatabaseAccessor tableHistoryDatabaseAccessor, + CdcConfigImpl cdcConfig, + SidecarCdcStats sidecarCdcStats, + SidecarSchema sidecarSchema, + CqlToAvroSchemaConverter cqlToAvroSchemaConverter) + { + super(); + this.cassandraClusterSchemaMonitor = cassandraClusterSchemaMonitor; + this.tableHistoryDatabaseAccessor = tableHistoryDatabaseAccessor; + this.sidecarSchema = sidecarSchema; + this.cqlToAvroSchemaConverter = cqlToAvroSchemaConverter; + this.avroSchemasCache.putAll(createSchemaCache(cassandraClusterSchemaMonitor.getCdcTables())); + AvroSchemas.registerLogicalTypes(); + cassandraClusterSchemaMonitor.addSchemaChangeListener(this::onSchemaChanged); + this.vertx = vertx; + this.cdcConfig = cdcConfig; + this.sidecarCdcStats = sidecarCdcStats; + + ThrowingRunnable configChangeCallback = () -> { + LOGGER.info("Services configuration changed. Reloading publisher..."); + loadPublisher(); + publishSchemas(); + }; + this.cdcConfig.registerConfigChangeListener(configChangeCallback); + configureSidecarServerEventListeners(); + } + + private void loadPublisher() + { + KafkaOptions kafkaOptions = new KafkaOptions() + { + public Map kafkaConfigs() + { + return cdcConfig.kafkaConfigs(); + } + }; + this.publisher = SchemaStorePublisherFactory.DEFAULT.buildPublisher(kafkaOptions); + } + + private void configureSidecarServerEventListeners() + { + EventBus eventBus = vertx.eventBus(); + + eventBus.localConsumer(ON_SERVER_START.address(), startMessage -> { + eventBus.localConsumer(ON_SIDECAR_SCHEMA_INITIALIZED.address(), message -> { + LOGGER.debug("Sidecar Schema initialized message={}", message); + Set refreshedCdcTables = cassandraClusterSchemaMonitor.getCdcTables(); + for (CqlTable cqlTable : refreshedCdcTables) + { + TableIdentifier tableIdentifier = TableIdentifier.of(cqlTable.keyspace(), cqlTable.table()); + avroSchemasCache.compute(tableIdentifier, (k, v) -> + { + tableHistoryDatabaseAccessor.insertTableSchemaHistory(cqlTable.keyspace(), cqlTable.table(), cqlTable.createStatement()); + return v; + }); + } + loadPublisher(); + publishSchemas(); + }); + }); + } + + private void publishSchemas() + { + Set refreshedCdcTables = cassandraClusterSchemaMonitor.getCdcTables(); + for (CqlTable cqlTable : refreshedCdcTables) + { + TableIdentifier tableIdentifier = TableIdentifier.of(cqlTable.keyspace(), cqlTable.table()); + avroSchemasCache.compute(tableIdentifier, (k, v) -> + { + if (null != publisher) + { + Schema schema = cqlToAvroSchemaConverter.convert(cqlTable); + TableSchemaPublisher.SchemaPublishMetadata metadata = new TableSchemaPublisher.SchemaPublishMetadata(); + metadata.put("name", cqlTable.table()); + metadata.put("namespace", cqlTable.keyspace()); + publisher.publishSchema(schema.toString(false), metadata); + sidecarCdcStats.capturePublishedSchema(); + } + return new SchemaCacheEntry(cqlTable, cqlToAvroSchemaConverter); + }); + } + } + + @VisibleForTesting + void onSchemaChanged() + { + Set refreshedCdcTables = cassandraClusterSchemaMonitor.getCdcTables(); + for (CqlTable cqlTable : refreshedCdcTables) + { + TableIdentifier tableIdentifier = TableIdentifier.of(cqlTable.keyspace(), cqlTable.table()); + avroSchemasCache.compute(tableIdentifier, (k, v) -> + { + if (v == null || !v.tableSchema().equals(cqlTable.createStatement())) + { + if (sidecarSchema.isInitialized()) + { + tableHistoryDatabaseAccessor.insertTableSchemaHistory(cqlTable.keyspace(), cqlTable.table(), cqlTable.createStatement()); + } + LOGGER.info("Re-generating Avro Schema after schema change keyspace={} table={}", tableIdentifier.keyspace(), tableIdentifier.table()); + return new SchemaCacheEntry(cqlTable, cqlToAvroSchemaConverter); + } + return v; + }); + loadPublisher(); + publishSchemas(); + } + // Remove any old schema entries for deleted tables, this operation can be done in the end as this is + // only for removing stale entries and no one is going to use these entries once the table is removed. + // This doesn't have to be an atomic operation. + avroSchemasCache.keySet().retainAll(refreshedCdcTables.stream().map(cqlTable -> TableIdentifier.of(cqlTable.keyspace(), cqlTable.table())).collect(Collectors.toList())); + vertx.eventBus().publish(ON_CDC_CACHE_WARMED_UP.address(), "Cdc cache warmed up"); + } + + @Override + public Schema getSchema(String namespace, String name) + { + TableIdentifier tableIdentifier = getTableIdentifierFromNamespace(namespace); + return avroSchemasCache.computeIfAbsent(tableIdentifier, k -> + { + LOGGER.warn("Unknown table for getting schema keyspace={} table={}", tableIdentifier.keyspace(), tableIdentifier.table()); + throw new RuntimeException("Unable to get schema for unknown table " + tableIdentifier); + }).schema; + } + + @Override + public GenericDatumWriter getWriter(String namespace, String name) + { + TableIdentifier tableIdentifier = getTableIdentifierFromNamespace(namespace); + return avroSchemasCache.computeIfAbsent(tableIdentifier, k -> { + LOGGER.warn("Unknown table for getting writer keyspace={} table={}", tableIdentifier.keyspace(), tableIdentifier.table()); + throw new RuntimeException("Unable to get writer for unknown table " + tableIdentifier); + }).writer; + } + + @Override + public GenericDatumReader getReader(String namespace, String name) + { + TableIdentifier tableIdentifier = getTableIdentifierFromNamespace(namespace); + return avroSchemasCache.computeIfAbsent(tableIdentifier, k -> { + LOGGER.warn("Unknown table for getting reader keyspace={} table={}", tableIdentifier.keyspace(), tableIdentifier.table()); + throw new RuntimeException("Unable to get reader for unknown table " + tableIdentifier); + }).reader; + } + + @Override + public String getVersion(String namespace, String name) + { + TableIdentifier tableIdentifier = getTableIdentifierFromNamespace(namespace); + return avroSchemasCache.computeIfAbsent(tableIdentifier, k -> { + LOGGER.warn("Unknown table for getting reader keyspace={} table={}", tableIdentifier.keyspace(), tableIdentifier.table()); + throw new RuntimeException("Unable to get reader for unknown table " + tableIdentifier); + }).schemaUuid; + } + + public Map getSchemas() + { + return avroSchemasCache.values().stream() + .collect(Collectors.toMap(e -> e.schema.getNamespace(), e -> e.schema)); + } + + private Map createSchemaCache(Set cdcTables) + { + + return cdcTables.stream() + .collect(Collectors.toMap(cqlTable -> TableIdentifier.of(cqlTable.keyspace(), cqlTable.table()), + cqlTable -> new SchemaCacheEntry(cqlTable, cqlToAvroSchemaConverter)) + ); + } + + private TableIdentifier getTableIdentifierFromNamespace(String namespace) + { + String[] namespaceParts = namespace.split("\\."); + return TableIdentifier.of(namespaceParts[0], namespaceParts[1]); + } + + private static class SchemaCacheEntry + { + private final CqlTable table; + private final Schema schema; + private final String schemaUuid; + private final GenericDatumWriter writer; + private final GenericDatumReader reader; + + private SchemaCacheEntry(CqlTable table, + CqlToAvroSchemaConverter cqlToAvroSchemaConverter) + { + this.table = table; + this.schema = cqlToAvroSchemaConverter.convert(table); + this.schemaUuid = UUID.nameUUIDFromBytes(table.createStatement().getBytes(StandardCharsets.UTF_8)).toString(); + this.writer = new GenericDatumWriter<>(schema); + this.reader = new GenericDatumReader<>(schema); + } + + public String tableSchema() + { + return table.createStatement(); + } + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java new file mode 100644 index 000000000..fa6c7b06d --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java @@ -0,0 +1,24 @@ +package org.apache.cassandra.sidecar.cdc; + +import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.cdc.TypeCache; +import org.apache.cassandra.cdc.kafka.AvroGenericRecordSerializer; +import org.apache.cassandra.cdc.schemastore.SchemaStore; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; + +/** + * Taking a schema, this class serializes a CDC into AVRO format + */ +public class CdcAvroSerializer extends AvroGenericRecordSerializer +{ + public CdcAvroSerializer(SchemaStore schemaStore, + InstanceMetadataFetcher instanceMetadataFetcher, + CassandraBridgeFactory cassandraBridgeFactory) + { + super(schemaStore, key -> + TypeCache.get(cassandraBridgeFactory + .get(instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> + instance.delegate().nodeSettings()).releaseVersion()).getVersion()) + .getType(key.keyspace, key.type), "org.apache.cassandra"); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java index aabfabe85..c8099574c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java @@ -17,37 +17,30 @@ */ package org.apache.cassandra.sidecar.cdc; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import com.google.inject.Inject; import com.google.inject.Singleton; -import io.vertx.core.Promise; +import io.vertx.core.Handler; +import io.vertx.core.Vertx; +import io.vertx.core.eventbus.Message; import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; -import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; import org.apache.cassandra.sidecar.common.server.utils.SecondBoundConfiguration; -import org.apache.cassandra.sidecar.config.CdcConfiguration; -import org.apache.cassandra.sidecar.config.SchemaKeyspaceConfiguration; -import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.db.CdcConfigAccessor; -import org.apache.cassandra.sidecar.db.KafkaConfigAccessor; -import org.apache.cassandra.sidecar.tasks.PeriodicTask; -import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; -import org.apache.cassandra.sidecar.tasks.ScheduleDecision; +import org.apache.cassandra.sidecar.tasks.CdcConfigRefresherNotifierTask; import org.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIGURATION_CHANGED; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; + /** * Implementation of the interface {@link CdcConfig}, an in-memory representation holding * CDC and Kafka configurations from "configs" table inside sidecar internal keyspace. @@ -55,40 +48,35 @@ @Singleton public class CdcConfigImpl implements CdcConfig { - private static final Logger LOGGER = LoggerFactory.getLogger(CdcConfigImpl.class); private static final int DEFAULT_MAX_WATERMARKER_SIZE = 400000; private static final String DEFAULT_JOB_ID = "test-job-id"; private static final int DEFAULT_MAX_COMMITLOGS_PER_INSTANCE = 4; private static final int DEFAULT_MAX_RECORD_BYTE_SIZE = -1; public static final int DEFAULT_WATERMARK_WINDOW = 259200; - private final SchemaKeyspaceConfiguration schemaKeyspaceConfiguration; - private final CdcConfiguration cdcConfiguration; + private final Vertx vertx; private final CdcConfigAccessor cdcConfigAccessor; - private final KafkaConfigAccessor kafkaConfigAccessor; - private final List configChangeListeners = Collections.synchronizedList(new ArrayList<>()); - private final ConfigRefreshNotifier configRefreshNotifier; - private volatile Map kafkaConfigMappings = Map.of(); - private volatile Map cdcConfigMappings = Map.of(); + + private Map kafkaConfigMappings = Map.of(); + private Map cdcConfigMappings = Map.of(); @Inject - public CdcConfigImpl(SidecarConfiguration sidecarConfiguration, - CdcConfigAccessor cdcConfigAccessor, - KafkaConfigAccessor kafkaConfigAccessor, - PeriodicTaskExecutor periodicTaskExecutor) + public CdcConfigImpl(Vertx vertx, + CdcConfigAccessor cdcConfigAccessor) { - this.schemaKeyspaceConfiguration = sidecarConfiguration.serviceConfiguration().schemaKeyspaceConfiguration(); - this.cdcConfiguration = sidecarConfiguration.serviceConfiguration().cdcConfiguration(); + this.vertx = vertx; this.cdcConfigAccessor = cdcConfigAccessor; - this.kafkaConfigAccessor = kafkaConfigAccessor; - if (this.schemaKeyspaceConfiguration.isEnabled()) - { - this.configRefreshNotifier = new ConfigRefreshNotifier(); - periodicTaskExecutor.schedule(configRefreshNotifier); - } - else + vertx.eventBus().localConsumer(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), new ConfigMappingsChanged()); + } + + private class ConfigMappingsChanged implements Handler> + { + public void handle(Message event) { - this.configRefreshNotifier = null; + CdcConfigRefresherNotifierTask.ConfigMappings configMappings = (CdcConfigRefresherNotifierTask.ConfigMappings) event.body(); + kafkaConfigMappings = configMappings.getKafkaConfigMappings(); + cdcConfigMappings = configMappings.getCdcConfigMappings(); + vertx.eventBus().publish(ON_CDC_CONFIGURATION_CHANGED.address(), "Cdc Configuration Changed"); } } @@ -110,6 +98,8 @@ public Map cdcConfigs() @Override public boolean isConfigReady() { + kafkaConfigs(); + cdcConfigs(); return cdcConfigAccessor.isAvailable() && !kafkaConfigMappings.isEmpty() && !cdcConfigMappings.isEmpty(); @@ -247,7 +237,7 @@ protected int getInt(String key, Supplier orDefault) */ public void registerConfigChangeListener(ThrowingRunnable listener) { - this.configChangeListeners.add(listener); +// this.configRefreshNotifier.registerConfigChangeListener(listener); } private Map getAuthConfigs() @@ -258,82 +248,11 @@ private Map getAuthConfigs() @VisibleForTesting void forceExecuteNotifier() { - if (configRefreshNotifier != null && - configRefreshNotifier.scheduleDecision() == ScheduleDecision.EXECUTE) - { - configRefreshNotifier.execute(Promise.promise()); - } - } - - @VisibleForTesting - ConfigRefreshNotifier configRefreshNotifier() - { - return configRefreshNotifier; - } - - class ConfigRefreshNotifier implements PeriodicTask - { - @Override - public DurationSpec delay() - { - return cdcConfiguration.cdcConfigRefreshTime(); - } - - @Override - public void execute(Promise promise) - { - for (ThrowingRunnable listener : configChangeListeners) - { - try - { - listener.run(); - } - catch (Throwable e) - { - LOGGER.error("There was an error with callback {}", listener, e); - } - } - promise.tryComplete(); - } - - // skip if any of the following condition is true - // - sidecar schema not enabled or cdc not enabled - // - both configs have not changed - @Override - public ScheduleDecision scheduleDecision() - { - if (!schemaKeyspaceConfiguration.isEnabled() || !cdcConfiguration.isEnabled()) - { - LOGGER.trace("Skipping config refreshing"); - return ScheduleDecision.SKIP; - } - - Map newKafkaConfigMappings; - Map newCdcConfigMappings; - try - { - newKafkaConfigMappings = kafkaConfigAccessor.getConfig().getConfigs(); - newCdcConfigMappings = cdcConfigAccessor.getConfig().getConfigs(); - } - catch (Throwable e) - { - LOGGER.error("Failed to access cdc/kafka configs", e); - return ScheduleDecision.SKIP; - } - - boolean shouldSkip = true; - if (!newKafkaConfigMappings.equals(kafkaConfigMappings)) - { - shouldSkip = false; - kafkaConfigMappings = newKafkaConfigMappings; - } - if (!newCdcConfigMappings.equals(cdcConfigMappings)) - { - shouldSkip = false; - cdcConfigMappings = newCdcConfigMappings; - } - return shouldSkip ? ScheduleDecision.SKIP : ScheduleDecision.EXECUTE; - } +// if (configRefreshNotifier != null && +// configRefreshNotifier.scheduleDecision() == ScheduleDecision.EXECUTE) +// { +// configRefreshNotifier.execute(Promise.promise()); +// } } enum ConfigKeys diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcDynamicSidecarInstancesProvider.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcDynamicSidecarInstancesProvider.java new file mode 100644 index 000000000..3be68f951 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcDynamicSidecarInstancesProvider.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.sidecar.cdc; + +import java.util.List; +import java.util.stream.Collectors; + + +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstance; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.sidecar.client.SidecarInstancesProvider; +import org.apache.cassandra.sidecar.cluster.InstancesMetadata; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; + +/** + * A {@link SidecarInstancesProvider} implementation that returns Sidecar instances based on the configured + * {@link InstancesMetadata} for the local Sidecar + */ +public class CdcDynamicSidecarInstancesProvider implements CdcSidecarInstancesProvider +{ + private final InstancesMetadata instancesMetadata; + private final ServiceConfiguration serviceConfiguration; + + public CdcDynamicSidecarInstancesProvider(InstancesMetadata instancesMetadata, ServiceConfiguration serviceConfiguration) + { + this.instancesMetadata = instancesMetadata; + this.serviceConfiguration = serviceConfiguration; + } + + /** + * {@inheritDoc} + */ + @Override + public List instances() + { + return instancesMetadata.instances() + .stream() + .map(instanceMetadata -> new CdcSidecarInstanceImpl(instanceMetadata.host(), serviceConfiguration.port())) + .collect(Collectors.toList()); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java new file mode 100644 index 000000000..87c375a40 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java @@ -0,0 +1,31 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.function.Consumer; + +import org.apache.cassandra.cdc.api.EventConsumer; +import org.apache.cassandra.cdc.kafka.KafkaPublisher; +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.jetbrains.annotations.NotNull; + +/** + * Implementation class for a Cdc Event Consumer using Kafka + */ +public class CdcEventConsumer implements EventConsumer +{ + private final transient KafkaPublisher kafka; + + public CdcEventConsumer(KafkaPublisher kafka) + { + this.kafka = kafka; + } + + public void accept(CdcEvent cdcEvent) + { + kafka.processEvent(cdcEvent); + } + + public @NotNull Consumer andThen(@NotNull Consumer after) + { + return EventConsumer.super.andThen(after); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java new file mode 100644 index 000000000..369fed1e7 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -0,0 +1,247 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.cassandra.cdc.api.CdcOptions; +import org.apache.cassandra.cdc.api.EventConsumer; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.cdc.api.TokenRangeSupplier; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.cdc.sidecar.SidecarCdc; +import org.apache.cassandra.cdc.sidecar.SidecarCdcClient; +import org.apache.cassandra.cdc.sidecar.SidecarCdcStats; +import org.apache.cassandra.cdc.sidecar.SidecarStatePersister; +import org.apache.cassandra.cdc.stats.ICdcStats; +import org.apache.cassandra.secrets.SecretsProvider; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool; +import org.apache.cassandra.sidecar.coordination.RangeManager; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.utils.AsyncExecutor; +import org.jetbrains.annotations.NotNull; + + +/** + * Class handling CDC iterators + */ +public class CdcManager +{ + private static final Logger LOGGER = LoggerFactory.getLogger(CdcManager.class); + private final CdcConfig conf; + private final RangeManager rangeManager; + private final InstanceMetadataFetcher instanceFetcher; + private final EventConsumer eventConsumer; + private final SchemaSupplier schemaSupplier; + private final ClusterConfigProvider clusterConfigProvider; + private final CdcSidecarInstancesProvider sidecarInstancesProvider; + private final SecretsProvider secretsProvider; + private final SidecarCdcClient.ClientConfig clientConfig; + private final ICdcStats cdcStats; + private List consumers = new ArrayList<>(); + private final TaskExecutorPool taskExecutorPool; + private final CdcDatabaseAccessor cdcDatabaseAccessor; + + + public CdcManager(EventConsumer eventConsumer, + SchemaSupplier schemaSupplier, + CdcConfig conf, + RangeManager rangeManager, + InstanceMetadataFetcher instanceFetcher, + ClusterConfigProvider clusterConfigProvider, + CdcSidecarInstancesProvider sidecarInstancesProvider, + SecretsProvider secretsProvider, + SidecarCdcClient.ClientConfig clientConfig, + ICdcStats cdcStats, + TaskExecutorPool taskExecutorPool, + CdcDatabaseAccessor cdcDatabaseAccessor) + { + this.eventConsumer = eventConsumer; + this.schemaSupplier = schemaSupplier; + this.conf = conf; + this.rangeManager = rangeManager; + this.instanceFetcher = instanceFetcher; + this.clusterConfigProvider = clusterConfigProvider; + this.sidecarInstancesProvider = sidecarInstancesProvider; + this.secretsProvider = secretsProvider; + this.clientConfig = clientConfig; + this.cdcStats = cdcStats; + this.taskExecutorPool = taskExecutorPool; + this.cdcDatabaseAccessor = cdcDatabaseAccessor; + } + + List buildCdcConsumers() + { + Map> ownedRanges = rangeManager.ownedTokenRanges(); + if (ownedRanges == null || ownedRanges.isEmpty()) + { + throw new IllegalStateException("No owned token ranges right now, cql session may still be initializing."); + } + + // NEW: Deduplicate by (instanceId, tokenRange) to prevent duplicate consumers + Map uniqueConsumers = new HashMap<>(); + + ownedRanges.entrySet().stream() + .flatMap(entry -> + entry.getValue().stream().map(range -> { + Integer instanceId = getInstanceId(entry.getKey()); + + // Create unique key: "instanceId:rangeStart:rangeEnd" + String uniqueKey = String.format("%d:%s:%s", + instanceId, + range.startAsBigInt(), + range.endAsBigInt()); + + // Only create consumer if not already created for this (instance, range) + return uniqueConsumers.computeIfAbsent(uniqueKey, k -> { + try + { + return loadOrBuildCdcConsumer(instanceId, + clusterConfigProvider, + eventConsumer, + schemaSupplier, + () -> org.apache.cassandra.bridge.TokenRange.openClosed(range.startAsBigInt(), range.endAsBigInt()), + sidecarInstancesProvider, + secretsProvider, + clientConfig, + conf, + cdcStats, + taskExecutorPool); + } + catch (IOException e) + { + throw new RuntimeException(e); + } + }); + })) + .collect(Collectors.toList()); + + consumers = new ArrayList<>(uniqueConsumers.values()); + return consumers; + } + + + SidecarCdc loadOrBuildCdcConsumer(Integer instanceId, + ClusterConfigProvider clusterConfigProvider, + EventConsumer eventConsumer, + SchemaSupplier schemaSupplier, + TokenRangeSupplier tokenRangeSupplier, + CdcSidecarInstancesProvider sidecarInstancesProvider, + SecretsProvider secretsProvider, + SidecarCdcClient.ClientConfig clientConfig, + CdcConfig conf, + ICdcStats cdcStats, + TaskExecutorPool taskExecutorPool) throws IOException + { + return buildConsumer(conf.jobId(), + instanceId, + new SidecarCdcOptions(instanceFetcher), + clusterConfigProvider, + eventConsumer, + schemaSupplier, + tokenRangeSupplier, + sidecarInstancesProvider, + clientConfig, + secretsProvider, + cdcStats, + taskExecutorPool); + } + + public void startConsumers() + { + consumers.forEach(SidecarCdc::initSchema); + consumers.forEach(SidecarCdc::start); + } + + public void stopConsumers() + { + consumers.forEach(SidecarCdc::stop); + } + + + private Integer getInstanceId(String instanceIp) + { + for (InstanceMetadata instance : instanceFetcher.allLocalInstances()) + { + String configuredHost = instance.ipAddress(); + + // Option 1a: Normalize both to InetAddress and compare + if (resolveToSameAddress(instanceIp, configuredHost)) + { + return instance.id(); + } + } + LOGGER.warn("Requested IP {} does not match with any instances", instanceIp); + return -1; + } + + private boolean resolveToSameAddress(String host1, String host2) + { + try + { + InetAddress addr1 = InetAddress.getByName(host1); + InetAddress addr2 = InetAddress.getByName(host2); + return addr1.equals(addr2); + } + catch (UnknownHostException e) + { + LOGGER.warn("Could not resolve hostname: {}", e.getMessage()); + return host1.equals(host2); // Fallback to string comparison + } + } + + + public SidecarCdc buildConsumer(@NotNull String jobId, + int partitionId, + CdcOptions cdcOptions, + ClusterConfigProvider clusterConfigProvider, + EventConsumer eventConsumer, + SchemaSupplier schemaSupplier, + TokenRangeSupplier tokenRangeSupplier, + CdcSidecarInstancesProvider sidecarInstancesProvider, + SidecarCdcClient.ClientConfig clientConfig, + SecretsProvider secretsProvider, + ICdcStats cdcStats, + TaskExecutorPool taskExecutorPool) throws IOException + { + + AsyncExecutor asyncExecutor = new ExecutorPoolsExecutor(taskExecutorPool); + + final SidecarStatePersister sidecarStatePersister = getSidecarStatePersister(cdcOptions, asyncExecutor); + return (SidecarCdc) SidecarCdc.builder(jobId, + partitionId, + cdcOptions, + clusterConfigProvider, + eventConsumer, + schemaSupplier, + tokenRangeSupplier, + sidecarInstancesProvider, + clientConfig, + secretsProvider, + cdcStats).withExecutor(asyncExecutor).withStatePersister(sidecarStatePersister).build(); + } + + private @NotNull SidecarStatePersister getSidecarStatePersister(CdcOptions cdcOptions, AsyncExecutor asyncExecutor) + { + SidecarStatePersister sidecarStatePersister = new SidecarStatePersister(org.apache.cassandra.cdc.sidecar.SidecarCdcOptions.DEFAULT, + cdcOptions, + SidecarCdcStats.STUB, + new StateSidecarCdcCassandraClient(cdcDatabaseAccessor), + asyncExecutor); + sidecarStatePersister.start(); + return sidecarStatePersister; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java new file mode 100644 index 000000000..4c137f039 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -0,0 +1,383 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.Handler; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; +import io.vertx.core.eventbus.Message; +import org.apache.cassandra.cdc.CdcLogMode; +import org.apache.cassandra.cdc.api.EventConsumer; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.cdc.kafka.KafkaPublisher; +import org.apache.cassandra.cdc.kafka.TopicSupplier; +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.cdc.sidecar.SidecarCdc; +import org.apache.cassandra.cdc.sidecar.SidecarCdcClient; +import org.apache.cassandra.cdc.stats.ICdcStats; +import org.apache.cassandra.secrets.SecretsProvider; +import org.apache.cassandra.secrets.SslConfig; +import org.apache.cassandra.secrets.SslConfigSecretsProvider; +import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.concurrent.ExecutorPools; +import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool; +import org.apache.cassandra.sidecar.config.KeyStoreConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.SslConfiguration; +import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; +import org.apache.cassandra.sidecar.coordination.RangeManager; +import org.apache.cassandra.sidecar.coordination.TokenRingProvider; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; +import org.apache.cassandra.sidecar.tasks.PeriodicTask; +import org.apache.cassandra.sidecar.tasks.ScheduleDecision; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.serialization.Serializer; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CACHE_WARMED_UP; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIGURATION_CHANGED; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SERVER_STOP; + +/** + * Class that handles CDC life cycle + */ +@Singleton +public class CdcPublisher implements Handler>, PeriodicTask +{ + static final Logger LOGGER = LoggerFactory.getLogger(CdcPublisher.class); + static final long INITIALIZATION_LOOP_DELAY_MILLIS = 1000; + private final Vertx vertx; + private final TaskExecutorPool executorPools; + private final CdcConfig conf; + private volatile boolean isRunning = false; + private volatile boolean isInitialized = false; + private volatile boolean cdcCacheWarmedUp = false; + private final CdcDatabaseAccessor databaseAccessor; + private final VirtualTablesDatabaseAccessor virtualTables; + private final SidecarCdcStats sidecarCdcStats; + private RangeManager rangeManager; + private final TokenRingProvider tokenRingProvider; + private final SchemaSupplier schemaSupplier; + private final CdcSidecarInstancesProvider sidecarInstancesProvider; + private final InstanceMetadataFetcher instanceMetadataFetcher; + private final ClusterConfigProvider clusterConfigProvider; + private final SidecarCdcClient.ClientConfig clientConfig; + private final ICdcStats cdcStats; + private final SidecarConfiguration sidecarConfiguration; + private CdcManager cdcManager; + Serializer avroSerializer; + + @Inject + public CdcPublisher(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + ExecutorPools executorPools, + ClusterConfigProvider clusterConfigProvider, + SchemaSupplier schemaSupplier, + CdcSidecarInstancesProvider sidecarInstancesProvider, + SidecarCdcClient.ClientConfig clientConfig, + InstanceMetadataFetcher instanceMetadataFetcher, + CdcConfig conf, + CdcDatabaseAccessor databaseAccessor, + ICdcStats cdcStats, + TokenRingProvider tokenRingProvider, + VirtualTablesDatabaseAccessor virtualTables, + SidecarCdcStats sidecarCdcStats, + Serializer avroSerializer) + { + this.vertx = vertx; + + this.sidecarCdcStats = sidecarCdcStats; + this.executorPools = executorPools.internal(); + this.conf = conf; + this.databaseAccessor = databaseAccessor; + this.tokenRingProvider = tokenRingProvider; + this.virtualTables = virtualTables; + + this.schemaSupplier = schemaSupplier; + this.sidecarInstancesProvider = sidecarInstancesProvider; + this.instanceMetadataFetcher = instanceMetadataFetcher; + this.clusterConfigProvider = clusterConfigProvider; + this.clientConfig = clientConfig; + this.cdcStats = cdcStats; + this.sidecarConfiguration = sidecarConfiguration; + this.avroSerializer = avroSerializer; + vertx.eventBus().localConsumer(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), this); + vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), this); + vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), this); + vertx.eventBus().localConsumer(ON_SERVER_STOP.address(), this); + vertx.eventBus().localConsumer(ON_CDC_CACHE_WARMED_UP.address(), this); + vertx.eventBus().localConsumer(ON_CDC_CONFIGURATION_CHANGED.address(), new ConfigChangedHandler()); + } + + public SecretsProvider secretsProvider() + { + SslConfiguration sslConfiguration = sidecarConfiguration.sidecarClientConfiguration().sslConfiguration(); + + if (sslConfiguration == null || !sslConfiguration.enabled()) + { + return null; + } + + Map sslConfigMap = new HashMap<>(); + + sslConfigMap.put("enabled", sslConfiguration.enabled() + ""); + sslConfigMap.put("preferOpenSSL", sslConfiguration.preferOpenSSL() + ""); + sslConfigMap.put("clientAuth", sslConfiguration.clientAuth()); + sslConfigMap.put("cipherSuites", String.join(",", sslConfiguration.cipherSuites())); + sslConfigMap.put("secureTransportProtocols", String.join(",", sslConfiguration.secureTransportProtocols())); + sslConfigMap.put("handshakeTimeout", sslConfiguration.handshakeTimeout().toString()); + + if (sslConfiguration.isKeystoreConfigured()) + { + KeyStoreConfiguration keystore = sslConfiguration.keystore(); + sslConfigMap.put("keystorePath", keystore.path()); + sslConfigMap.put("keystorePassword", keystore.password()); + sslConfigMap.put("keystoreType", keystore.type()); + } + + if (sslConfiguration.isTrustStoreConfigured()) + { + KeyStoreConfiguration truststore = sslConfiguration.truststore(); + sslConfigMap.put("truststorePath", truststore.path()); + sslConfigMap.put("truststorePassword", truststore.password()); + sslConfigMap.put("truststoreType", truststore.type()); + } + + SslConfig sslConfig = SslConfig.create(sslConfigMap); + return new SslConfigSecretsProvider(sslConfig); + } + + public EventConsumer eventConsumer(CdcConfig conf, + Serializer avroSerializer) + { + KafkaProducer producer = new KafkaProducer<>(conf.kafkaConfigs()); + KafkaPublisher kafkaPublisher = new KafkaPublisher(TopicSupplier.staticTopicSupplier(conf.kafkaTopic()), + producer, + avroSerializer, + conf.maxRecordSizeBytes(), + conf.failOnRecordTooLargeError(), + conf.failOnKafkaError(), + CdcLogMode.FULL); + return new CdcEventConsumer(kafkaPublisher); + } + + private class ConfigChangedHandler implements Handler> + { + public void handle(Message event) + { + sidecarCdcStats.captureCdcConfigChange(); + // Execute restart on worker thread to avoid blocking event loop + executorPools.executeBlocking(() -> { + restart(); + return null; + }); + } + } + + @SuppressWarnings("resource") + private synchronized void run() throws IllegalStateException + { + if (isRunning) + { + return; + } + databaseAccessor.session(); // throws IllegalStateException if session unavailable + + cdcManager = new CdcManager(eventConsumer(conf, avroSerializer), + schemaSupplier, + conf, + rangeManager, + instanceMetadataFetcher, + clusterConfigProvider, + sidecarInstancesProvider, + secretsProvider(), + clientConfig, + cdcStats, + this.executorPools, + databaseAccessor); + + List consumers = cdcManager.buildCdcConsumers(); + cdcManager.startConsumers(); + LOGGER.info("{} CDC iterators started successfully", consumers.size()); + isRunning = true; + sidecarCdcStats.captureCdcStarted(consumers.size()); + } + + protected synchronized void restart() + { + try + { + stop(); + initialize(); + + LOGGER.info("Iterators restarted."); + sidecarCdcStats.captureCdcRestart(); + } + catch (Exception e) + { + LOGGER.error("Failed to restart iterators", e); + sidecarCdcStats.captureCdcStartFailure(e); + } + } + + public boolean isRunning() + { + return isRunning; + } + + public synchronized void stop() + { + if (!isRunning) + { + return; + } + + try + { + cdcManager.stopConsumers(); + sidecarCdcStats.captureCdcStopped(); + } + catch (Throwable t) + { + LOGGER.error("Failed to gracefully shutdown CDC", t); + sidecarCdcStats.captureCdcStopFailed(t); + } + finally + { + isRunning = false; + isInitialized = false; + } + } + + private void initialize() + { + try + { + if (this.rangeManager == null) + { + this.rangeManager = new ContentionFreeRangeManager(vertx, tokenRingProvider); + } + String localDc = rangeManager.getLocalDcSafe(); + if (conf.datacenter() != null && !conf.datacenter().isEmpty() && !conf.datacenter().equals(localDc)) + { + LOGGER.info("Cdc not enabled in this DC localDc={} cdcDc={}", localDc, conf.datacenter()); + return; + } + else if (virtualTables.isCdcOnRepairEnabled()) + { + LOGGER.warn("Cannot run CDC while cdc on repair is enabled, disable cdc_on_repair_enabled in the yaml file."); + sidecarCdcStats.captureCdcOnRepairEnabled(); + return; + } + else if (conf.cdcEnabled()) + { + try + { + LOGGER.info("Initialization of all delegates complete, attempting to start CDC"); + isInitialized = true; + } + catch (Throwable t) + { + LOGGER.error("Error initializing CDC", t); + sidecarCdcStats.captureCdcStartFailure(t); + } + } + + } + catch (Exception e) + { + LOGGER.error("Unexpected error initializing CdcPublisher", e); + sidecarCdcStats.captureCdcStartFailure(e); + } + + } + + // EventBus handlers + @Override + public synchronized void handle(Message msg) + { + if (msg.address().equals(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address())) + { + handleTokenRangeChange(); + } + else if (msg.address().equals(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address())) + { + handleRangeGained((RangeManager.RangeChangeEvent) msg.body()); + } + else if (msg.address().equals(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address())) + { + handleRangeLost((RangeManager.RangeChangeEvent) msg.body()); + } + else if (msg.address().equals(ON_SERVER_STOP.address())) + { + stop(); + } + else if (msg.address().equals(ON_CDC_CACHE_WARMED_UP.address())) + { + cdcCacheWarmedUp = true; + } + } + + protected synchronized void handleTokenRangeChange() + { + if (isRunning) + { + //TODO: detect if topology change affects active cdc iterators + LOGGER.info("Token Range changed, probably due to a change in cluster topology, restarting iterators"); + sidecarCdcStats.captureCdcClusterTopologyChange(); + restart(); + } + } + + protected synchronized void handleRangeGained(RangeManager.RangeChangeEvent event) + { + if (isRunning) + { + //TODO: start/restart consumers based on ranges gained in event + sidecarCdcStats.captureCdcClusterTopologyChange(); + restart(); + } + } + + protected synchronized void handleRangeLost(RangeManager.RangeChangeEvent event) + { + if (isRunning) + { + //TODO: stop/restart consumers based on ranges lost in event + sidecarCdcStats.captureCdcTokenRangeLost(); + restart(); + } + } + + public DurationSpec delay() + { + return MillisecondBoundConfiguration.parse(INITIALIZATION_LOOP_DELAY_MILLIS + "ms"); + } + + @Override + public void execute(Promise promise) + { + run(); + promise.complete(); + } + + @Override + public ScheduleDecision scheduleDecision() + { + return isInitialized && cdcCacheWarmedUp + ? ScheduleDecision.EXECUTE + : ScheduleDecision.SKIP; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java new file mode 100644 index 000000000..4da13f618 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java @@ -0,0 +1,105 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +import com.google.inject.Singleton; +import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CdcBridge; +import org.apache.cassandra.bridge.CdcBridgeFactory; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.sidecar.common.response.NodeSettings; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.utils.CdcUtil; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.data.CqlTable; +import org.apache.cassandra.spark.data.ReplicationFactor; +import org.apache.cassandra.spark.data.partitioner.Partitioner; +import org.apache.cassandra.spark.utils.CqlUtils; +import org.apache.cassandra.spark.utils.TableIdentifier; +import org.jetbrains.annotations.NotNull; + + +/** + * Class providing the schema for CDC + */ +@Singleton +public class CdcSchemaSupplier implements SchemaSupplier +{ + private final InstanceMetadataFetcher instanceMetadataFetcher; + private final CassandraBridgeFactory cassandraBridgeFactory; + private final CdcDatabaseAccessor cdcDatabaseAccessor; + private final ConcurrentHashMap tableIdCache = new ConcurrentHashMap<>(); + + public CdcSchemaSupplier(InstanceMetadataFetcher instanceMetadataFetcher, CassandraBridgeFactory cassandraBridgeFactory, CdcDatabaseAccessor cdcDatabaseAccessor) + { + this.instanceMetadataFetcher = instanceMetadataFetcher; + this.cassandraBridgeFactory = cassandraBridgeFactory; + this.cdcDatabaseAccessor = cdcDatabaseAccessor; + } + + public CompletableFuture> getCdcEnabledTables() + { + String schema = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().metadata().exportSchemaAsString()); + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + CassandraBridge cassandraBridge = cassandraBridgeFactory.get(nodeSettings.releaseVersion()); + CdcBridge cdcBridge = CdcBridgeFactory.getCdcBridge(cassandraBridge); + + Set cqlTables = buildCdcTables(schema, + cdcDatabaseAccessor.partitioner(), + tableIdCache, + cdcDatabaseAccessor::getTableId, + cassandraBridge); + cdcBridge.updateCdcSchema(cqlTables, getPartitioner(nodeSettings), + ((keyspace, table) -> tableIdCache.get(TableIdentifier.of(keyspace, table)))); + return CompletableFuture.completedFuture(cqlTables); + } + + private Partitioner getPartitioner(NodeSettings nodeSettings) + { + if (nodeSettings.partitioner().contains(".")) + { + String[] splitPartitionerName = nodeSettings.partitioner().split("\\."); + return Partitioner.valueOf(splitPartitionerName[splitPartitionerName.length - 1]); + } + return Partitioner.valueOf(nodeSettings.partitioner()); + } + + private static Set buildCdcTables(@NotNull String fullSchema, + @NotNull Partitioner partitioner, + @NotNull ConcurrentHashMap tableIdCache, + @NotNull Function tableIdLoaderFunction, + @NotNull CassandraBridge cassandraBridge) + { + Map createStmts = CdcUtil.extractCdcTables(fullSchema); + Map> udtsPerKeyspace = createStmts.keySet() + .stream() + .map(TableIdentifier::keyspace) + .distinct() // remove duplicated keyspace strings + .collect(Collectors.toMap(Function.identity(), + keyspace -> CqlUtils.extractUdts(fullSchema, keyspace))); + + Map tableIds = createStmts.keySet() + .stream() + .collect(Collectors.toMap(Function.identity(), + id -> tableIdCache.computeIfAbsent(id, tableIdLoaderFunction))); + + return createStmts.entrySet().stream() + .map(e -> + { + TableIdentifier id = e.getKey(); + ReplicationFactor rf = CqlUtils.extractReplicationFactor(fullSchema, id.keyspace()); + + return cassandraBridge.buildSchema(e.getValue(), id.keyspace(), rf, + partitioner, udtsPerKeyspace.get(id.keyspace()), + tableIds.get(id), 0, true); + }) + .collect(Collectors.toSet()); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSidecarInstanceImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSidecarInstanceImpl.java new file mode 100644 index 000000000..1d122fdca --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSidecarInstanceImpl.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.cassandra.sidecar.cdc; + +import java.util.Objects; + +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstance; + +/** + * A simple implementation of the {@link CdcSidecarInstance} interface + */ +public class CdcSidecarInstanceImpl implements CdcSidecarInstance +{ + protected int port; + protected String hostname; + + /** + * Constructs a new Sidecar instance with the given {@code port} and {@code hostname} + * + * @param hostname the host name where Sidecar is running + * @param port the port where Sidecar is running + */ + public CdcSidecarInstanceImpl(String hostname, int port) + { + if (port < 1 || port > 65535) + { + throw new IllegalArgumentException(String.format("Invalid port number for the Sidecar service: %d", + port)); + } + this.port = port; + this.hostname = Objects.requireNonNull(hostname, "The Sidecar hostname must be non-null"); + } + + /** + * {@inheritDoc} + */ + @Override + public int port() + { + return port; + } + + /** + * {@inheritDoc} + */ + @Override + public String hostname() + { + return hostname; + } + + /** + * {@inheritDoc} + */ + @Override + public boolean equals(Object o) + { + if (this == o) + { + return true; + } + if (o == null || getClass() != o.getClass()) + { + return false; + } + CdcSidecarInstanceImpl that = (CdcSidecarInstanceImpl) o; + return port == that.port && Objects.equals(hostname, that.hostname); + } + + /** + * {@inheritDoc} + */ + @Override + public int hashCode() + { + return Objects.hash(port, hostname); + } + + /** + * {@inheritDoc} + */ + @Override + public String toString() + { + return "SidecarInstanceImpl{" + + "port=" + port + + ", hostname='" + hostname + '\'' + + '}'; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java new file mode 100644 index 000000000..d15ef6251 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java @@ -0,0 +1,76 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; + +import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool; +import org.apache.cassandra.spark.utils.AsyncExecutor; + +/** + * Wrapper to pass an executor pool to cdc classes. + */ +public class ExecutorPoolsExecutor implements AsyncExecutor +{ + public ExecutorPoolsExecutor(TaskExecutorPool executorPool) + { + this.executorPool = executorPool; + } + + private final TaskExecutorPool executorPool; + + @Override + public CompletableFuture submit(Supplier blockingAction) + { + try + { + return executorPool.executeBlocking(promise -> promise.complete(blockingAction.get()), false) + .toCompletionStage() + .toCompletableFuture(); + } + catch (Exception e) + { + return CompletableFuture.failedFuture(e); + } + } + + public CompletableFuture submit(Runnable blockingAction) + { + return executorPool.executeBlocking((promise) -> { + blockingAction.run(); + promise.complete(); + }, false).toCompletionStage().toCompletableFuture() + .thenApply(a -> null); + } + + public CompletableFuture schedule(Runnable task, long delayMillis) + { + final CompletableFuture future = new CompletableFuture<>(); + executorPool.setTimer(delayMillis, (timerId) -> { + try + { + task.run(); + future.complete(null); + } + catch (Throwable t) + { + future.completeExceptionally(t); + } + }); + return future; + } + + public long periodicTimer(Runnable task, long delayMillis) + { + return executorPool.setPeriodic(delayMillis, (promise) -> task.run()); + } + + public boolean cancelTimer(long timerId) + { + return executorPool.cancelTimer(timerId); + } + + static ExecutorPoolsExecutor wrap(TaskExecutorPool executorPool) + { + return new ExecutorPoolsExecutor(executorPool); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java new file mode 100644 index 000000000..9cb5a31db --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java @@ -0,0 +1,35 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Map; + +import org.apache.cassandra.cdc.api.CdcOptions; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.data.ReplicationFactor; + +/** + * Specific sidecar CDC options + */ +public class SidecarCdcOptions implements CdcOptions +{ + + private final InstanceMetadataFetcher instanceMetadataFetcher; + + public SidecarCdcOptions(InstanceMetadataFetcher instanceMetadataFetcher) + { + this.instanceMetadataFetcher = instanceMetadataFetcher; + } + + + public ReplicationFactor replicationFactor(String keyspace) + { + + Map replication = instanceMetadataFetcher + .callOnFirstAvailableInstance(instance-> instance.delegate().metadata().getKeyspace(keyspace).getReplication()); + return new ReplicationFactor(replication); + } + + public String dc() + { + return instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings().datacenter()); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java new file mode 100644 index 000000000..f5f17df54 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java @@ -0,0 +1,42 @@ +package org.apache.cassandra.sidecar.cdc; + + +import com.google.inject.Inject; +import org.apache.avro.Schema; +import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CdcBridgeFactory; +import org.apache.cassandra.cdc.avro.CqlToAvroSchemaConverter; +import org.apache.cassandra.sidecar.common.response.NodeSettings; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.data.CqlTable; + +/** + * Class to convert CQL schemas into Avro schemas + */ +public class SidecarCqlToAvroSchemaConverter implements CqlToAvroSchemaConverter +{ + private final InstanceMetadataFetcher instanceMetadataFetcher; + private final CassandraBridgeFactory cassandraBridgeFactory; + + + @Inject + public SidecarCqlToAvroSchemaConverter(InstanceMetadataFetcher instanceMetadataFetcher, + CassandraBridgeFactory cassandraBridgeFactory) + { + this.instanceMetadataFetcher = instanceMetadataFetcher; + this.cassandraBridgeFactory = cassandraBridgeFactory; + } + + public CassandraBridge cassandraBridge() + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + return cassandraBridgeFactory.get(nodeSettings.releaseVersion()); + } + + + public Schema convert(CqlTable cqlTable) + { + return CdcBridgeFactory.getCqlToAvroSchemaConverter(cassandraBridge()).convert(cqlTable); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java new file mode 100644 index 000000000..9442cd6ed --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java @@ -0,0 +1,39 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.stream.Stream; + +import com.datastax.driver.core.ResultSetFuture; +import org.apache.cassandra.bridge.TokenRange; +import org.apache.cassandra.cdc.sidecar.SidecarCdcCassandraClient; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Class managing the CDC state through the database accessor + */ +public class StateSidecarCdcCassandraClient implements SidecarCdcCassandraClient +{ + final CdcDatabaseAccessor cdcDatabaseAccessor; + + public StateSidecarCdcCassandraClient(CdcDatabaseAccessor cdcDatabaseAccessor) + { + this.cdcDatabaseAccessor = cdcDatabaseAccessor; + } + + public List storeStateAsync(@NotNull String jobId, @NotNull TokenRange range, @NotNull ByteBuffer buf, long timestamp) + { + return cdcDatabaseAccessor.storeStateAsync(jobId, range, buf, timestamp); + } + + public Stream loadStateForRange(String jobId, @Nullable TokenRange tokenRange) + { + if (tokenRange == null) + { + return Stream.empty(); + } + return cdcDatabaseAccessor.loadStateForRange(jobId, tokenRange); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java new file mode 100644 index 000000000..9f32c7beb --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java @@ -0,0 +1,66 @@ +package org.apache.cassandra.sidecar.codecs; + +import java.math.BigInteger; + +import org.apache.commons.lang3.mutable.MutableInt; + +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; + +/** + * Message codec for encoding and decoding BigInteger values over the Vert.x event bus. + */ +public class BigIntegerCodec implements MessageCodec +{ + public static final BigIntegerCodec INSTANCE = new BigIntegerCodec(); + + /** + * Encodes a BigInteger to the wire buffer. + */ + public void encodeToWire(Buffer buf, BigInteger bigInteger) + { + CommonCodecs.BYTE_ARRAY.encodeToWire(buf, bigInteger.toByteArray()); + } + + /** + * Decodes a BigInteger from the wire buffer. + */ + public BigInteger decodeFromWire(MutableInt pos, Buffer buf) + { + byte[] ar = CommonCodecs.BYTE_ARRAY.decodeFromWire(pos.intValue(), buf); + pos.add(4 + ar.length); + return new BigInteger(ar); + } + + /** + * Decodes a BigInteger from the wire buffer at the specified position. + */ + public BigInteger decodeFromWire(int pos, Buffer buf) + { + return decodeFromWire(new MutableInt(pos), buf); + } + + /** + * Returns the BigInteger unchanged for local delivery. + */ + public BigInteger transform(BigInteger bigInteger) + { + return bigInteger; + } + + /** + * Returns the codec name. + */ + public String name() + { + return "big-integer"; + } + + /** + * Returns the system codec ID. + */ + public byte systemCodecID() + { + return -1; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java new file mode 100644 index 000000000..ca5e30f64 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java @@ -0,0 +1,78 @@ +package org.apache.cassandra.sidecar.codecs; + +import java.util.HashMap; +import java.util.Map; + +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.tasks.CdcConfigRefresherNotifierTask; + +/** + * Message codec for encoding and decoding Cdc Config Mappings change events over the Vert.x event bus. + */ +public class CdcConfigMappingsCodec implements MessageCodec +{ + public static final CdcConfigMappingsCodec INSTANCE = new CdcConfigMappingsCodec(); + + + public void encodeToWire(Buffer buffer, CdcConfigRefresherNotifierTask.ConfigMappings configMappings) + { + CommonCodecs.SHORT.encodeToWire(buffer, (short) configMappings.getCdcConfigMappings().size()); + for (Map.Entry entry : configMappings.getCdcConfigMappings().entrySet()) + { + CommonCodecs.STRING.encodeToWire(buffer, entry.getKey()); + CommonCodecs.STRING.encodeToWire(buffer, entry.getValue()); + } + + CommonCodecs.SHORT.encodeToWire(buffer, (short) configMappings.getKafkaConfigMappings().size()); + for (Map.Entry entry : configMappings.getKafkaConfigMappings().entrySet()) + { + CommonCodecs.STRING.encodeToWire(buffer, entry.getKey()); + CommonCodecs.STRING.encodeToWire(buffer, entry.getValue()); + } + + } + + public CdcConfigRefresherNotifierTask.ConfigMappings decodeFromWire(int pos, Buffer buffer) + { + int cdcMapSize = CommonCodecs.SHORT.decodeFromWire(pos, buffer); + pos += 2; + + Map cdcConfigMapping = new HashMap<>(); + for (int i = 0; i < cdcMapSize; i++) + { + String key = CommonCodecs.STRING.decodeFromWire(pos, buffer); + String value = CommonCodecs.STRING.decodeFromWire(pos, buffer); + cdcConfigMapping.put(key, value); + } + + Map kafkaConfigMapping = new HashMap<>(); + int kafkaMapSize = CommonCodecs.SHORT.decodeFromWire(pos, buffer); + for (int i = 0; i < kafkaMapSize; i++) + { + String key = CommonCodecs.STRING.decodeFromWire(pos, buffer); + String value = CommonCodecs.STRING.decodeFromWire(pos, buffer); + kafkaConfigMapping.put(key, value); + } + + CdcConfigRefresherNotifierTask.ConfigMappings configMappings = new CdcConfigRefresherNotifierTask.ConfigMappings(); + configMappings.setCdcConfigMappings(cdcConfigMapping); + configMappings.setKafkaConfigMappings(kafkaConfigMapping); + return configMappings; + } + + public CdcConfigRefresherNotifierTask.ConfigMappings transform(CdcConfigRefresherNotifierTask.ConfigMappings configMappings) + { + return configMappings; + } + + public String name() + { + return "cdc-config-mappings"; + } + + public byte systemCodecID() + { + return -1; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java new file mode 100644 index 000000000..cca4a408b --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java @@ -0,0 +1,24 @@ +package org.apache.cassandra.sidecar.codecs; + +import io.vertx.core.eventbus.impl.codecs.BooleanMessageCodec; +import io.vertx.core.eventbus.impl.codecs.ByteArrayMessageCodec; +import io.vertx.core.eventbus.impl.codecs.IntMessageCodec; +import io.vertx.core.eventbus.impl.codecs.ShortMessageCodec; +import io.vertx.core.eventbus.impl.codecs.StringMessageCodec; + +/** + * Common message codecs for primitive types used across the event bus. + */ +public class CommonCodecs +{ + /** String message codec instance. */ + public static final StringMessageCodec STRING = new StringMessageCodec(); + /** Short message codec instance. */ + public static final ShortMessageCodec SHORT = new ShortMessageCodec(); + /** Byte array message codec instance. */ + public static final ByteArrayMessageCodec BYTE_ARRAY = new ByteArrayMessageCodec(); + /** Integer message codec instance. */ + public static final IntMessageCodec INT = new IntMessageCodec(); + /** Boolean message codec instance. */ + public static final BooleanMessageCodec BOOL = new BooleanMessageCodec(); +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java new file mode 100644 index 000000000..0be449549 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java @@ -0,0 +1,134 @@ +package org.apache.cassandra.sidecar.codecs; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; + +import org.apache.commons.lang3.mutable.MutableInt; + +import io.netty.util.CharsetUtil; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.tasks.ClusterTopologyMonitor; + +/** + * Message codec for encoding and decoding datacenter-local topology change events over the Vert.x event bus. + */ +public class DcLocalTopologyChangeEventCodec implements MessageCodec +{ + public static final DcLocalTopologyChangeEventCodec INSTANCE = new DcLocalTopologyChangeEventCodec(); + + /** + * Encodes a topology change event to the wire buffer. + */ + public void encodeToWire(Buffer buf, ClusterTopologyMonitor.DcLocalTopologyChangeEvent event) + { + CommonCodecs.STRING.encodeToWire(buf, event.dc); + encodeMap(buf, event.prev); + encodeMap(buf, event.curr); + } + + protected String decodeString(MutableInt pos, Buffer buf) + { + int len = buf.getInt(pos.intValue()); + pos.add(4); + byte[] bytes = buf.getBytes(pos.intValue(), pos.intValue() + len); + pos.add(len); + return new String(bytes, CharsetUtil.UTF_8); + } + + /** + * Decodes a topology change event from the wire buffer. + */ + public ClusterTopologyMonitor.DcLocalTopologyChangeEvent decodeFromWire(int pos, Buffer buf) + { + MutableInt mutablePos = new MutableInt(pos); + String dc = decodeString(mutablePos, buf); + Map> prev = decodeMap(mutablePos, buf); + Map> curr = decodeMap(mutablePos, buf); + + return new ClusterTopologyMonitor.DcLocalTopologyChangeEvent(dc, prev, Objects.requireNonNull(curr, "Current topology is null")); + } + + /** + * Encodes a map of instance IP to token ranges. + */ + protected static void encodeMap(Buffer buf, @Nullable Map> map) + { + CommonCodecs.SHORT.encodeToWire(buf, (short) (map == null ? -1 : map.size())); // map size + if (map != null) + { + for (Map.Entry> entry : map.entrySet()) + { + CommonCodecs.STRING.encodeToWire(buf, entry.getKey()); // instanceIP + final List ranges = entry.getValue(); + CommonCodecs.SHORT.encodeToWire(buf, (short) ranges.size()); // range size + for (TokenRange range : ranges) + { + RangeCodec.INSTANCE.encodeToWire(buf, range); + } + } + } + } + + /** + * Decodes a map of instance IP to token ranges. + */ + @Nullable + protected static Map> decodeMap(MutableInt pos, Buffer buf) + { + int mapSize = buf.getShort(pos.intValue()); + pos.add(2); + if (mapSize < 0) + { + return null; + } + + Map> map = new HashMap<>(mapSize); + for (int i = 0; i < mapSize; i++) + { + String instanceIp = INSTANCE.decodeString(pos, buf); + + int rangeSize = buf.getShort(pos.intValue()); + pos.add(2); + + List ranges = new ArrayList<>(rangeSize); + for (int j = 0; j < rangeSize; j++) + { + ranges.add(RangeCodec.INSTANCE.decodeFromWire(pos, buf)); + } + map.put(instanceIp, ranges); + } + + return map; + } + + /** + * Returns the event unchanged for local delivery. + */ + public ClusterTopologyMonitor.DcLocalTopologyChangeEvent transform(ClusterTopologyMonitor.DcLocalTopologyChangeEvent + event) + { + return event; // TopologyChangeEvent is immutable + } + + /** + * Returns the codec name. + */ + public String name() + { + return "topology-change-event"; + } + + /** + * Returns the system codec ID. + */ + public byte systemCodecID() + { + return -1; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java new file mode 100644 index 000000000..4d9bcea7c --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java @@ -0,0 +1,57 @@ +package org.apache.cassandra.sidecar.codecs; + +import org.apache.commons.lang3.mutable.MutableInt; + +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.coordination.RangeManager; + +/** + * Message codec for encoding and decoding token range change events over the Vert.x event bus. + */ +public class RangeChangeEventCodec implements MessageCodec +{ + public static final RangeChangeEventCodec INSTANCE = new RangeChangeEventCodec(); + + /** + * Encodes a range change event to the wire buffer. + */ + public void encodeToWire(Buffer buf, RangeManager.RangeChangeEvent event) + { + RangeMapCodec.INSTANCE.encodeToWire(buf, event.change); + RangeMapCodec.INSTANCE.encodeToWire(buf, event.newView); + } + + /** + * Decodes a range change event from the wire buffer. + */ + public RangeManager.RangeChangeEvent decodeFromWire(int pos, Buffer buf) + { + final MutableInt mutablePos = new MutableInt(pos); + return new RangeManager.RangeChangeEvent(RangeMapCodec.INSTANCE.decodeMap(mutablePos, buf), RangeMapCodec.INSTANCE.decodeMap(mutablePos, buf)); + } + + /** + * Returns the event unchanged for local delivery. + */ + public RangeManager.RangeChangeEvent transform(RangeManager.RangeChangeEvent event) + { + return event; // immutable + } + + /** + * Returns the codec name. + */ + public String name() + { + return "range-change-event"; + } + + /** + * Returns the system codec ID. + */ + public byte systemCodecID() + { + return -1; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java new file mode 100644 index 000000000..58e98431d --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java @@ -0,0 +1,83 @@ +package org.apache.cassandra.sidecar.codecs; + +import java.math.BigInteger; + +import com.google.common.collect.BoundType; +import org.apache.commons.lang3.mutable.MutableInt; + +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; + +/** + * Message codec for encoding and decoding TokenRange objects over the Vert.x event bus. + */ +public class RangeCodec implements MessageCodec +{ + public static final RangeCodec INSTANCE = new RangeCodec(); + + /** + * Encodes a TokenRange to the wire buffer. + */ + public void encodeToWire(Buffer buf, TokenRange range) + { + encodeType(buf, range.range.lowerBoundType()); + BigIntegerCodec.INSTANCE.encodeToWire(buf, range.range.lowerEndpoint().toBigInteger()); + encodeType(buf, range.range.upperBoundType()); + BigIntegerCodec.INSTANCE.encodeToWire(buf, range.range.upperEndpoint().toBigInteger()); + } + + /** + * Decodes a TokenRange from the wire buffer. + */ + public TokenRange decodeFromWire(MutableInt pos, Buffer buf) + { + BigInteger lower = BigIntegerCodec.INSTANCE.decodeFromWire(pos, buf); + + BigInteger upper = BigIntegerCodec.INSTANCE.decodeFromWire(pos, buf); + + return new TokenRange(lower, upper); + } + + /** + * Decodes a TokenRange from the wire buffer at the specified position. + */ + public TokenRange decodeFromWire(int pos, Buffer buf) + { + return decodeFromWire(new MutableInt(pos), buf); + } + + protected static void encodeType(Buffer buf, BoundType type) + { + CommonCodecs.BOOL.encodeToWire(buf, type == BoundType.OPEN); + } + + protected static BoundType decodeType(MutableInt pos, Buffer buf) + { + return CommonCodecs.BOOL.decodeFromWire(pos.getAndIncrement(), buf) ? BoundType.OPEN : BoundType.CLOSED; + } + + /** + * Returns the TokenRange unchanged for local delivery. + */ + public TokenRange transform(TokenRange range) + { + return range; + } + + /** + * Returns the codec name. + */ + public String name() + { + return "big-integer-range"; + } + + /** + * Returns the system codec ID. + */ + public byte systemCodecID() + { + return -1; + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java new file mode 100644 index 000000000..2369369ee --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java @@ -0,0 +1,109 @@ +package org.apache.cassandra.sidecar.codecs; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import com.google.common.collect.ImmutableMap; +import org.apache.commons.lang3.mutable.MutableInt; + +import io.netty.util.CharsetUtil; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; + +/** + * Message codec for encoding and decoding maps of instance IP to token ranges over the Vert.x event bus. + */ +public class RangeMapCodec implements MessageCodec>, Map>> +{ + public static final RangeMapCodec INSTANCE = new RangeMapCodec(); + + /** + * Encodes a range map to the wire buffer. + */ + public void encodeToWire(Buffer buf, Map> map) + { + CommonCodecs.SHORT.encodeToWire(buf, (short) map.size()); // map size + + for (Map.Entry> entry : map.entrySet()) + { + CommonCodecs.STRING.encodeToWire(buf, entry.getKey()); // instanceIP + + final Set ranges = entry.getValue(); + CommonCodecs.SHORT.encodeToWire(buf, (short) ranges.size()); // numRanges + for (TokenRange range : ranges) + { + RangeCodec.INSTANCE.encodeToWire(buf, range); + } + } + } + + /** + * Decodes a range map from the wire buffer. + */ + public Map> decodeMap(MutableInt pos, Buffer buf) + { + int mapSize = buf.getShort(pos.intValue()); + pos.add(2); + final Map> result = new HashMap<>(mapSize); + + for (int i = 0; i < mapSize; i++) + { + String ip = decodeString(pos, buf); + int numRanges = buf.getShort(pos.intValue()); + pos.add(2); + + Set ranges = new HashSet<>(numRanges); + for (int j = 0; j < numRanges; j++) + { + ranges.add(RangeCodec.INSTANCE.decodeFromWire(pos, buf)); + } + result.put(ip, ranges); + } + + return result; + } + + /** + * Decodes a range map from the wire buffer at the specified position. + */ + public Map> decodeFromWire(int pos, Buffer buf) + { + return decodeMap(new MutableInt(pos), buf); + } + + /** + * Returns an immutable copy of the map for local delivery. + */ + public Map> transform(Map> map) + { + return ImmutableMap.copyOf(map); + } + + /** + * Returns the codec name. + */ + public String name() + { + return "range-map"; + } + + /** + * Returns the system codec ID. + */ + public byte systemCodecID() + { + return -1; + } + + protected String decodeString(MutableInt pos, Buffer buf) + { + int len = buf.getInt(pos.intValue()); + pos.add(4); + byte[] bytes = buf.getBytes(pos.intValue(), pos.intValue() + len); + pos.add(len); + return new String(bytes, CharsetUtil.UTF_8); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java index 8988ff02c..79f56afe8 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java @@ -37,7 +37,7 @@ public class CdcConfigurationImpl implements CdcConfiguration public static final String CONFIGURATION_REFRESH_TIME_PROPERTY = "config_refresh_time"; public static final String TABLE_SCHEMA_REFRESH_TIME_PROPERTY = "table_schema_refresh_time"; public static final String SEGMENT_HARD_LINK_CACHE_EXPIRY_PROPERTY = "segment_hardlink_cache_expiry"; - public static final boolean DEFAULT_IS_ENABLED = false; + public static final boolean DEFAULT_IS_ENABLED = true; public static final MillisecondBoundConfiguration DEFAULT_CDC_CONFIG_REFRESH_TIME = MillisecondBoundConfiguration.parse("30s"); public static final SecondBoundConfiguration DEFAULT_TABLE_SCHEMA_REFRESH_TIME = diff --git a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java index 31c0ffaa2..867ccbb46 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java @@ -35,7 +35,7 @@ public class SchemaKeyspaceConfigurationImpl implements SchemaKeyspaceConfiguration { private static final Logger LOGGER = LoggerFactory.getLogger(SchemaKeyspaceConfigurationImpl.class); - public static final boolean DEFAULT_IS_ENABLED = false; + public static final boolean DEFAULT_IS_ENABLED = true; public static final String DEFAULT_KEYSPACE = "sidecar_internal"; public static final String DEFAULT_REPLICATION_STRATEGY = "SimpleStrategy"; public static final int DEFAULT_REPLICATION_FACTOR = 1; @@ -161,7 +161,7 @@ static ConfigurationException leaseTTLConfigurationException(SecondBoundConfigur */ public static class Builder implements DataObjectBuilder { - private boolean isEnabled = DEFAULT_IS_ENABLED; + private boolean isEnabled = true; private String keyspace = DEFAULT_KEYSPACE; private String replicationStrategy = DEFAULT_REPLICATION_STRATEGY; private int replicationFactor = DEFAULT_REPLICATION_FACTOR; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/CassandraClientTokenRingProvider.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/CassandraClientTokenRingProvider.java index d38bdba78..ed55de856 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/coordination/CassandraClientTokenRingProvider.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/CassandraClientTokenRingProvider.java @@ -37,7 +37,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; -import com.google.common.collect.Range; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -68,7 +67,7 @@ public class CassandraClientTokenRingProvider extends TokenRingProvider implemen { private static final Logger LOGGER = LoggerFactory.getLogger(CassandraClientTokenRingProvider.class); @GuardedBy("this") - private volatile Map>>> assignedRangesOfAllInstancesByDcCache = null; + private volatile Map>> assignedRangesOfAllInstancesByDcCache = null; @GuardedBy("this") private volatile Map localHostsCache = null; @GuardedBy("this") @@ -110,7 +109,7 @@ public Set allInstances() } @Override - protected Map>> getAllTokenRanges(Partitioner partitioner, + protected Map> getAllTokenRanges(Partitioner partitioner, String dc) { checkAndReloadReloadCaches(); @@ -122,8 +121,8 @@ protected Map>> getAllTokenRanges(Partitioner par } @Override - public Map>> getPrimaryRanges(SidecarInstance instance, - String dc) + public Map> getPrimaryRanges(SidecarInstance instance, + String dc) { checkAndReloadReloadCaches(); return assignedRangesOfAllInstancesByDcCache.entrySet() @@ -180,13 +179,13 @@ private boolean matchesSidecar(String hostIp, SidecarInstance sidecarInstance) return getIp(sidecarInstance.hostname()).equals(hostIp); } - public Map>>> assignedRangesOfAllInstancesByDc(Metadata metadata) + public Map>> assignedRangesOfAllInstancesByDc(Metadata metadata) { return assignedRangesOfAllInstancesByDc(dnsResolver, metadata); } @VisibleForTesting - public static Map>>> assignedRangesOfAllInstancesByDc(DnsResolver dnsResolver, Metadata metadata) + public static Map>> assignedRangesOfAllInstancesByDc(DnsResolver dnsResolver, Metadata metadata) { Partitioner partitioner = extractPartitioner(metadata); Map> perDcHosts = new HashMap<>(4); @@ -215,7 +214,7 @@ else if (token.getType() == DataType.varint()) throw new UnsupportedOperationException("Unsupported token type: " + token.getType()); } - protected static Map>> calculateTokenRanges(Partitioner partitioner, List sortedPerDcHosts) + protected static Map> calculateTokenRanges(Partitioner partitioner, List sortedPerDcHosts) { // RingTopologyRefresher.calculate... return calculateTokenRanges(sortedPerDcHosts, 1, partitioner) @@ -266,7 +265,7 @@ private static Map>> perKeySpaceTokenRangesOfA return perKeyspaceTokenRanges; } - static Multimap> calculateTokenRanges(List instances, + static Multimap calculateTokenRanges(List instances, int replicationFactor, Partitioner partitioner) { @@ -275,7 +274,7 @@ static Multimap> calculateTokenRanges(List< "Calculation token ranges wouldn't work when RF (" + replicationFactor + ") is greater than number of Cassandra instances " + instances.size()); - Multimap> tokenRanges = ArrayListMultimap.create(); + Multimap tokenRanges = ArrayListMultimap.create(); for (int index = 0; index < instances.size(); ++index) { @@ -285,13 +284,13 @@ static Multimap> calculateTokenRanges(List< BigInteger rangeEnd = new BigInteger(instance.token); if (rangeStart.compareTo(rangeEnd) >= 0) { - tokenRanges.put(instance, Range.openClosed(rangeStart, partitioner.maximumToken().toBigInteger())); + tokenRanges.put(instance, new TokenRange(rangeStart, partitioner.maximumToken().toBigInteger())); if (!rangeEnd.equals(partitioner.minimumToken().toBigInteger())) - tokenRanges.put(instance, Range.openClosed(partitioner.minimumToken().toBigInteger(), rangeEnd)); + tokenRanges.put(instance, new TokenRange(partitioner.minimumToken().toBigInteger(), rangeEnd)); } else { - tokenRanges.put(instance, Range.openClosed(rangeStart, rangeEnd)); + tokenRanges.put(instance, new TokenRange(rangeStart, rangeEnd)); } } return tokenRanges; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java new file mode 100644 index 000000000..4e8a7ad84 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java @@ -0,0 +1,34 @@ +package org.apache.cassandra.sidecar.coordination; + +import java.util.Map; +import java.util.Set; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.Future; +import io.vertx.core.Vertx; +import org.apache.cassandra.sidecar.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; + +/** + * Stub implementation of the RangeManager that provides contention-free failover for token ranges without coordinating with other Sidecar instances, at the cost of consistency. + */ +@Singleton +public class ContentionFreeRangeManager extends RangeManager +{ + @Inject + public ContentionFreeRangeManager(Vertx vertx, TokenRingProvider tokenRingProvider) + { + super(vertx, tokenRingProvider); + } + + Future proposeOwnership(SidecarInstance current, Map> ranges) + { + return Future.succeededFuture(true); + } + + Future releaseOwnership(SidecarInstance primaryOwner, Map> ranges) + { + return Future.succeededFuture(true); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java new file mode 100644 index 000000000..64101de20 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java @@ -0,0 +1,398 @@ +package org.apache.cassandra.sidecar.coordination; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.stream.Collectors; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import io.vertx.core.Future; +import io.vertx.core.Handler; +import io.vertx.core.Vertx; +import io.vertx.core.eventbus.Message; +import org.apache.cassandra.sidecar.client.SidecarInstance; +import org.apache.cassandra.sidecar.codecs.RangeChangeEventCodec; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.spark.utils.Pair; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CASSANDRA_CQL_READY; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_DOWN; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_UP; +import static org.apache.cassandra.sidecar.tasks.ClusterTopologyMonitor.ClusterTopologyEventType.ON_DC_TOPOLOGY_CHANGE; + +/** + * This class manages the token ranges owned by this Sidecar instance and listens on the DownDetector for other Sidecar instances going up/down. + * The underlying implementation can implement a consensus algorithm to provide strong guarantees around gaining/releasing token range ownership. + */ +public abstract class RangeManager implements Handler> +{ + private static final Logger LOGGER = LoggerFactory.getLogger(RangeManager.class); + + /** + * Events published when token range ownership changes. + */ + public enum LeadershipEvents + { + ON_TOKEN_RANGE_GAINED, + ON_TOKEN_RANGE_LOST; + + /** + * Returns the event bus address for this leadership event type. + */ + public String address() + { + return LeadershipEvents.class.getName() + "." + name(); + } + } + + /** + * Events published when the token ring topology changes. + */ + public enum RangeManagerEvents + { + ON_TOKEN_RANGE_CHANGED; + + /** + * Returns the event bus address for this range manager event type. + */ + public String address() + { + return RangeManagerEvents.class.getName() + "." + name(); + } + } + + /** + * Event data containing information about token range ownership changes. + */ + public static class RangeChangeEvent + { + public final ImmutableMap> change; // ranges gained/lost by this change + public final ImmutableMap> newView; // new view of the ring after this change + + /** + * Creates a new range change event with the specified changes and new view. + */ + public RangeChangeEvent(Map> change, Map> newView) + { + this.change = ImmutableMap.copyOf(change); + this.newView = ImmutableMap.copyOf(newView); + } + + public boolean equals(Object o) + { + if (this == o) + { + return true; + } + if (o == null || getClass() != o.getClass()) + { + return false; + } + + RangeChangeEvent event = (RangeChangeEvent) o; + return Objects.equals(change, event.change) && Objects.equals(newView, event.newView); + } + + public int hashCode() + { + return Objects.hash(change, newView); + } + } + + protected final Vertx vertx; + protected final TokenRingProvider tokenRingProvider; + @Nullable + protected volatile ImmutableMap> ownedRanges = null; + + public RangeManager(Vertx vertx, TokenRingProvider tokenRingProvider) + { + this.vertx = vertx; + this.tokenRingProvider = tokenRingProvider; + initializeRanges(); + try + { + vertx.eventBus().registerDefaultCodec(RangeChangeEvent.class, RangeChangeEventCodec.INSTANCE); + } + catch (IllegalStateException ise) + { + if (ise.getMessage().contains("Already a default codec registered")) + { + // TODO: do not register codec in the constructor when OSS; All codecs should be managed in a central repository + // ignore + } + else + { + throw ise; + } + } + vertx.eventBus().localConsumer(ON_CASSANDRA_CQL_READY.address(), this); + vertx.eventBus().localConsumer(ON_SIDECAR_PEER_UP.address(), this); + vertx.eventBus().localConsumer(ON_SIDECAR_PEER_DOWN.address(), this); + vertx.eventBus().localConsumer(ON_DC_TOPOLOGY_CHANGE.address(), this); + } + + /** + * Returns the token ranges currently owned by this Sidecar instance. + */ + @Nullable + public Map> ownedTokenRanges() + { + return this.ownedRanges; + } + + // Sidecar coordination + + /** + * @param primaryOwner SidecarInstance that is the primary owner for the token ranges. + * @param ranges token ranges this SidecarInstance wishes to gain ownership. + * @return a future that completes when propose request completes, returning the ranges that were successfully gained as part of the request. + */ + /** + * Proposes to gain ownership of the specified token ranges. + */ + abstract Future proposeOwnership(SidecarInstance primaryOwner, Map> ranges); + + /** + * @param primaryOwner SidecarInstance that is the primary owner for the token ranges. + * @param ranges token ranges this SidecarInstance wishes to release. + * @return a future that completes when release request completes, returning the ranges that were released as part of the request. + */ + /** + * Proposes to release ownership of the specified token ranges. + */ + abstract Future releaseOwnership(SidecarInstance primaryOwner, Map> ranges); + + // DownDetector notification handler + + /** + * Handles event bus messages for peer up/down and topology changes. + */ + @Override + public void handle(Message msg) + { + if (ON_SIDECAR_PEER_UP.address().equals(msg.address())) + { + onSidecarUp((SidecarInstance) msg.body()); + } + else if (ON_SIDECAR_PEER_DOWN.address().equals(msg.address())) + { + onSidecarDown((SidecarInstance) msg.body()); + } + else if (ON_CASSANDRA_CQL_READY.address().equals(msg.address()) || ON_DC_TOPOLOGY_CHANGE.address().equals(msg.address())) + { + initializeRanges(); + } + } + + /** + * Initializes owned token ranges from the token ring provider. + */ + protected synchronized void initializeRanges() + { + LOGGER.info("Initializing ranges"); + final String dc = getLocalDcSafe(); + if (dc == null) + { + LOGGER.error("DC null, cannot initialize owned ranges"); + return; + } + ImmutableMap> newRanges = ImmutableMap.copyOf(toSet(tokenRingProvider.getPrimaryTokenRanges(dc))); + if (!newRanges.equals(this.ownedRanges)) + { + this.ownedRanges = newRanges; + vertx.eventBus().publish(RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), null); + } + } + + /** + * Handles a Sidecar instance coming online by releasing its primary ranges. + */ + protected synchronized void onSidecarUp(SidecarInstance instance) + { + // on Sidecar detected to be UP, initiate proposal to release ownership of its primary ranges + final String dc = getLocalDcSafe(); + if (dc == null) + { + LOGGER.error("DC null, cannot respond to onSidecarUp event"); + return; + } + Map> primaryRanges = toSet(tokenRingProvider.getPrimaryRanges(instance, dc)); + if (primaryRanges.isEmpty()) + { + LOGGER.warn("Empty primary token ranges for Sidecar host={} port={}", instance.hostname(), instance.port()); + return; + } + + releaseOwnership(instance, primaryRanges) + .onSuccess(result -> { + if (result) + { + onRangesLost(primaryRanges); + } + else + { + LOGGER.warn("Failed to release ownership of instance primary range host={} port={} ranges='{}'", instance.hostname(), instance.port(), primaryRanges); + } + }) + .onFailure(throwable -> LOGGER.warn("Error attempting to release range ownership", throwable)); + } + + /** + * Handles a Sidecar instance going offline by gaining its primary ranges. + */ + protected synchronized void onSidecarDown(SidecarInstance instance) + { + // on Sidecar detected to be DOWN, initiate proposal to gain ownership of its primary ranges + final String dc = getLocalDcSafe(); + if (dc == null) + { + LOGGER.error("DC null, cannot respond to onSidecarDown event"); + return; + } + Map> primaryRanges = toSet(tokenRingProvider.getPrimaryRanges(instance, dc)); + if (primaryRanges.isEmpty()) + { + LOGGER.warn("Empty primary token ranges for Sidecar host={} port={}", instance.hostname(), instance.port()); + return; + } + + proposeOwnership(instance, primaryRanges) + .onSuccess(result -> { + if (result) + { + onRangesGained(primaryRanges); + } + else + { + LOGGER.warn("Failed to gain ownership of instance primary range host={} port={} ranges='{}'", instance.hostname(), instance.port(), primaryRanges); + } + }) + .onFailure(throwable -> LOGGER.warn("Error attempting to gain range ownership", throwable)); + } + + // Range change events + + /** + * Handles gaining ownership of token ranges and publishes the change event. + */ + public synchronized void onRangesGained(Map> gainedRanges) + { + ImmutableMap> ownedRanges = this.ownedRanges; + if (ownedRanges == null) + { + LOGGER.warn("Cannot handle gained range notification, cql session not initialized yet ranges='{}'", gainedRanges); + return; + } + ImmutableMap> newView = ImmutableMap.copyOf(unionRangeMap(ownedRanges, gainedRanges)); + if (!newView.equals(ownedRanges)) + { + LOGGER.info("Gained ownership of ranges ranges='{}'", gainedRanges); + this.ownedRanges = newView; + this.vertx.eventBus().publish(LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), new RangeChangeEvent(gainedRanges, newView)); + } + } + + /** + * Handles losing ownership of token ranges and publishes the change event. + */ + public synchronized void onRangesLost(Map> lostRanges) + { + ImmutableMap> ownedRanges = this.ownedRanges; + if (ownedRanges == null) + { + LOGGER.warn("Cannot handle lost range notification, cql session not initialized yet ranges='{}'", lostRanges); + return; + } + ImmutableMap> newView = ImmutableMap.copyOf(differenceRangeMap(ownedRanges, lostRanges)); + if (!newView.equals(ownedRanges)) + { + LOGGER.info("Lost ownership of ranges ranges='{}'", lostRanges); + this.ownedRanges = newView; + this.vertx.eventBus().publish(LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), new RangeChangeEvent(lostRanges, newView)); + } + } + + // utils + + protected static Map> unionRangeMap(@NotNull Map> currRanges, Map> newRanges) + { + if (newRanges == null || newRanges.isEmpty()) + { + return currRanges; + } + else if (currRanges.isEmpty()) + { + return newRanges; + } + + Map> merged = new HashMap<>(currRanges); + for (Map.Entry> entry : newRanges.entrySet()) + { + merged.put(entry.getKey(), unionRanges(merged.get(entry.getKey()), entry.getValue())); + } + return merged; + } + + protected static Set unionRanges(@Nullable Set current, @NotNull Set gained) + { + if (current == null || current.isEmpty()) + { + return gained; + } + else if (gained.isEmpty()) + { + return current; + } + + return Sets.union(current, gained).immutableCopy(); + } + + protected static Set differenceRangeMap(@NotNull Set current, @Nullable Set lost) + { + if (lost == null || lost.isEmpty() || current.isEmpty()) + { + return current; + } + return Sets.difference(current, lost).immutableCopy(); + } + + protected static Map> differenceRangeMap(@NotNull Map> curr, Map> lost) + { + return curr.entrySet().stream() + .map(e -> Pair.of(e.getKey(), differenceRangeMap(e.getValue(), lost.get(e.getKey())))) + .filter(f -> !f.right.isEmpty()) + .collect(Collectors.toMap(e -> e.left, e -> e.right)); + } + + protected static Map> toSet(Map> ranges) + { + return ranges.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, e -> new HashSet<>(e.getValue()))); + } + + /** + * Returns the local datacenter name safely, handling exceptions. + */ + public String getLocalDcSafe() + { + try + { + return tokenRingProvider.localDc(); + } + catch (RuntimeException runtimeException) + { + LOGGER.warn("localDC is unavailable", runtimeException); + // localDc throws when no local instances are found + return null; + } + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/TokenRingProvider.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/TokenRingProvider.java index c3a67630a..59f1ea864 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/coordination/TokenRingProvider.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/TokenRingProvider.java @@ -19,7 +19,6 @@ package org.apache.cassandra.sidecar.coordination; -import java.math.BigInteger; import java.net.InetAddress; import java.net.UnknownHostException; import java.util.Collections; @@ -28,7 +27,6 @@ import java.util.Set; import java.util.stream.Collectors; -import com.google.common.collect.Range; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,6 +37,7 @@ import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.common.server.cluster.locator.Partitioner; import org.apache.cassandra.sidecar.common.server.cluster.locator.Partitioners; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; import org.apache.cassandra.sidecar.common.server.dns.DnsResolver; import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; import org.jetbrains.annotations.Nullable; @@ -68,7 +67,7 @@ public TokenRingProvider(InstancesMetadata instancesMetadata, InstanceMetadataFe * @param dc data center * @return map of token ranges per Cassandra instance IP */ - protected abstract Map>> getAllTokenRanges(Partitioner partitioner, @Nullable String dc); + protected abstract Map> getAllTokenRanges(Partitioner partitioner, @Nullable String dc); /** * Gets primary token ranges of the given sidecar instance. @@ -77,7 +76,7 @@ public TokenRingProvider(InstancesMetadata instancesMetadata, InstanceMetadataFe * @param dc data center * @return primary token ranges for the SidecarInstance */ - public abstract Map>> getPrimaryRanges(SidecarInstance instance, String dc); + public abstract Map> getPrimaryRanges(SidecarInstance instance, String dc); /** * Gets primary token ranges for local cassandra instances. @@ -85,7 +84,7 @@ public TokenRingProvider(InstancesMetadata instancesMetadata, InstanceMetadataFe * @param dc optionally filter by DC. * @return token ranges per local Cassandra instance */ - public Map>> getPrimaryTokenRanges(@Nullable String dc) + public Map> getPrimaryTokenRanges(@Nullable String dc) { Set instanceIpsManagedBySidecar = instanceIpsManagedBySidecar(); if (instanceIpsManagedBySidecar.isEmpty()) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java index 91c0212fd..62b70956b 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java @@ -20,9 +20,10 @@ package org.apache.cassandra.sidecar.db; import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Optional; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -39,11 +40,19 @@ import com.google.inject.Provider; import com.google.inject.ProvisionException; import com.google.inject.Singleton; +import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraBridgeFactory; + import org.apache.cassandra.bridge.TokenRange; +import org.apache.cassandra.cdc.CdcKryoRegister; +import org.apache.cassandra.cdc.state.CdcState; +import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; import org.apache.cassandra.sidecar.db.schema.CdcStatesSchema; -import org.apache.cassandra.sidecar.db.schema.TableHistorySchema; +import org.apache.cassandra.sidecar.db.schema.SidecarSchema; import org.apache.cassandra.sidecar.utils.ByteBufUtils; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; import org.apache.cassandra.sidecar.utils.TokenSplitUtil; import org.apache.cassandra.spark.data.partitioner.Partitioner; import org.apache.cassandra.spark.utils.TableIdentifier; @@ -61,19 +70,19 @@ public class CdcDatabaseAccessor extends DatabaseAccessor { private static final Logger LOGGER = LoggerFactory.getLogger(CdcDatabaseAccessor.class); - private final TableHistorySchema tableHistorySchema; private final Provider tokenSplitUtilProvider; private volatile TokenSplitUtil tokenSplitUtil = null; + private final InstanceMetadataFetcher instanceMetadataFetcher; @Inject - public CdcDatabaseAccessor(CdcStatesSchema cdcStatesSchema, - TableHistorySchema tableHistorySchema, + public CdcDatabaseAccessor(SidecarSchema sidecarSchema, CQLSessionProvider sessionProvider, - Provider tokenSplitUtilProvider) + Provider tokenSplitUtilProvider, + InstanceMetadataFetcher instanceMetadataFetcher) { - super(cdcStatesSchema, sessionProvider); - this.tableHistorySchema = tableHistorySchema; + super(sidecarSchema.tableSchema(CdcStatesSchema.class), sessionProvider); this.tokenSplitUtilProvider = tokenSplitUtilProvider; + this.instanceMetadataFetcher = instanceMetadataFetcher; } protected TokenSplitUtil tokenSplitUtil() @@ -115,7 +124,7 @@ public List storeStateAsync(String jobId, // write state into all overlapping splits int[] splits = tokenSplitUtil().findOverlappingSplitIds(partitioner(), range); LOGGER.debug("Inserting CDC state into splits lower={} upper={} splits='{}'", - range.lowerEndpoint(), range.upperEndpoint(), + range.lowerEndpoint(), range.upperEndpoint(), Arrays.stream(splits).mapToObj(Integer::toString).collect(Collectors.joining(","))); return Arrays.stream(splits) .mapToObj(split -> session().executeAsync(tableSchema.insertState().bind( @@ -129,6 +138,32 @@ public List storeStateAsync(String jobId, .collect(Collectors.toList()); } + /** + * Load cdc state for a given jobId and token range and merge into canonical view + * + * @param stats SidecarCdcStats to publish metrics + * @param jobId Cdc job id + * @param range token range + * @return merged SidecarCdcState object that merges previous state objects that overlap with token range to given canonical view of Cdc state. + */ + public Optional loadSidecarCdcState(SidecarCdcStats stats, String jobId, TokenRange range) + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + CassandraBridge cassandraBridge = new CassandraBridgeFactory().get(nodeSettings.releaseVersion()); + List sizes = new ArrayList<>(); + // deserialize and merge the CDC state objects into canonical view + Optional result = loadStateForRange(jobId, range) + .peek(bytes -> sizes.add(bytes.length)) + .map((byte[] compressed) -> CdcState.deserialize(CdcKryoRegister.kryo(), cassandraBridge.compressionUtil(), compressed)) + .reduce((s1, s2) -> s1.merge(range, s2)); + int count = sizes.size(); + int len = sizes.stream().mapToInt(i -> i).sum(); + LOGGER.debug("Read CDC state from Cassandra jobId={} start={} end={} stateCount={} stateSize={}", + jobId, range.lowerEndpoint(), range.upperEndpoint(), count, len); + stats.captureCdcConsumerReadFromState(count, len); + return result; + } + /** * @param jobId Cdc job id * @param range token range @@ -141,7 +176,7 @@ public Stream loadStateForRange(String jobId, TokenRange range) // read state from multiple shards that could overlap with range int[] splits = tokenSplitUtil().findOverlappingSplitIds(partitioner(), range); LOGGER.info("Reading CDC state from splits lower={} upper={} splits='{}'", - range.lowerEndpoint(), range.upperEndpoint(), + range.lowerEndpoint(), range.upperEndpoint(), Arrays.stream(splits).mapToObj(Integer::toString).collect(Collectors.joining(","))); Stream futures = Arrays.stream(splits) .mapToObj(split -> selectCdcRange(jobId, split)); @@ -181,31 +216,4 @@ ResultSetFuture selectCdcRange(String jobId, int split) { return session().executeAsync(tableSchema.select().bind(jobId, (short) split)); } - - public ResultSetFuture insertTableSchemaHistory(String keyspace, String tableName, String schema) - { - UUID schemaUuid = UUID.nameUUIDFromBytes(schema.getBytes(StandardCharsets.UTF_8)); - return session().executeAsync(tableHistorySchema - .insertTableSchema() - .bind(keyspace, tableName, schemaUuid, schema)); - } - - public String tableSchemaFromVersion(String keyspace, String tableName, String version) - { - UUID schemaUuid = UUID.fromString(version); - try - { - Row row = session() - .executeAsync(tableHistorySchema - .selectVersionTableSchema() - .bind(keyspace, tableName, schemaUuid)) - .get() - .one(); - return row == null ? null : row.getString("table_schema"); - } - catch (Exception e) - { - throw new RuntimeException(e); - } - } } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java new file mode 100644 index 000000000..eae5b4f8b --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.cassandra.sidecar.db; + +import java.nio.charset.StandardCharsets; +import java.util.UUID; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.ResultSetFuture; +import com.datastax.driver.core.Row; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; +import org.apache.cassandra.sidecar.db.schema.SidecarSchema; +import org.apache.cassandra.sidecar.db.schema.TableHistorySchema; + +/** + * Database accessor for Table History operations. + */ +@SuppressWarnings("resource") +@Singleton +public class TableHistoryDatabaseAccessor extends DatabaseAccessor +{ + private static final Logger LOGGER = LoggerFactory.getLogger(TableHistoryDatabaseAccessor.class); + + @Inject + public TableHistoryDatabaseAccessor(SidecarSchema sidecarSchema, + CQLSessionProvider sessionProvider) + { + super(sidecarSchema.tableSchema(TableHistorySchema.class), sessionProvider); + } + + public ResultSetFuture insertTableSchemaHistory(String keyspace, String tableName, String schema) + { + UUID schemaUuid = UUID.nameUUIDFromBytes(schema.getBytes(StandardCharsets.UTF_8)); + return session().executeAsync(tableSchema + .insertTableSchema() + .bind(keyspace, tableName, schemaUuid, schema)); + } + + public String tableSchemaFromVersion(String keyspace, String tableName, String version) + { + UUID schemaUuid = UUID.fromString(version); + try + { + Row row = session() + .executeAsync(tableSchema + .selectVersionTableSchema() + .bind(keyspace, tableName, schemaUuid)) + .get() + .one(); + return row == null ? null : row.getString("table_schema"); + } + catch (Exception e) + { + throw new RuntimeException(e); + } + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java new file mode 100644 index 000000000..614f6049b --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java @@ -0,0 +1,43 @@ +package org.apache.cassandra.sidecar.db; + +import com.datastax.driver.core.Row; +import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.querybuilder.Select; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; +import org.apache.cassandra.sidecar.db.schema.TableSchema; + +import static com.datastax.driver.core.querybuilder.QueryBuilder.eq; + +/** + * Database accessor for querying Cassandra virtual tables in the system_views keyspace. + */ +@Singleton +public class VirtualTablesDatabaseAccessor extends DatabaseAccessor +{ + public static final String SYSTEM_VIEWS_KS = "system_views"; + public static final String SYSTEM_VIEWS_SETTINGS_TBL = "settings"; + public static final String CDC_ON_REPAIR_ENABLED_FLAG = "cdc_on_repair_enabled"; + + /** + * Creates a new virtual tables database accessor. + */ + @Inject + public VirtualTablesDatabaseAccessor(TableSchema tableSchema, CQLSessionProvider sessionProvider) + { + super(tableSchema, sessionProvider); + } + + /** + * Checks if CDC on repair is enabled in the system settings. + */ + public boolean isCdcOnRepairEnabled() + { + Select.Where query = QueryBuilder.select("value") + .from(SYSTEM_VIEWS_KS, SYSTEM_VIEWS_SETTINGS_TBL) + .where(eq("name", CDC_ON_REPAIR_ENABLED_FLAG)); + Row row = session().execute(query).one(); + return row != null && !row.isNull(0) && "true".equalsIgnoreCase(row.getString(0)); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/schema/TableHistorySchema.java b/server/src/main/java/org/apache/cassandra/sidecar/db/schema/TableHistorySchema.java index 96d55b90d..da6499e12 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/db/schema/TableHistorySchema.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/schema/TableHistorySchema.java @@ -92,7 +92,7 @@ protected String createSchemaStatement() " version uuid," + " created_at timeuuid," + " table_schema text," + - " PRIMARY KEY ((ks, tb), version)" + + " PRIMARY KEY ((keyspace_name, table_name), version)" + ")", keyspaceConfig.keyspace(), TABLE_SCHEMA_HISTORY); } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java b/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java index 7060a3b3c..aa19c394b 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java @@ -87,8 +87,8 @@ public void handle(RoutingContext context) try { requestParams = extractParamsOrThrow(context); - logger.debug("{} received request={}, remoteAddress={}, instance={}", - this.getClass().getSimpleName(), requestParams, remoteAddress, host); +// logger.debug("{} received request={}, remoteAddress={}, instance={}", +// this.getClass().getSimpleName(), requestParams, remoteAddress, host); handleInternal(context, request, host, remoteAddress, requestParams); } catch (Exception exception) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java index 3e2b10d67..ab68f050c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java @@ -18,39 +18,68 @@ package org.apache.cassandra.sidecar.modules; +import java.util.Set; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.Host; import com.google.inject.AbstractModule; import com.google.inject.Provides; import com.google.inject.Singleton; import com.google.inject.multibindings.ProvidesIntoMap; +import io.vertx.core.Vertx; import jakarta.ws.rs.DELETE; import jakarta.ws.rs.GET; import jakarta.ws.rs.PUT; import jakarta.ws.rs.Path; import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.cdc.avro.CqlToAvroSchemaConverter; +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.cdc.sidecar.SidecarCdcClient; +import org.apache.cassandra.cdc.stats.CdcStats; +import org.apache.cassandra.cdc.stats.ICdcStats; +import org.apache.cassandra.sidecar.cdc.CachingSchemaStore; +import org.apache.cassandra.sidecar.cdc.CdcAvroSerializer; import org.apache.cassandra.sidecar.cdc.CdcConfig; import org.apache.cassandra.sidecar.cdc.CdcConfigImpl; +import org.apache.cassandra.sidecar.cdc.CdcDynamicSidecarInstancesProvider; import org.apache.cassandra.sidecar.cdc.CdcLogCache; +import org.apache.cassandra.sidecar.cdc.CdcPublisher; +import org.apache.cassandra.sidecar.cdc.CdcSchemaSupplier; import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.cdc.SidecarCqlToAvroSchemaConverter; import org.apache.cassandra.sidecar.client.SidecarInstancesProvider; import org.apache.cassandra.sidecar.cluster.InstancesMetadata; import org.apache.cassandra.sidecar.common.ApiEndpointsV1; import org.apache.cassandra.sidecar.common.request.data.AllServicesConfigPayload; import org.apache.cassandra.sidecar.common.response.ListCdcSegmentsResponse; +import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.concurrent.ExecutorPools; import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.coordination.CassandraClientTokenRingProvider; import org.apache.cassandra.sidecar.coordination.DynamicSidecarInstancesProvider; import org.apache.cassandra.sidecar.coordination.InnerDcTokenAdjacentPeerProvider; import org.apache.cassandra.sidecar.coordination.SidecarHttpHealthProvider; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; import org.apache.cassandra.sidecar.coordination.SidecarPeerProvider; +import org.apache.cassandra.sidecar.coordination.TokenRingProvider; import org.apache.cassandra.sidecar.db.CdcConfigAccessor; import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; import org.apache.cassandra.sidecar.db.KafkaConfigAccessor; import org.apache.cassandra.sidecar.db.TokenSplitConfigAccessor; +import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; +import org.apache.cassandra.sidecar.db.schema.CdcStatesSchema; import org.apache.cassandra.sidecar.db.schema.ConfigsSchema; import org.apache.cassandra.sidecar.db.schema.SystemViewsSchema; +import org.apache.cassandra.sidecar.db.schema.TableHistorySchema; import org.apache.cassandra.sidecar.db.schema.TableSchema; import org.apache.cassandra.sidecar.handlers.cdc.AllServiceConfigHandler; import org.apache.cassandra.sidecar.handlers.cdc.DeleteServiceConfigHandler; @@ -64,12 +93,15 @@ import org.apache.cassandra.sidecar.routes.RouteBuilder; import org.apache.cassandra.sidecar.routes.VertxRoute; import org.apache.cassandra.sidecar.tasks.CassandraClusterSchemaMonitor; +import org.apache.cassandra.sidecar.tasks.CdcConfigRefresherNotifierTask; import org.apache.cassandra.sidecar.tasks.CdcRawDirectorySpaceCleaner; import org.apache.cassandra.sidecar.tasks.PeriodicTask; -import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; import org.apache.cassandra.sidecar.utils.SidecarClientProvider; import org.apache.cassandra.sidecar.utils.TokenSplitUtil; +import org.apache.cassandra.spark.data.partitioner.CassandraInstance; +import org.apache.cassandra.spark.data.partitioner.Partitioner; +import org.apache.kafka.common.serialization.Serializer; import org.eclipse.microprofile.openapi.annotations.Operation; import org.eclipse.microprofile.openapi.annotations.enums.SchemaType; import org.eclipse.microprofile.openapi.annotations.media.Content; @@ -82,6 +114,9 @@ @Path("/") public class CdcModule extends AbstractModule { + + private static final Logger LOGGER = LoggerFactory.getLogger(CdcModule.class); + @ProvidesIntoMap @KeyClassMapKey(PeriodicTaskMapKeys.SidecarPeerHealthMonitorTaskKey.class) PeriodicTask sidecarPeerHealthMonitorTask(SidecarPeerHealthMonitorTask task) @@ -97,24 +132,39 @@ PeriodicTask cdcRawDirectorySpaceCleanercPeriodicTask(CdcRawDirectorySpaceCleane return cleanerTask; } + @Provides + @Singleton + CassandraClusterSchemaMonitor cassandraClusterSchemaMonitorInstance(InstanceMetadataFetcher instanceMetadataFetcher, + CdcDatabaseAccessor databaseAccessor, + SidecarConfiguration configuration, + CassandraBridgeFactory cassandraBridgeFactory) + { + return new CassandraClusterSchemaMonitor(instanceMetadataFetcher, databaseAccessor, configuration, cassandraBridgeFactory); + } + @ProvidesIntoMap @KeyClassMapKey(PeriodicTaskMapKeys.CassandraClusterSchemaTaskKey.class) - PeriodicTask cassandraClusterSchemaMonitor(InstanceMetadataFetcher instanceMetadataFetcher, - CdcDatabaseAccessor databaseAccessor, - SidecarConfiguration configuration, - CassandraBridgeFactory cassandraBridgeFactory) + PeriodicTask cassandraClusterSchemaMonitor(CassandraClusterSchemaMonitor monitor) { - return new CassandraClusterSchemaMonitor(instanceMetadataFetcher, databaseAccessor, configuration, cassandraBridgeFactory); + // Wire the singleton instance into the periodic tasks map + return monitor; } @Provides @Singleton - CdcConfig cdcConfig(SidecarConfiguration sidecarConfiguration, - CdcConfigAccessor cdcConfigAccessor, - KafkaConfigAccessor kafkaConfigAccessor, - PeriodicTaskExecutor periodicTaskExecutor) + CqlToAvroSchemaConverter cqlToAvroSchemaConverter(InstanceMetadataFetcher instanceMetadataFetcher, + CassandraBridgeFactory cassandraBridgeFactory) { - return new CdcConfigImpl(sidecarConfiguration, cdcConfigAccessor, kafkaConfigAccessor, periodicTaskExecutor); + + return new SidecarCqlToAvroSchemaConverter(instanceMetadataFetcher, cassandraBridgeFactory); + } + + @Provides + @Singleton + CdcConfig cdcConfig(Vertx vertx, + CdcConfigAccessor cdcConfigAccessor) + { + return new CdcConfigImpl(vertx, cdcConfigAccessor); } @Provides @@ -254,6 +304,13 @@ public SidecarPeerProvider sidecarPeerProvider(InnerDcTokenAdjacentPeerProvider return innerDcTokenAdjacentPeerProvider; } + @Provides + @Singleton + public CdcSidecarInstancesProvider cdcSidecarInstancesProvider(InstancesMetadata instancesMetadata, ServiceConfiguration serviceConfiguration) + { + return new CdcDynamicSidecarInstancesProvider(instancesMetadata, serviceConfiguration); + } + @Provides @Singleton public SidecarInstancesProvider sidecarInstancesProvider(InstancesMetadata instancesMetadata, ServiceConfiguration serviceConfiguration) @@ -269,4 +326,156 @@ public SidecarCdcStats sidecarCdcStats() { }; } + + @Provides + @Singleton + public Serializer getSerializer(CachingSchemaStore schemaStore, + InstanceMetadataFetcher instanceMetadataFetcher, + CassandraBridgeFactory cassandraBridgeFactory) + { + return new CdcAvroSerializer(schemaStore, instanceMetadataFetcher, cassandraBridgeFactory); + } + + @Provides + @Singleton + public SchemaSupplier schemaSupplier(InstanceMetadataFetcher instanceMetadataFetcher, + CassandraBridgeFactory cassandraBridgeFactory, + CdcDatabaseAccessor cdcDatabaseAccessor) + { + return new CdcSchemaSupplier(instanceMetadataFetcher, cassandraBridgeFactory, cdcDatabaseAccessor); + } + + @Provides + @Singleton + public ClusterConfigProvider clusterConfigProvider(InstanceMetadataFetcher instanceMetadataFetcher, CassandraBridgeFactory cassandraBridgeFactory) + { + return new ClusterConfigProvider() + { + public String dc() + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + return nodeSettings.datacenter(); + } + + public Set getCluster() + { + Set hosts = instanceMetadataFetcher.callOnFirstAvailableInstance(instance -> + instance.delegate().metadata().getAllHosts()); + return hosts.stream() + .filter(host -> host.getListenAddress() != null) + .flatMap(host -> host.getTokens().stream() + .map(token -> new CassandraInstance( + token.toString(), + host.getEndPoint().resolve().getHostName(), + host.getDatacenter() + )) + ).collect(Collectors.toSet()); + } + + public Partitioner partitioner() + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + String[] parts = nodeSettings.partitioner().split("\\."); + return Partitioner.valueOf(parts[parts.length - 1]); + } + }; + } + + @Provides + @Singleton + public ICdcStats cdcStats() + { + return new CdcStats() + { + }; + } + + @Provides + @Singleton + public TokenRingProvider tokenRingProvider(InstancesMetadata instancesMetadata, InstanceMetadataFetcher instanceMetadataFetcher, ServiceConfiguration configuration) + { + return new CassandraClientTokenRingProvider(instancesMetadata, instanceMetadataFetcher, configuration.dnsResolver()); + } + + @Provides + @Singleton + public SidecarCdcClient.ClientConfig clientConfig(SidecarConfiguration sidecarConfiguration) + { + SidecarClientConfiguration sidecarClientConfiguration = sidecarConfiguration.sidecarClientConfiguration(); + return SidecarCdcClient.ClientConfig.create(sidecarConfiguration.serviceConfiguration().port(), + sidecarClientConfiguration.maxRetries(), + sidecarClientConfiguration.retryDelay().toIntMillis()); + } + + @Provides + @Singleton + public TableSchema virtualTablesDatabaseAccessor(ServiceConfiguration configuration) + { + return new CdcStatesSchema(configuration); + } + + @ProvidesIntoMap + @KeyClassMapKey(PeriodicTaskMapKeys.CdcPublisherTaskKey.class) + PeriodicTask cdcPublisherTask(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + ExecutorPools executorPools, + ClusterConfigProvider clusterConfigProvider, + SchemaSupplier schemaSupplier, + CdcSidecarInstancesProvider sidecarInstancesProvider, + SidecarCdcClient.ClientConfig clientConfig, + InstanceMetadataFetcher instanceMetadataFetcher, + CdcConfig conf, + CdcDatabaseAccessor databaseAccessor, + ICdcStats cdcStats, + TokenRingProvider tokenRingProvider, + VirtualTablesDatabaseAccessor virtualTables, + SidecarCdcStats sidecarCdcStats, + Serializer avroSerializer) + { + return new CdcPublisher(vertx, + sidecarConfiguration, + executorPools, + clusterConfigProvider, + schemaSupplier, + sidecarInstancesProvider, + clientConfig, + instanceMetadataFetcher, + conf, + databaseAccessor, + cdcStats, + tokenRingProvider, + virtualTables, + sidecarCdcStats, + avroSerializer); + } + + @Singleton + @ProvidesIntoMap + @KeyClassMapKey(PeriodicTaskMapKeys.CdcConfigRefresherNotifierKey.class) + PeriodicTask cdcConfigRefresherNotifier(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + KafkaConfigAccessor kafkaConfigAccessor, + CdcConfigAccessor cdcConfigAccessor) + { + return new CdcConfigRefresherNotifierTask(vertx, + sidecarConfiguration, + kafkaConfigAccessor, + cdcConfigAccessor); + } + + @Singleton + @ProvidesIntoMap + @KeyClassMapKey(TableSchemaMapKeys.TableHistorySchemaKey.class) + TableSchema tableHistorySchema(SidecarConfiguration configuration) + { + return new TableHistorySchema(configuration.serviceConfiguration()); + } + + @Singleton + @ProvidesIntoMap + @KeyClassMapKey(TableSchemaMapKeys.CdcStatesSchemaKey.class) + TableSchema cdcStatesSchema(SidecarConfiguration configuration) + { + return new CdcStatesSchema(configuration.serviceConfiguration()); + } } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/PeriodicTaskMapKeys.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/PeriodicTaskMapKeys.java index 5e3626684..293a57a22 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/PeriodicTaskMapKeys.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/PeriodicTaskMapKeys.java @@ -34,5 +34,7 @@ interface SchemaReportingTaskKey extends ClassKey {} interface SidecarPeerHealthMonitorTaskKey extends ClassKey {} interface SidecarSchemaInitializerTaskKey extends ClassKey {} interface CdcRawDirectorySpaceCleanerTaskKey extends ClassKey {} + interface CdcPublisherTaskKey extends ClassKey {} + interface CdcConfigRefresherNotifierKey extends ClassKey {} interface CassandraClusterSchemaTaskKey extends ClassKey {} } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/TableSchemaMapKeys.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/TableSchemaMapKeys.java index a3aa05e63..13a1ec831 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/TableSchemaMapKeys.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/multibindings/TableSchemaMapKeys.java @@ -32,4 +32,6 @@ interface SidecarRolePermissionsSchemaKey extends ClassKey {} interface SystemAuthSchemaKey extends ClassKey {} interface SystemViewsSchemaKey extends ClassKey {} interface SystemViewsClientsSchemaKey extends ClassKey {} + interface TableHistorySchemaKey extends ClassKey {} + interface CdcStatesSchemaKey extends ClassKey {} } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java b/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java index 7763ef752..9c876ac79 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java @@ -113,6 +113,10 @@ public enum SidecarServerEvents * has lost the cluster-wide lease. */ ON_SIDECAR_GLOBAL_LEASE_LOST, + + ON_CDC_CONFIGURATION_CHANGED, + ON_CDC_CACHE_WARMED_UP, + ON_CDC_CONFIG_MAPPINGS_CHANGED, ; public String address() diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CassandraClusterSchemaMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CassandraClusterSchemaMonitor.java index cfabf3d9e..4d9df6c5d 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CassandraClusterSchemaMonitor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CassandraClusterSchemaMonitor.java @@ -129,6 +129,11 @@ public void refresh() } } + public Set getCdcTables() + { + return cdcTables.get(); + } + private Partitioner getPartitioner(NodeSettings nodeSettings) { if (nodeSettings.partitioner().contains(".")) @@ -219,8 +224,8 @@ private static Set buildCdcTables(@NotNull String fullSchema, TableIdentifier id = e.getKey(); ReplicationFactor rf = CqlUtils.extractReplicationFactor(fullSchema, id.keyspace()); - return cassandraBridge.buildSchema(e.getValue(), id.keyspace(), rf, - partitioner, udtsPerKeyspace.get(id.keyspace()), + return cassandraBridge.buildSchema(e.getValue(), id.keyspace(), rf, + partitioner, udtsPerKeyspace.get(id.keyspace()), tableIds.get(id), 0, true); }) .collect(Collectors.toSet()); diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java new file mode 100644 index 000000000..da4049cb4 --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java @@ -0,0 +1,145 @@ +package org.apache.cassandra.sidecar.tasks; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; +import org.apache.cassandra.sidecar.codecs.CdcConfigMappingsCodec; +import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; +import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.db.CdcConfigAccessor; +import org.apache.cassandra.sidecar.db.KafkaConfigAccessor; +import org.apache.cassandra.sidecar.utils.EventBusUtils; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_SCHEMA_INITIALIZED; + +/** + * Periodic task notifying if the CDC config changed + */ +@Singleton +public class CdcConfigRefresherNotifierTask implements PeriodicTask +{ + private static final Logger LOGGER = LoggerFactory.getLogger(CdcConfigRefresherNotifierTask.class); + private final SidecarConfiguration sidecarConfiguration; + private final CdcConfigAccessor cdcConfigAccessor; + private final KafkaConfigAccessor kafkaConfigAccessor; + private final Vertx vertx; + private final List configChangeListeners = Collections.synchronizedList(new ArrayList<>()); + public ConfigMappings configMappings = new ConfigMappings(); + + @Inject + public CdcConfigRefresherNotifierTask(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + KafkaConfigAccessor kafkaConfigAccessor, + CdcConfigAccessor cdcConfigAccessor) + { + this.sidecarConfiguration = sidecarConfiguration; + this.kafkaConfigAccessor = kafkaConfigAccessor; + this.cdcConfigAccessor = cdcConfigAccessor; + this.vertx = vertx; + + vertx.eventBus().registerDefaultCodec(ConfigMappings.class, CdcConfigMappingsCodec.INSTANCE); + } + + public void registerConfigChangeListener(ThrowingRunnable listener) + { + this.configChangeListeners.add(listener); + } + + @Override + public DurationSpec delay() + { + return sidecarConfiguration.serviceConfiguration().cdcConfiguration().cdcConfigRefreshTime(); + } + + @Override + public void execute(Promise promise) + { + vertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), configMappings); + promise.tryComplete(); + } + + // skip if any of the following condition is true + // - sidecar schema not enabled or cdc not enabled + // - both configs have not changed + @Override + public ScheduleDecision scheduleDecision() + { + if (!sidecarConfiguration.serviceConfiguration().schemaKeyspaceConfiguration().isEnabled() || !sidecarConfiguration.serviceConfiguration().cdcConfiguration().isEnabled()) + { + LOGGER.trace("Skipping config refreshing"); + return ScheduleDecision.SKIP; + } + + Map newKafkaConfigMappings; + Map newCdcConfigMappings; + try + { + newKafkaConfigMappings = kafkaConfigAccessor.getConfig().getConfigs(); + newCdcConfigMappings = cdcConfigAccessor.getConfig().getConfigs(); + } + catch (Throwable e) + { + LOGGER.error("Failed to access cdc/kafka configs", e); + return ScheduleDecision.SKIP; + } + + boolean shouldSkip = true; + if (!newKafkaConfigMappings.equals(configMappings.getKafkaConfigMappings())) + { + shouldSkip = false; + configMappings.setKafkaConfigMappings(newKafkaConfigMappings); + } + if (!newCdcConfigMappings.equals(configMappings.getCdcConfigMappings())) + { + shouldSkip = false; + configMappings.setCdcConfigMappings(newCdcConfigMappings); + } + return shouldSkip ? ScheduleDecision.SKIP : ScheduleDecision.EXECUTE; + } + + @Override + public void deploy(Vertx vertx, PeriodicTaskExecutor executor) + { + EventBusUtils.onceLocalConsumer(vertx.eventBus(), ON_SIDECAR_SCHEMA_INITIALIZED.address(), ignored -> executor.schedule(this)); + } + + /** + * Wrapper class containing kafka and cdc configs + */ + public static class ConfigMappings + { + private Map kafkaConfigMappings = Map.of(); + private Map cdcConfigMappings = Map.of(); + + public Map getKafkaConfigMappings() + { + return kafkaConfigMappings; + } + + public void setKafkaConfigMappings(Map kafkaConfigMappings) + { + this.kafkaConfigMappings = kafkaConfigMappings; + } + + public Map getCdcConfigMappings() + { + return cdcConfigMappings; + } + + public void setCdcConfigMappings(Map cdcConfigMappings) + { + this.cdcConfigMappings = cdcConfigMappings; + } + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java new file mode 100644 index 000000000..c6d9afc6b --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java @@ -0,0 +1,296 @@ +package org.apache.cassandra.sidecar.tasks; + +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; +import org.apache.cassandra.sidecar.codecs.DcLocalTopologyChangeEventCodec; + +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.coordination.TokenRingProvider; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; + +/** + * Periodic task that monitors Cassandra cluster topology changes and publishes events when detected. + */ +@Singleton +public class ClusterTopologyMonitor implements PeriodicTask +{ + /** + * Event types published when cluster topology changes are detected. + */ + public enum ClusterTopologyEventType + { + ON_DC_TOPOLOGY_CHANGE; + + /** + * Returns the event bus address for this event type. + */ + public String address() + { + return ClusterTopologyMonitor.class.getName() + "." + name(); + } + } + + private static final Logger LOGGER = LoggerFactory.getLogger(ClusterTopologyMonitor.class); + private final Vertx vertx; + private final TokenRingProvider tokenRingProvider; + + private final ConcurrentHashMap>> perDcRanges = new ConcurrentHashMap<>(4); + + /** + * Event data containing datacenter-local topology change information. + */ + public static class DcLocalTopologyChangeEvent + { + public final String dc; + @Nullable + public final Map> prev; + @NotNull + public final Map> curr; + + /** + * Creates a new datacenter topology change event. + */ + public DcLocalTopologyChangeEvent(String dc, + @Nullable Map> prev, + @NotNull Map> curr) + { + this.dc = dc; + this.prev = prev; + this.curr = curr; + } + + /** + * Factory method to create a topology change event with instance ID keyed maps. + */ + public static DcLocalTopologyChangeEvent of(String dc, + @Nullable Map> prev, + @NotNull Map> curr) + { + return new DcLocalTopologyChangeEvent(dc, keyByInstanceId(prev), keyByInstanceId(curr)); + } + + private static Map> keyByInstanceId(@Nullable Map> map) + { + return map == null ? null : map.entrySet().stream() + .collect(Collectors.toUnmodifiableMap(Map.Entry::getKey, + Map.Entry::getValue)); + } + + public boolean equals(Object o) + { + if (this == o) + { + return true; + } + if (o == null || getClass() != o.getClass()) + { + return false; + } + + DcLocalTopologyChangeEvent that = (DcLocalTopologyChangeEvent) o; + return Objects.equals(dc, that.dc) && + Objects.equals(prev, that.prev) && + curr.equals(that.curr); + } + + public int hashCode() + { + return Objects.hash(dc, prev, curr); + } + } + + /** + * Creates a cluster topology monitor with periodic task scheduling. + */ + @Inject + public ClusterTopologyMonitor(Vertx vertx, TokenRingProvider tokenRingProvider, PeriodicTaskExecutor periodicTaskExecutor) + { + this(vertx, tokenRingProvider); + LOGGER.info("Starting Cluster Topology Monitor"); + periodicTaskExecutor.schedule(this); + LOGGER.info("Cluster Topology Monitor started"); + } + + /** + * Creates a cluster topology monitor without scheduling. + */ + public ClusterTopologyMonitor(Vertx vertx, TokenRingProvider tokenRingProvider) + { + this.vertx = vertx; + this.tokenRingProvider = tokenRingProvider; + try + { + vertx.eventBus().registerDefaultCodec(DcLocalTopologyChangeEvent.class, DcLocalTopologyChangeEventCodec.INSTANCE); + } + catch (IllegalStateException ise) + { + if (ise.getMessage().contains("Already a default codec registered")) + { + // TODO: do not register codec in the constructor when OSS; All codecs should be managed in a central repository + // ignore + } + else + { + throw ise; + } + } + } + + // periodic task + + /** + * Returns the delay between topology refresh cycles. + */ + @Override + public DurationSpec delay() + { +// return MillisecondBoundConfiguration.parse(sidecarConfiguration.serviceConfiguration().refreshClusterTopology()); + return MillisecondBoundConfiguration.parse("1000"); + } + + /** + * Executes the topology monitoring task. + */ + @Override + public void execute(Promise promise) + { + try + { + refresh(); + } + catch (Exception e) + { + LOGGER.error("Unexpected error in ClusterTopologyMonitor", e); + } + finally + { + promise.complete(); + } + } + + // internal methods + + /** + * Refreshes topology information for all datacenters and publishes change events. + */ + protected void refresh() + { + for (String dc : tokenRingProvider.dcs()) + { + @Nullable final Map> prev = perDcRanges.get(dc); + + final Map> curr = tokenRingProvider.getPrimaryTokenRanges(dc); + if (!curr.equals(prev) && changeInInstanceTopology(prev, curr) && update(dc, prev, curr)) + { + LOGGER.info("Publishing ON_DC_TOPOLOGY_CHANGE event"); + vertx.eventBus().publish(ClusterTopologyEventType.ON_DC_TOPOLOGY_CHANGE.address(), + DcLocalTopologyChangeEvent.of(dc, prev, curr)); + } + } + } + + protected boolean update(String dc, + @Nullable Map> prev, + Map> curr) + { + if (prev == null) + { + return perDcRanges.putIfAbsent(dc, curr) == null; + } + return perDcRanges.replace(dc, prev, curr); + } + + protected static boolean changeInInstanceTopology(@Nullable Map> prev, + @NotNull Map> curr) + { + if (prev == null) + { + LOGGER.info("Bootstrapping cluster topology curr={}", curr.size()); + return true; + } + + boolean result = false; + if (prev.size() != curr.size()) + { + LOGGER.info("Cluster topology change detected: change in instance count prev={} curr={}", prev.size(), curr.size()); + result = true; + } + + for (Map.Entry> entry : curr.entrySet()) + { + final String instanceId = entry.getKey(); + @Nullable final List prevRanges = prev.get(instanceId); + final List currRanges = entry.getValue(); + if (prevRanges == null) + { + LOGGER.info("Cluster topology change detected: new instance added instanceId={} prev='{}' curr='{}'", instanceId, prev, curr); + result = true; + } + else if (changeInInstanceTopology(instanceId, prevRanges, currRanges)) + { + result = true; + } + } + + for (Map.Entry> entry : prev.entrySet()) + { + final String instanceId = entry.getKey(); + if (!curr.containsKey(instanceId)) + { + LOGGER.info("Cluster topology change detected: existing instance removed instanceId={} prev='{}' curr='{}'", instanceId, prev, curr); + result = true; + } + } + + return result; + } + + protected static boolean changeInInstanceTopology(String instanceId, List prev, List curr) + { + if (prev.size() != curr.size()) + { + LOGGER.info("Change in instance tokens instanceId={} prev='{}' curr='{}'", instanceId, prev, curr); + return true; + } + + prev = new ArrayList<>(prev); + prev.sort(Comparator.comparing(t -> t.range.lowerEndpoint())); + curr = new ArrayList<>(curr); + curr.sort(Comparator.comparing(t -> t.range.lowerEndpoint())); + + boolean result = false; + for (int i = 0; i < prev.size(); i++) + { + final TokenRange prevRange = prev.get(i); + final TokenRange currRange = curr.get(i); + if (!prevRange.range.lowerEndpoint().equals(currRange.range.lowerEndpoint())) + { + LOGGER.info("Change in instance lower token instanceId={} prev={} curr={}", instanceId, prevRange.range.lowerEndpoint(), currRange.range.lowerEndpoint()); + result = true; + } + if (!prevRange.range.upperEndpoint().equals(currRange.range.upperEndpoint())) + { + LOGGER.info("Change in instance upper token instanceId={} prev={} curr={}", instanceId, prevRange.range.upperEndpoint(), currRange.range.upperEndpoint()); + result = true; + } + } + + return result; + } +} diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java new file mode 100644 index 000000000..b12c559af --- /dev/null +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java @@ -0,0 +1,334 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Collections; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +import io.vertx.core.Vertx; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.bridge.CdcBridge; +import org.apache.cassandra.bridge.CdcBridgeFactory; +import org.apache.cassandra.cdc.api.TableIdLookup; +import org.apache.cassandra.cdc.avro.AvroSchemas; +import org.apache.cassandra.cdc.avro.CqlToAvroSchemaConverter; +import org.apache.cassandra.sidecar.db.TableHistoryDatabaseAccessor; +import org.apache.cassandra.sidecar.db.schema.SidecarSchema; +import org.apache.cassandra.sidecar.tasks.CassandraClusterSchemaMonitor; +import org.apache.cassandra.sidecar.utils.TokenSplitUtil; +import org.apache.cassandra.spark.data.CqlTable; +import org.apache.cassandra.spark.data.ReplicationFactor; +import org.apache.cassandra.spark.data.partitioner.Partitioner; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.assertj.core.api.Assertions.assertThatRuntimeException; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Tests for Caching Schema Store + */ +public class CachingSchemaStoreTest +{ + private static final String TEST_KEYSPACE = "test_keyspace"; + private static final String TEST_TABLE = "test_table"; + private static final String CREATE_STATEMENT = "CREATE TABLE " + TEST_KEYSPACE + '.' + TEST_TABLE + " (\n" + + " a bigint PRIMARY KEY,\n" + + " b text\n" + + ") WITH cdc = true;"; + private static final ReplicationFactor REPLICATION_FACTOR = new ReplicationFactor(ReplicationFactor.ReplicationStrategy.NetworkTopologyStrategy, + Map.of("DC1", 3)); + private static final Partitioner PARTITIONER = Partitioner.Murmur3Partitioner; + + CassandraBridge cassandraBridge; + + final Vertx vertx = Vertx.vertx(); + SidecarSchema mockSidecarSchema; + CassandraClusterSchemaMonitor mockCassandraClusterSchemaMonitor; + CachingSchemaStore cachingSchemaStore; + CdcConfigImpl mockCdcConfig; + SidecarCdcStats mockSidecarCdcStats; + TableHistoryDatabaseAccessor spyTableHistoryDatabaseAccessor; + CqlToAvroSchemaConverter cqlToAvroSchemaConverter; + + private void setupForVersion(CassandraVersion version) + { + TokenSplitUtil tokenSplitUtil = new TokenSplitUtil(32); + + mockSidecarSchema = mock(SidecarSchema.class); + when(mockSidecarSchema.isInitialized()).thenReturn(true); + + cassandraBridge = new CassandraBridgeFactory().get(version); + cqlToAvroSchemaConverter = CdcBridgeFactory.getCqlToAvroSchemaConverter(cassandraBridge); + + mockCassandraClusterSchemaMonitor = mock(CassandraClusterSchemaMonitor.class); + + mockCdcConfig = mock(CdcConfigImpl.class); + + TableHistoryDatabaseAccessor tableHistoryDatabaseAccessor = new TableHistoryDatabaseAccessor(mockSidecarSchema, null); + spyTableHistoryDatabaseAccessor = spy(tableHistoryDatabaseAccessor); + doReturn(null).when(spyTableHistoryDatabaseAccessor).insertTableSchemaHistory(anyString(), anyString(), anyString()); + + mockSidecarCdcStats = mock(SidecarCdcStats.class); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStore(CassandraVersion version) + { + setupForVersion(version); + Set initialTables = cqlTables(CREATE_STATEMENT); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables); + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + + // Schema + Schema schema = cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + TEST_TABLE, ""); + verifyFieldType(schema.getField("a").schema(), Schema.Type.LONG, "bigint"); + verifyFieldType(schema.getField("b").schema(), Schema.Type.STRING, "text"); + + // Writer + GenericDatumWriter writer = cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + TEST_TABLE, ""); + assertNotNull(writer); + + // Reader + GenericDatumReader reader = cachingSchemaStore.getReader(TEST_KEYSPACE + "." + TEST_TABLE, ""); + assertNotNull(reader); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreWithSidecarSchemaEnabled(CassandraVersion version) + { + setupForVersion(version); + String newTableName = "new_test_table"; + String newSchema = modifyTableName(CREATE_STATEMENT, newTableName); + Set initialTables = cqlTables(CREATE_STATEMENT); + + Set addedTables = new HashSet<>(cqlTables(newSchema)); + addedTables.addAll(initialTables); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables, addedTables); + + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + cachingSchemaStore.onSchemaChanged(); + + verify(spyTableHistoryDatabaseAccessor, times(1)).insertTableSchemaHistory(eq(TEST_KEYSPACE), eq(newTableName), any()); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreUnKnownTable(CassandraVersion version) + { + setupForVersion(version); + String newTableName = "new_test_table"; + String newSchema = modifyTableName(CREATE_STATEMENT, newTableName); + Set initialTables = cqlTables(CREATE_STATEMENT); + + Set addedTables = new HashSet<>(cqlTables(newSchema)); + addedTables.addAll(initialTables); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables, addedTables); + + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + + assertUnknownTableException("unknown", "schema"); + assertUnknownTableException("unknown", "writer"); + assertUnknownTableException("unknown", "reader"); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreSchemaNotChanged(CassandraVersion version) + { + setupForVersion(version); + String newTableName = "new_test_table"; + String newSchema = modifyTableName(CREATE_STATEMENT, newTableName); + Set initialTables = cqlTables(CREATE_STATEMENT); + + Set addedTables = new HashSet<>(cqlTables(newSchema)); + addedTables.addAll(initialTables); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables, addedTables); + + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + + // Should return cached schema object when schema is not changed + assertSchemaCached(TEST_TABLE); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreNewTableAdded(CassandraVersion version) + { + setupForVersion(version); + String newTable = "new_test_table"; + String newSchema = modifyTableName(CREATE_STATEMENT, newTable); + Set initialTables = cqlTables(CREATE_STATEMENT); + Set addedTables = new HashSet<>(cqlTables(newSchema)); + addedTables.addAll(initialTables); + + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables, addedTables); + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + + assertUnknownTableException(newTable, "schema"); + assertUnknownTableException(newTable, "writer"); + assertUnknownTableException(newTable, "reader"); + + cachingSchemaStore.onSchemaChanged(); + + assertSchemaAccessible(newTable); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreTableDeleted(CassandraVersion version) + { + setupForVersion(version); + Set tables = cqlTables(CREATE_STATEMENT); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(tables, Collections.emptySet()); + + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + assertSchemaAccessible(TEST_TABLE); + + cachingSchemaStore.onSchemaChanged(); + + assertUnknownTableException(TEST_TABLE, "schema"); + assertUnknownTableException(TEST_TABLE, "writer"); + assertUnknownTableException(TEST_TABLE, "reader"); + } + + @ParameterizedTest + @EnumSource(value = CassandraVersion.class, names = {"FOURZERO", "FOURONE" /*, "FIVEZERO" is having problems*/}) + public void testCachingSchemaStoreTableSchemaChanged(CassandraVersion version) + { + setupForVersion(version); + String changedSchema = "CREATE TABLE test_keyspace.test_table (\n" + + " a bigint PRIMARY KEY,\n" + + " b text,\n" + + " c bigint\n" + + ") WITH additional_write_policy = '99p'\n" + + " AND cdc = true;"; + + Set initialTables = cqlTables(CREATE_STATEMENT); + mockCassandraClusterSchemaMonitor = createMockClusterSchema(initialTables); + cachingSchemaStore = createCachingSchemaStore(mockCassandraClusterSchemaMonitor); + + Schema schema1 = cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + TEST_TABLE, ""); + GenericDatumReader reader1 = cachingSchemaStore.getReader(TEST_KEYSPACE + "." + TEST_TABLE, ""); + GenericDatumWriter writer1 = cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + TEST_TABLE, ""); + + + // Update mock to return changed schema + Set changedTables = cqlTables(changedSchema); + when(mockCassandraClusterSchemaMonitor.getCdcTables()).thenReturn(changedTables); + + cachingSchemaStore.onSchemaChanged(); + + Schema schema2 = cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + TEST_TABLE, ""); + GenericDatumReader reader2 = cachingSchemaStore.getReader(TEST_KEYSPACE + "." + TEST_TABLE, ""); + GenericDatumWriter writer2 = cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + TEST_TABLE, ""); + + assertNotEquals(schema1, schema2); + assertNotEquals(reader1, reader2); + assertNotEquals(writer1, writer2); + } + + public Set cqlTables(String createStatement) + { + Set tables = Set.of(cassandraBridge.buildSchema(createStatement, TEST_KEYSPACE, REPLICATION_FACTOR, PARTITIONER)); + CdcBridge cdcBridge = CdcBridgeFactory.getCdcBridge(cassandraBridge); + cdcBridge.updateCdcSchema(tables, PARTITIONER, TableIdLookup.STUB); + return Set.of(cassandraBridge.buildSchema(createStatement, TEST_KEYSPACE, REPLICATION_FACTOR, PARTITIONER)); + } + + private CachingSchemaStore createCachingSchemaStore(CassandraClusterSchemaMonitor clusterSchema) + { + return new CachingSchemaStore(vertx, clusterSchema, spyTableHistoryDatabaseAccessor, + mockCdcConfig, mockSidecarCdcStats, + mockSidecarSchema, cqlToAvroSchemaConverter); + } + + private CassandraClusterSchemaMonitor createMockClusterSchema(Set tables) + { + CassandraClusterSchemaMonitor mock = mock(CassandraClusterSchemaMonitor.class); + when(mock.getCdcTables()).thenReturn(tables); + return mock; + } + + private CassandraClusterSchemaMonitor createMockClusterSchema(Set initialTables, Set changedTables) + { + CassandraClusterSchemaMonitor mock = mock(CassandraClusterSchemaMonitor.class); + when(mock.getCdcTables()).thenReturn(initialTables).thenReturn(changedTables); + return mock; + } + + private void assertUnknownTableException(String tableName, String operation) + { + String expectedMessage = String.format("Unable to get %s for unknown table TableIdentifier{keyspace='%s', table='%s'}", + operation, TEST_KEYSPACE, tableName); + switch (operation) + { + case "schema": + assertThatRuntimeException() + .isThrownBy(() -> cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + tableName, "")) + .withMessage(expectedMessage); + break; + case "writer": + assertThatRuntimeException() + .isThrownBy(() -> cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + tableName, "")) + .withMessage(expectedMessage); + break; + case "reader": + assertThatRuntimeException() + .isThrownBy(() -> cachingSchemaStore.getReader(TEST_KEYSPACE + "." + tableName, "")) + .withMessage(expectedMessage); + break; + } + } + + private void assertSchemaAccessible(String tableName) + { + assertNotNull(cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + tableName, "")); + assertNotNull(cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + tableName, "")); + assertNotNull(cachingSchemaStore.getReader(TEST_KEYSPACE + "." + tableName, "")); + } + + private void assertSchemaCached(String tableName) + { + assertSame(cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + tableName, ""), + cachingSchemaStore.getSchema(TEST_KEYSPACE + "." + tableName, "")); + assertSame(cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + tableName, ""), + cachingSchemaStore.getWriter(TEST_KEYSPACE + "." + tableName, "")); + assertSame(cachingSchemaStore.getReader(TEST_KEYSPACE + "." + tableName, ""), + cachingSchemaStore.getReader(TEST_KEYSPACE + "." + tableName, "")); + } + + private String modifyTableName(String createStatement, String newTableName) + { + return createStatement.replace(TEST_TABLE, newTableName); + } + + private void verifyFieldType(Schema fieldSchema, Schema.Type expectedType, String expectedCqlType) + { + Schema schema = fieldSchema.getTypes().stream().filter(Objects::nonNull).findAny().get(); + assertEquals(expectedType, schema.getType()); + assertEquals(expectedCqlType, AvroSchemas.cqlType(schema)); + } +} diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java index 63f44c894..952a84712 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java @@ -39,7 +39,6 @@ import org.apache.cassandra.sidecar.db.CdcConfigAccessor; import org.apache.cassandra.sidecar.db.KafkaConfigAccessor; import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; -import org.apache.cassandra.sidecar.tasks.ScheduleDecision; import static org.apache.cassandra.testing.utils.AssertionUtils.loopAssert; import static org.assertj.core.api.Assertions.assertThat; @@ -82,8 +81,7 @@ void testIsConfigReadySchemaNotInitialized() KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.isAvailable()).thenReturn(false); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.isConfigReady()).isFalse(); } @@ -93,8 +91,7 @@ void testIsConfigReadyKafkaConfigsEmpty() CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("k1", "v1")); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.isConfigReady()).isFalse(); } @@ -104,8 +101,7 @@ void testIsConfigReadyCdcConfigsEmpty() CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("k1", "v1")); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.isConfigReady()).isFalse(); } @@ -114,8 +110,7 @@ void testReturnDefaultValuesWhenConfigsAreEmpty() { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.datacenter()).isEqualTo(null); assertThat(cdcConfig.env()).isEqualTo(""); assertThat(cdcConfig.kafkaTopic()).isNull(); @@ -139,8 +134,7 @@ void testConfigsWhenConfigsAreNotEmpty() when(kafkaConfigAccessor.getConfig().getConfigs()) .thenReturn(Map.of("k1", "v1")); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); loopAssert(5, ()-> assertThat(cdcConfig.isConfigReady()).isTrue()); assertThat(cdcConfig.datacenter()).isEqualTo("DC1"); assertThat(cdcConfig.env()).isEqualTo("if"); @@ -161,8 +155,7 @@ void testConfigChanged() throws Exception "log_only", "false")); when(kafkaConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("topic", "topic1")); - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); + CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); cdcConfig.registerConfigChangeListener(listener); // do not wait the periodic task execution, we force running it immediately. @@ -189,23 +182,6 @@ void testConfigChanged() throws Exception verify(listener, times(2)).run(); } - @Test - void testNotifierIsSkippedWhenCdcIsDisabled() - { - CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - when(cdcConfigAccessor.isAvailable()).thenReturn(true); - - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); - CdcConfiguration cdcConfiguration = mockCdcConfiguration(); - when(cdcConfiguration.isEnabled()).thenReturn(false); - - CdcConfigImpl cdcConfig = - new CdcConfigImpl(mockSidecarConfiguration(), cdcConfigAccessor, kafkaConfigAccessor, executor); - assertThat(cdcConfig.configRefreshNotifier().scheduleDecision()) - .isEqualTo(ScheduleDecision.SKIP) - .describedAs("When sidecarSchema is enabled but cdc is disabled, the refresh notifier should skip"); - } - private CdcConfigAccessor mockCdcConfigAccessor() { CdcConfigAccessor cdcConfigAccessor = mock(CdcConfigAccessor.class, RETURNS_DEEP_STUBS); diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java new file mode 100644 index 000000000..e518fa6bc --- /dev/null +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java @@ -0,0 +1,297 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.io.IOException; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.cdc.api.EventConsumer; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.cdc.sidecar.SidecarCdc; +import org.apache.cassandra.cdc.sidecar.SidecarCdcClient; +import org.apache.cassandra.cdc.stats.ICdcStats; +import org.apache.cassandra.secrets.SecretsProvider; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool; +import org.apache.cassandra.sidecar.coordination.RangeManager; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +/** + * Unit tests for CdcManager + */ +public class CdcManagerTest +{ + @Mock + private EventConsumer eventConsumer; + @Mock + private SchemaSupplier schemaSupplier; + @Mock + private CdcConfig cdcConfig; + @Mock + private RangeManager rangeManager; + @Mock + private InstanceMetadataFetcher instanceFetcher; + @Mock + private ClusterConfigProvider clusterConfigProvider; + @Mock + private CdcSidecarInstancesProvider sidecarInstancesProvider; + @Mock + private SecretsProvider secretsProvider; + @Mock + private SidecarCdcClient.ClientConfig clientConfig; + @Mock + private ICdcStats cdcStats; + @Mock + private TaskExecutorPool taskExecutorPool; + @Mock + private CdcDatabaseAccessor cdcDatabaseAccessor; + + private CdcManager cdcManager; + + @BeforeEach + void setUp() + { + MockitoAnnotations.openMocks(this); + + cdcManager = new CdcManager( + eventConsumer, + schemaSupplier, + cdcConfig, + rangeManager, + instanceFetcher, + clusterConfigProvider, + sidecarInstancesProvider, + secretsProvider, + clientConfig, + cdcStats, + taskExecutorPool, + cdcDatabaseAccessor + ); + } + + @Test + void testNullOwnedRangesThrowsException() + { + when(rangeManager.ownedTokenRanges()).thenReturn(null); + + assertThatThrownBy(() -> cdcManager.buildCdcConsumers()) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("No owned token ranges"); + } + + @Test + void testEmptyOwnedRangesThrowsException() + { + when(rangeManager.ownedTokenRanges()).thenReturn(Collections.emptyMap()); + + assertThatThrownBy(() -> cdcManager.buildCdcConsumers()) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("No owned token ranges"); + } + + @Test + void testSingleInstanceSingleRangeCreatesOneConsumer() throws IOException + { + String instanceIp = "127.0.0.1"; + int instanceId = 1; + + TokenRange range = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + Set ranges = Collections.singleton(range); + Map> ownedRanges = Collections.singletonMap(instanceIp, ranges); + + InstanceMetadata instance = mockInstance(instanceId, instanceIp); + + when(rangeManager.ownedTokenRanges()).thenReturn(ownedRanges); + when(instanceFetcher.allLocalInstances()).thenReturn(Collections.singletonList(instance)); + when(cdcConfig.jobId()).thenReturn("test-job"); + + // Spy to mock loadOrBuildCdcConsumer + CdcManager spyManager = spy(cdcManager); + SidecarCdc mockConsumer = mock(SidecarCdc.class); + doReturn(mockConsumer).when(spyManager).loadOrBuildCdcConsumer( + anyInt(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any() + ); + + List consumers = spyManager.buildCdcConsumers(); + + // Assert + assertThat(consumers).hasSize(1); + } + + @Test + void testSingleInstanceMultipleRangesCreatesMultipleConsumers() throws IOException + { + String instanceIp = "127.0.0.1"; + int instanceId = 1; + + TokenRange range1 = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + TokenRange range2 = mockTokenRange(BigInteger.TEN, new BigInteger("20")); + Set ranges = new HashSet<>(); + ranges.add(range1); + ranges.add(range2); + + Map> ownedRanges = Collections.singletonMap(instanceIp, ranges); + + InstanceMetadata instance = mockInstance(instanceId, instanceIp); + + when(rangeManager.ownedTokenRanges()).thenReturn(ownedRanges); + when(instanceFetcher.allLocalInstances()).thenReturn(Collections.singletonList(instance)); + when(cdcConfig.jobId()).thenReturn("test-job"); + + // Spy to mock loadOrBuildCdcConsumer + CdcManager spyManager = spy(cdcManager); + SidecarCdc mockConsumer1 = mock(SidecarCdc.class); + SidecarCdc mockConsumer2 = mock(SidecarCdc.class); + doReturn(mockConsumer1, mockConsumer2).when(spyManager).loadOrBuildCdcConsumer( + anyInt(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any() + ); + + List consumers = spyManager.buildCdcConsumers(); + + // Assert + assertThat(consumers).hasSize(2); + } + + @Test + void testMultipleInstancesMultipleRangesCreatesConsumers() throws IOException + { + String instance1Ip = "127.0.0.1"; + String instance2Ip = "127.0.0.2"; + int instance1Id = 1; + int instance2Id = 2; + + TokenRange range1 = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + TokenRange range2 = mockTokenRange(BigInteger.TEN, new BigInteger("20")); + + Map> ownedRanges = new HashMap<>(); + ownedRanges.put(instance1Ip, Collections.singleton(range1)); + ownedRanges.put(instance2Ip, Collections.singleton(range2)); + + InstanceMetadata instance1 = mockInstance(instance1Id, instance1Ip); + InstanceMetadata instance2 = mockInstance(instance2Id, instance2Ip); + + List instances = new ArrayList<>(); + instances.add(instance1); + instances.add(instance2); + + when(rangeManager.ownedTokenRanges()).thenReturn(ownedRanges); + when(instanceFetcher.allLocalInstances()).thenReturn(instances); + when(cdcConfig.jobId()).thenReturn("test-job"); + + // Spy to mock loadOrBuildCdcConsumer + CdcManager spyManager = spy(cdcManager); + SidecarCdc mockConsumer1 = mock(SidecarCdc.class); + SidecarCdc mockConsumer2 = mock(SidecarCdc.class); + doReturn(mockConsumer1, mockConsumer2).when(spyManager).loadOrBuildCdcConsumer( + anyInt(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any() + ); + + List consumers = spyManager.buildCdcConsumers(); + + // Assert + assertThat(consumers).hasSize(2); + } + + @Test + void testDuplicateRangesDeduplicates() throws IOException + { + String instanceIp = "127.0.0.1"; + int instanceId = 1; + + // Create two identical ranges + TokenRange range1 = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + TokenRange range2 = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + + Set ranges = new HashSet<>(); + ranges.add(range1); + ranges.add(range2); + + Map> ownedRanges = Collections.singletonMap(instanceIp, ranges); + + InstanceMetadata instance = mockInstance(instanceId, instanceIp); + + when(rangeManager.ownedTokenRanges()).thenReturn(ownedRanges); + when(instanceFetcher.allLocalInstances()).thenReturn(Collections.singletonList(instance)); + when(cdcConfig.jobId()).thenReturn("test-job"); + + // Spy to mock loadOrBuildCdcConsumer + CdcManager spyManager = spy(cdcManager); + SidecarCdc mockConsumer = mock(SidecarCdc.class); + doReturn(mockConsumer).when(spyManager).loadOrBuildCdcConsumer( + anyInt(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any() + ); + + List consumers = spyManager.buildCdcConsumers(); + + // Assert - Should deduplicate to 1 consumer + assertThat(consumers).hasSize(1); + } + + @Test + void testUnknownInstanceHandlesGracefully() throws IOException + { + String unknownIp = "192.168.1.100"; + + TokenRange range = mockTokenRange(BigInteger.ZERO, BigInteger.TEN); + Map> ownedRanges = Collections.singletonMap(unknownIp, Collections.singleton(range)); + + // No matching instances + when(rangeManager.ownedTokenRanges()).thenReturn(ownedRanges); + when(instanceFetcher.allLocalInstances()).thenReturn(Collections.emptyList()); + when(cdcConfig.jobId()).thenReturn("test-job"); + + // Spy to mock loadOrBuildCdcConsumer - will be called with instanceId = -1 + CdcManager spyManager = spy(cdcManager); + SidecarCdc mockConsumer = mock(SidecarCdc.class); + doReturn(mockConsumer).when(spyManager).loadOrBuildCdcConsumer( + anyInt(), any(), any(), any(), any(), any(), any(), any(), any(), any(), any() + ); + + List consumers = spyManager.buildCdcConsumers(); + + // Assert - Should still create consumer with instanceId = -1 + assertThat(consumers).hasSize(1); + } + + // Helper methods + + private TokenRange mockTokenRange(BigInteger start, BigInteger end) + { + TokenRange range = mock(TokenRange.class, RETURNS_DEEP_STUBS); + when(range.startAsBigInt()).thenReturn(start); + when(range.endAsBigInt()).thenReturn(end); + return range; + } + + private InstanceMetadata mockInstance(int id, String ipAddress) + { + InstanceMetadata instance = mock(InstanceMetadata.class, RETURNS_DEEP_STUBS); + when(instance.id()).thenReturn(id); + when(instance.ipAddress()).thenReturn(ipAddress); + return instance; + } +} diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java new file mode 100644 index 000000000..331e57a60 --- /dev/null +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java @@ -0,0 +1,291 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import io.vertx.core.Vertx; +import org.apache.cassandra.cdc.api.EventConsumer; +import org.apache.cassandra.cdc.api.SchemaSupplier; +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.apache.cassandra.cdc.sidecar.CdcSidecarInstancesProvider; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.cdc.sidecar.SidecarCdcClient; +import org.apache.cassandra.cdc.stats.ICdcStats; +import org.apache.cassandra.secrets.SecretsProvider; +import org.apache.cassandra.sidecar.common.server.utils.SecondBoundConfiguration; +import org.apache.cassandra.sidecar.concurrent.ExecutorPools; +import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool; +import org.apache.cassandra.sidecar.config.KeyStoreConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.SslConfiguration; +import org.apache.cassandra.sidecar.coordination.TokenRingProvider; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.kafka.common.serialization.Serializer; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.RETURNS_DEEP_STUBS; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Unit tests for CdcPublisher + */ +public class CdcPublisherTests +{ + @Mock + private Vertx vertx; + @Mock + private ExecutorPools executorPools; + @Mock + private TaskExecutorPool taskExecutorPool; + @Mock + private ClusterConfigProvider clusterConfigProvider; + @Mock + private SchemaSupplier schemaSupplier; + @Mock + private CdcSidecarInstancesProvider sidecarInstancesProvider; + @Mock + private SidecarCdcClient.ClientConfig clientConfig; + @Mock + private InstanceMetadataFetcher instanceMetadataFetcher; + @Mock + private CdcDatabaseAccessor databaseAccessor; + @Mock + private ICdcStats cdcStats; + @Mock + private TokenRingProvider tokenRingProvider; + @Mock + private VirtualTablesDatabaseAccessor virtualTables; + @Mock + private SidecarCdcStats sidecarCdcStats; + @Mock + private Serializer avroSerializer; + + private SidecarConfiguration sidecarConfiguration; + private CdcConfig cdcConfig; + private CdcPublisher cdcPublisher; + + @BeforeEach + void setUp() + { + MockitoAnnotations.openMocks(this); + + // Mock deep stubs for complex configuration objects + sidecarConfiguration = mock(SidecarConfiguration.class, RETURNS_DEEP_STUBS); + cdcConfig = mock(CdcConfig.class, RETURNS_DEEP_STUBS); + + // Mock ExecutorPools behavior + when(executorPools.internal()).thenReturn(taskExecutorPool); + + // Mock Vertx EventBus for event listeners + when(vertx.eventBus()).thenReturn(mock(io.vertx.core.eventbus.EventBus.class, RETURNS_DEEP_STUBS)); + + cdcPublisher = new CdcPublisher( + vertx, + sidecarConfiguration, + executorPools, + clusterConfigProvider, + schemaSupplier, + sidecarInstancesProvider, + clientConfig, + instanceMetadataFetcher, + cdcConfig, + databaseAccessor, + cdcStats, + tokenRingProvider, + virtualTables, + sidecarCdcStats, + avroSerializer + ); + } + + + @Test + void testSecretsProviderReturnsNullWhenSslDisabled() + { + SslConfiguration sslConfig = mock(SslConfiguration.class); + when(sidecarConfiguration.sidecarClientConfiguration().sslConfiguration()).thenReturn(sslConfig); + when(sslConfig.enabled()).thenReturn(false); + + SecretsProvider result = cdcPublisher.secretsProvider(); + + assertThat(result).isNull(); + } + + @Test + void testSecretsProviderWithSslEnabledNoKeystoreNoTruststore() + { + SslConfiguration sslConfig = mockSslConfiguration( + true, // enabled + true, // preferOpenSSL + "REQUIRED", // clientAuth + Arrays.asList("TLS_RSA_128"), // cipherSuites + Arrays.asList("TLSv1.2"), // secureTransportProtocols + "10s", // handshakeTimeout + false, // keystoreConfigured + false // truststoreConfigured + ); + + when(sidecarConfiguration.sidecarClientConfiguration().sslConfiguration()).thenReturn(sslConfig); + + SecretsProvider result = cdcPublisher.secretsProvider(); + + assertThat(result).isNotNull(); + } + + @Test + void testSecretsProviderWithKeystoreOnly() + { + KeyStoreConfiguration keystoreConfig = mockKeystoreConfiguration( + "/path/to/keystore.jks", + "keystorePassword", + "JKS" + ); + + SslConfiguration sslConfig = mockSslConfiguration( + true, + false, + "OPTIONAL", + Arrays.asList("TLS_RSA_256"), + Arrays.asList("TLSv1.3"), + "15s", + true, + false + ); + + when(sslConfig.keystore()).thenReturn(keystoreConfig); + when(sidecarConfiguration.sidecarClientConfiguration().sslConfiguration()).thenReturn(sslConfig); + + SecretsProvider result = cdcPublisher.secretsProvider(); + + assertThat(result).isNotNull(); + } + + @Test + void testSecretsProviderWithTruststoreOnly() + { + KeyStoreConfiguration truststoreConfig = mockKeystoreConfiguration( + "/path/to/truststore.jks", + "truststorePassword", + "PKCS12" + ); + + SslConfiguration sslConfig = mockSslConfiguration( + true, + true, + "NONE", + Collections.emptyList(), + Arrays.asList("TLSv1.2", "TLSv1.3"), + "20s", + false, + true + ); + + when(sslConfig.truststore()).thenReturn(truststoreConfig); + when(sidecarConfiguration.sidecarClientConfiguration().sslConfiguration()).thenReturn(sslConfig); + + SecretsProvider result = cdcPublisher.secretsProvider(); + + assertThat(result).isNotNull(); + } + + @Test + void testSecretsProviderWithBothKeystoreAndTruststore() + { + KeyStoreConfiguration keystoreConfig = mockKeystoreConfiguration( + "/path/to/keystore.p12", + "keystorePass123", + "PKCS12" + ); + + KeyStoreConfiguration truststoreConfig = mockKeystoreConfiguration( + "/path/to/truststore.p12", + "truststorePass456", + "PKCS12" + ); + + SslConfiguration sslConfig = mockSslConfiguration( + true, + true, + "REQUIRED", + Arrays.asList("TLS_ECDHE_RSA", "TLS_AES_256"), + Arrays.asList("TLSv1.2", "TLSv1.3"), + "30s", + true, + true + ); + + when(sslConfig.keystore()).thenReturn(keystoreConfig); + when(sslConfig.truststore()).thenReturn(truststoreConfig); + when(sidecarConfiguration.sidecarClientConfiguration().sslConfiguration()).thenReturn(sslConfig); + + SecretsProvider result = cdcPublisher.secretsProvider(); + + assertThat(result).isNotNull(); + } + + @Test + void testEventConsumerCreatesValidConsumer() + { + Map kafkaConfigs = new HashMap<>(); + kafkaConfigs.put("bootstrap.servers", "localhost:9092"); + kafkaConfigs.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer"); + kafkaConfigs.put("value.serializer", "org.apache.kafka.common.serialization.ByteArraySerializer"); + + when(cdcConfig.kafkaConfigs()).thenReturn(kafkaConfigs); + when(cdcConfig.kafkaTopic()).thenReturn("test-cdc-topic"); + when(cdcConfig.maxRecordSizeBytes()).thenReturn(1048576); // 1MB + when(cdcConfig.failOnRecordTooLargeError()).thenReturn(false); + when(cdcConfig.failOnKafkaError()).thenReturn(true); + + EventConsumer result = cdcPublisher.eventConsumer(cdcConfig, avroSerializer); + + assertThat(result).isNotNull(); + assertThat(result).isInstanceOf(CdcEventConsumer.class); + } + + + private SslConfiguration mockSslConfiguration(boolean enabled, + boolean preferOpenSSL, + String clientAuth, + java.util.List cipherSuites, + java.util.List secureTransportProtocols, + String handshakeTimeout, + boolean keystoreConfigured, + boolean truststoreConfigured) + { + SslConfiguration sslConfig = mock(SslConfiguration.class, RETURNS_DEEP_STUBS); + when(sslConfig.enabled()).thenReturn(enabled); + when(sslConfig.preferOpenSSL()).thenReturn(preferOpenSSL); + when(sslConfig.clientAuth()).thenReturn(clientAuth); + when(sslConfig.cipherSuites()).thenReturn(cipherSuites); + when(sslConfig.secureTransportProtocols()).thenReturn(secureTransportProtocols); + + SecondBoundConfiguration durationSpec = mock(SecondBoundConfiguration.class); + when(durationSpec.toString()).thenReturn(handshakeTimeout); + when(sslConfig.handshakeTimeout()).thenReturn(durationSpec); + + when(sslConfig.isKeystoreConfigured()).thenReturn(keystoreConfigured); + when(sslConfig.isTrustStoreConfigured()).thenReturn(truststoreConfigured); + + return sslConfig; + } + + private KeyStoreConfiguration mockKeystoreConfiguration(String path, String password, String type) + { + KeyStoreConfiguration config = mock(KeyStoreConfiguration.class); + when(config.path()).thenReturn(path); + when(config.password()).thenReturn(password); + when(config.type()).thenReturn(type); + return config; + } +} diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java new file mode 100644 index 000000000..8d2401f5c --- /dev/null +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java @@ -0,0 +1,169 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; + +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import org.apache.cassandra.bridge.CassandraBridge; +import org.apache.cassandra.bridge.CassandraBridgeFactory; +import org.apache.cassandra.bridge.CassandraVersion; +import org.apache.cassandra.bridge.CdcBridge; +import org.apache.cassandra.sidecar.common.response.NodeSettings; +import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.data.CqlTable; +import org.apache.cassandra.spark.data.partitioner.Partitioner; +import org.apache.cassandra.spark.utils.TableIdentifier; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +/** + * Unit tests for CdcSchemaSupplier + */ +public class CdcSchemaSupplierTest +{ + @Mock + private InstanceMetadataFetcher instanceMetadataFetcher; + @Mock + private CassandraBridgeFactory cassandraBridgeFactory; + @Mock + private CdcDatabaseAccessor cdcDatabaseAccessor; + @Mock + private CassandraBridge cassandraBridge; + @Mock + private CdcBridge cdcBridge; + + private CdcSchemaSupplier cdcSchemaSupplier; + + private static final String SAMPLE_SCHEMA = + "CREATE KEYSPACE test_keyspace WITH REPLICATION = {'class':'NetworkTopologyStrategy','DC1':'3'} AND DURABLE_WRITES = true;\n" + + "CREATE TABLE test_keyspace.cdc_table (\n" + + " id uuid PRIMARY KEY,\n" + + " name text,\n" + + " value int\n" + + ") WITH cdc = true;\n"; + + private static final String SCHEMA_NO_CDC = + "CREATE KEYSPACE test_keyspace WITH REPLICATION = {'class':'NetworkTopologyStrategy','DC1':'3'} AND DURABLE_WRITES = true;\n" + + "CREATE TABLE test_keyspace.regular_table (\n" + + " id uuid PRIMARY KEY,\n" + + " data text\n" + + ");\n"; + + @BeforeEach + void setUp() + { + MockitoAnnotations.openMocks(this); + + cdcSchemaSupplier = new CdcSchemaSupplier( + instanceMetadataFetcher, + cassandraBridgeFactory, + cdcDatabaseAccessor + ); + } + + @Test + void testGetCdcEnabledTablesReturnsCompletedFuture() throws ExecutionException, InterruptedException + { + NodeSettings nodeSettings = mockNodeSettings("4.1.0", "org.apache.cassandra.dht.Murmur3Partitioner"); + + when(instanceMetadataFetcher.callOnFirstAvailableInstance(any())) + .thenReturn(SAMPLE_SCHEMA) // First call returns schema + .thenReturn(nodeSettings); // Second call returns node settings + + CassandraBridge realBridge = new CassandraBridgeFactory().get(CassandraVersion.FOURONE); + when(cassandraBridgeFactory.get(anyString())).thenReturn(realBridge); + when(cdcDatabaseAccessor.partitioner()).thenReturn(Partitioner.Murmur3Partitioner); + when(cdcDatabaseAccessor.getTableId(any(TableIdentifier.class))).thenReturn(UUID.randomUUID()); + + CompletableFuture> result = cdcSchemaSupplier.getCdcEnabledTables(); + + assertThat(result).isNotNull(); + assertThat(result.isDone()).isTrue(); + assertThat(result.isCompletedExceptionally()).isFalse(); + + Set tables = result.get(); + assertThat(tables).isNotNull(); + } + + @Test + void testGetCdcEnabledTablesWithMurmur3Partitioner() throws ExecutionException, InterruptedException + { + NodeSettings nodeSettings = mockNodeSettings("4.1.0", "Murmur3Partitioner"); + + when(instanceMetadataFetcher.callOnFirstAvailableInstance(any())) + .thenReturn(SAMPLE_SCHEMA) + .thenReturn(nodeSettings); + + CassandraBridge realBridge = new CassandraBridgeFactory().get(CassandraVersion.FOURONE); + when(cassandraBridgeFactory.get(anyString())).thenReturn(realBridge); + when(cdcDatabaseAccessor.partitioner()).thenReturn(Partitioner.Murmur3Partitioner); + when(cdcDatabaseAccessor.getTableId(any(TableIdentifier.class))).thenReturn(UUID.randomUUID()); + + CompletableFuture> result = cdcSchemaSupplier.getCdcEnabledTables(); + + assertThat(result).isCompleted(); + Set tables = result.get(); + assertThat(tables).isNotNull(); + } + + @Test + void testGetCdcEnabledTablesWithNoCdcTables() throws ExecutionException, InterruptedException + { + NodeSettings nodeSettings = mockNodeSettings("4.1.0", "Murmur3Partitioner"); + + when(instanceMetadataFetcher.callOnFirstAvailableInstance(any())) + .thenReturn(SCHEMA_NO_CDC) // Schema with no CDC tables + .thenReturn(nodeSettings); + + CassandraBridge realBridge = new CassandraBridgeFactory().get(CassandraVersion.FOURONE); + when(cassandraBridgeFactory.get(anyString())).thenReturn(realBridge); + when(cdcDatabaseAccessor.partitioner()).thenReturn(Partitioner.Murmur3Partitioner); + + CompletableFuture> result = cdcSchemaSupplier.getCdcEnabledTables(); + + assertThat(result).isCompleted(); + Set tables = result.get(); + assertThat(tables).isNotNull(); + assertThat(tables).isEmpty(); + } + + @Test + void testGetCdcEnabledTablesCallsCassandraBridgeFactory() + { + String releaseVersion = "4.1.0"; + NodeSettings nodeSettings = mockNodeSettings(releaseVersion, "Murmur3Partitioner"); + + when(instanceMetadataFetcher.callOnFirstAvailableInstance(any())) + .thenReturn(SAMPLE_SCHEMA) + .thenReturn(nodeSettings); + + CassandraBridge realBridge = new CassandraBridgeFactory().get(CassandraVersion.FOURONE); + when(cassandraBridgeFactory.get(anyString())).thenReturn(realBridge); + when(cdcDatabaseAccessor.partitioner()).thenReturn(Partitioner.Murmur3Partitioner); + when(cdcDatabaseAccessor.getTableId(any(TableIdentifier.class))).thenReturn(UUID.randomUUID()); + + cdcSchemaSupplier.getCdcEnabledTables(); + + verify(cassandraBridgeFactory).get(releaseVersion); + } + + private NodeSettings mockNodeSettings(String releaseVersion, String partitioner) + { + NodeSettings nodeSettings = mock(NodeSettings.class); + when(nodeSettings.releaseVersion()).thenReturn(releaseVersion); + when(nodeSettings.partitioner()).thenReturn(partitioner); + return nodeSettings; + } +} diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraClientTokenRingProviderTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraClientTokenRingProviderTest.java index 86b0c8d8d..39ccc16e7 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraClientTokenRingProviderTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cluster/CassandraClientTokenRingProviderTest.java @@ -19,6 +19,7 @@ package org.apache.cassandra.sidecar.cluster; +import java.lang.reflect.Field; import java.math.BigInteger; import java.net.InetAddress; import java.net.UnknownHostException; @@ -41,10 +42,11 @@ import com.datastax.driver.core.Metadata; import com.datastax.driver.core.ProtocolVersion; import com.datastax.driver.core.Token; -import com.datastax.driver.core.TokenRange; + import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.common.server.cluster.locator.Partitioners; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; import org.apache.cassandra.sidecar.common.server.dns.DnsResolver; import org.apache.cassandra.sidecar.coordination.CassandraClientTokenRingProvider; @@ -143,46 +145,65 @@ public String reverseResolve(String s) .collect(Collectors.toList()); for (int i = 0; i < tokens.size(); i++) { - TokenRange tokenRange = mock(TokenRange.class, RETURNS_DEEP_STUBS); Token end = tokens.get(i); - Token start = dc.equals("DC1") ? tokens.get((i - 1 + tokens.size()) % tokens.size()) : ((MockToken) end).prev(); - when(tokenRange.getStart()).thenAnswer(invocation -> start); - when(tokenRange.getEnd()).thenAnswer(invocation -> end); - when(tokenRange.isWrappedAround()).thenReturn(start.compareTo(end) < 0); + Token start; + if (dc.equals("DC1")) + { + start = tokens.get((i - 1 + tokens.size()) % tokens.size()); + } + else + { + // Handle the special case where prev() would fail for MIN_VALUE + MockToken mockEnd = (MockToken) end; + if (mockEnd.token == Long.MIN_VALUE) + { + // For MIN_VALUE, wrap around to MAX_VALUE + start = new MockToken(Long.MAX_VALUE); + } + else + { + start = mockEnd.prev(); + } + } + + // Create TokenRange with reflection-based mocking to handle final field + TokenRange tokenRange = createMockTokenRange(start, end); result.add(tokenRange); } } when(metadata.getTokenRanges()).thenAnswer(invocation -> result); - Map>>> tokens = CassandraClientTokenRingProvider.assignedRangesOfAllInstancesByDc(dnsResolver, metadata); + Map>> tokens = CassandraClientTokenRingProvider.assignedRangesOfAllInstancesByDc(dnsResolver, metadata); assertFalse(tokens.isEmpty()); assertTrue(tokens.containsKey("DC1")); assertTrue(tokens.containsKey("DC2")); // DC1 should have zero '1-range' token ranges. - List> dc1Ranges = tokens.get("DC1").values().stream().flatMap(Collection::stream).collect(Collectors.toList()); + List dc1Ranges = tokens.get("DC1").values().stream().flatMap(Collection::stream).collect(Collectors.toList()); assertTrue(dc1Ranges.stream().allMatch(range -> { - if (range.lowerEndpoint().compareTo(BigInteger.valueOf(Long.parseLong("-9223372036854775808"))) == 0 && - range.lowerEndpoint().compareTo(range.upperEndpoint()) == 0) + if (range.range.lowerEndpoint().toBigInteger().compareTo(BigInteger.valueOf(Long.parseLong("-9223372036854775808"))) == 0 && + range.range.lowerEndpoint().compareTo(range.range.upperEndpoint()) == 0) { return true; } - return range.upperEndpoint().subtract(range.lowerEndpoint()).abs().compareTo(BigInteger.ONE) > 0; + return range.range.upperEndpoint().toBigInteger().subtract(range.range.lowerEndpoint().toBigInteger()).abs().compareTo(BigInteger.ONE) > 0; })); // DC2 is offset by 1 token so there will be 1 '1-range' token range at minToken - List> dc2Ranges = tokens.get("DC2").values().stream().flatMap(Collection::stream).collect(Collectors.toList()); - List> oneTokenRanges = dc2Ranges.stream().filter(range -> - range.upperEndpoint() - .subtract(range.lowerEndpoint()) + List dc2Ranges = tokens.get("DC2").values().stream().flatMap(Collection::stream).collect(Collectors.toList()); + List oneTokenRanges = dc2Ranges.stream().filter(range -> + range.range.upperEndpoint().toBigInteger() + .subtract(range.range.lowerEndpoint().toBigInteger()) .abs() .compareTo(BigInteger.ONE) <= 0) .collect(Collectors.toList()); assertEquals(1, oneTokenRanges.size()); - assertEquals(Range.openClosed(BigInteger.valueOf(Long.MIN_VALUE), BigInteger.valueOf(Long.MIN_VALUE + 1)), oneTokenRanges.get(0)); + TokenRange oneTokenRange = oneTokenRanges.get(0); + assertEquals(BigInteger.valueOf(Long.MIN_VALUE), oneTokenRange.range.lowerEndpoint().toBigInteger()); + assertEquals(BigInteger.valueOf(Long.MIN_VALUE + 1), oneTokenRange.range.upperEndpoint().toBigInteger()); assertTrue(dc2Ranges.stream().filter(f -> f != oneTokenRanges.get(0)) - .allMatch(range -> range.upperEndpoint() - .subtract(range.lowerEndpoint()) + .allMatch(range -> range.range.upperEndpoint().toBigInteger() + .subtract(range.range.lowerEndpoint().toBigInteger()) .abs() .compareTo(BigInteger.ONE) > 0)); } @@ -206,6 +227,61 @@ public void testLocalInstances() assertEquals(3, localInstances.size()); } + /** + * Creates a properly mocked TokenRange using reflection to set the final range field. + * This is necessary because TokenRange.range is a final field that can't be mocked normally. + */ + private static TokenRange createMockTokenRange(Token start, Token end) + { + try + { + // Convert Datastax tokens to sidecar tokens + org.apache.cassandra.sidecar.common.server.cluster.locator.Token sidecarStart = + org.apache.cassandra.sidecar.common.server.cluster.locator.Token.from(((MockToken) start).token); + org.apache.cassandra.sidecar.common.server.cluster.locator.Token sidecarEnd = + org.apache.cassandra.sidecar.common.server.cluster.locator.Token.from(((MockToken) end).token); + + // Create mock sidecar tokens with proper behavior + org.apache.cassandra.sidecar.common.server.cluster.locator.Token mockSidecarStart = + mock(org.apache.cassandra.sidecar.common.server.cluster.locator.Token.class); + org.apache.cassandra.sidecar.common.server.cluster.locator.Token mockSidecarEnd = + mock(org.apache.cassandra.sidecar.common.server.cluster.locator.Token.class); + + // Configure mock token behavior for test assertions + when(mockSidecarStart.toBigInteger()).thenReturn(sidecarStart.toBigInteger()); + when(mockSidecarEnd.toBigInteger()).thenReturn(sidecarEnd.toBigInteger()); + when(mockSidecarStart.compareTo(mockSidecarEnd)).thenReturn(sidecarStart.compareTo(sidecarEnd)); + when(mockSidecarStart.compareTo(any())).thenAnswer(invocation -> { + org.apache.cassandra.sidecar.common.server.cluster.locator.Token other = invocation.getArgument(0); + return sidecarStart.toBigInteger().compareTo(other.toBigInteger()); + }); + when(mockSidecarEnd.compareTo(any())).thenAnswer(invocation -> { + org.apache.cassandra.sidecar.common.server.cluster.locator.Token other = invocation.getArgument(0); + return sidecarEnd.toBigInteger().compareTo(other.toBigInteger()); + }); + + // Create mock Range with proper endpoints + @SuppressWarnings("unchecked") + Range mockRange = + (Range) mock(Range.class); + + when(mockRange.lowerEndpoint()).thenReturn(mockSidecarStart); + when(mockRange.upperEndpoint()).thenReturn(mockSidecarEnd); + + // Create TokenRange instance and use reflection to set the final range field + TokenRange tokenRange = mock(TokenRange.class); + Field rangeField = TokenRange.class.getDeclaredField("range"); + rangeField.setAccessible(true); + rangeField.set(tokenRange, mockRange); + + return tokenRange; + } + catch (Exception e) + { + throw new RuntimeException("Failed to create mock TokenRange with reflection", e); + } + } + protected DnsResolver mockDnsResolver() { diff --git a/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java b/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java index f0ee7b732..bba3acd53 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java @@ -47,7 +47,7 @@ import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; import org.apache.cassandra.sidecar.db.schema.CdcStatesSchema; -import org.apache.cassandra.sidecar.db.schema.TableHistorySchema; +import org.apache.cassandra.sidecar.db.schema.SidecarSchema; import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; import org.apache.cassandra.sidecar.utils.TokenSplitUtil; import org.apache.cassandra.spark.data.partitioner.Partitioner; @@ -134,18 +134,18 @@ void testShrink(int numNodes) Partitioner partitioner = Partitioner.Murmur3Partitioner; MockCdcStateV2 datastore = new MockCdcStateV2(); String jobId = UUID.randomUUID().toString(); + SidecarSchema mockSidecarSchema = mock(SidecarSchema.class, RETURNS_DEEP_STUBS); CdcStatesSchema mockCdcStatesSchema = mock(CdcStatesSchema.class, RETURNS_DEEP_STUBS); - TableHistorySchema mockTableHistorySchema = mock(TableHistorySchema.class, RETURNS_DEEP_STUBS); List tokensBeforeShrink = TokenSplitUtil.splitTokens(numNodes, partitioner); List tokensAfterShrink = TokenSplitUtil.splitTokens(numNodes / 2, partitioner); TokenSplitUtil tokenSplitUtil = new TokenSplitUtil(numNodes); Provider tokenSplitUtilProvider = () -> tokenSplitUtil; - CdcDatabaseAccessor db = new CdcDatabaseAccessor(mockCdcStatesSchema, - mockTableHistorySchema, + CdcDatabaseAccessor db = new CdcDatabaseAccessor(mockSidecarSchema, getMockCQLSessionProvider(datastore, mockCdcStatesSchema), - tokenSplitUtilProvider); + tokenSplitUtilProvider, + getMockInstanceMetaDataFetcher()); ByteBuffer[] buffers = new ByteBuffer[tokensBeforeShrink.size()]; for (int i = 0; i < tokensBeforeShrink.size(); i++) @@ -183,18 +183,18 @@ void testExpand(int numNodes) Partitioner partitioner = Partitioner.Murmur3Partitioner; MockCdcStateV2 datastore = new MockCdcStateV2(); String jobId = UUID.randomUUID().toString(); + SidecarSchema mockSidecarSchema = mock(SidecarSchema.class, RETURNS_DEEP_STUBS); CdcStatesSchema mockCdcStatesSchema = mock(CdcStatesSchema.class, RETURNS_DEEP_STUBS); - TableHistorySchema mockTableHistorySchema = mock(TableHistorySchema.class, RETURNS_DEEP_STUBS); List tokensBeforeExpansion = TokenSplitUtil.splitTokens(numNodes, partitioner); List tokensAfterExpansion = TokenSplitUtil.splitTokens(numNodes * 2, partitioner); TokenSplitUtil tokenSplitUtil = new TokenSplitUtil(numNodes); Provider tokenSplitUtilProvider = () -> tokenSplitUtil; - CdcDatabaseAccessor db = new CdcDatabaseAccessor(mockCdcStatesSchema, - mockTableHistorySchema, + CdcDatabaseAccessor db = new CdcDatabaseAccessor(mockSidecarSchema, getMockCQLSessionProvider(datastore, mockCdcStatesSchema), - tokenSplitUtilProvider); + tokenSplitUtilProvider, + getMockInstanceMetaDataFetcher()); ByteBuffer[] buffers = new ByteBuffer[tokensBeforeExpansion.size()]; for (int i = 0; i < tokensBeforeExpansion.size(); i++) From dc9ef3bc291bcdd0775e289256a9499ef9a628de Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Mon, 1 Dec 2025 09:05:17 -0800 Subject: [PATCH 02/20] Revert not needed changes --- conf/sidecar.yaml | 20 ++-- gradle.properties | 2 +- .../sidecar/cdc/CachingSchemaStore.java | 6 - .../cassandra/sidecar/cdc/CdcConfigImpl.java | 22 ---- .../cassandra/sidecar/cdc/CdcManager.java | 2 - .../config/yaml/CdcConfigurationImpl.java | 2 +- .../yaml/SchemaKeyspaceConfigurationImpl.java | 4 +- .../sidecar/handlers/AbstractHandler.java | 4 +- .../sidecar/tasks/ClusterTopologyMonitor.java | 4 - .../sidecar/cdc/CdcConfigImplTest.java | 111 +++--------------- 10 files changed, 30 insertions(+), 147 deletions(-) diff --git a/conf/sidecar.yaml b/conf/sidecar.yaml index d270bf277..645aee33c 100644 --- a/conf/sidecar.yaml +++ b/conf/sidecar.yaml @@ -21,7 +21,7 @@ # cassandra_instances: - id: 1 - host: localhost + host: localhost1 port: 9042 # The instance's storage directory as defined per the cassandra.storagedir property @@ -42,13 +42,11 @@ cassandra_instances: # Directory where Cassandra stores mutations. If not set, the default directory is # /cdc_raw. - cdc_dir: /Users/bernardobotellacorbi/Documents/dev/cassandra_trunk/data/cdc_raw -# cdc_dir: /var/lib/cassandra/cdc_raw + cdc_dir: /var/lib/cassandra/cdc_raw # Directory where Cassandra stores the commit logs. If not set, the default directory # is /commitlog - commitlog_dir: /Users/bernardobotellacorbi/Documents/dev/cassandra_trunk/data/commitlog -# commitlog_dir: /var/lib/cassandra/commitlog + commitlog_dir: /var/lib/cassandra/commitlog # Directory where Cassandra stores hints. If not set, the default directory is # /hints. @@ -63,14 +61,14 @@ cassandra_instances: # first of the data directories specified by data_file_directories. # local_system_data_file_dir: - jmx_host: localhost + jmx_host: 127.0.0.1 jmx_port: 7199 jmx_ssl_enabled: false # jmx_role: # jmx_role_password: sidecar: - host: localhost + host: 0.0.0.0 port: 9043 request_idle_timeout: 5m request_timeout: 5m @@ -123,7 +121,7 @@ sidecar: max_retries: 3 retry_delay: 200ms schema: - is_enabled: true + is_enabled: false keyspace: sidecar_internal replication_strategy: SimpleStrategy replication_factor: 1 @@ -304,7 +302,7 @@ access_control: driver_parameters: contact_points: - - "localhost:9042" + - "127.0.0.1:9042" username: cassandra password: cassandra ssl: @@ -350,8 +348,8 @@ sidecar_client: #max_retries: 5 # The amount of retries the client will attempt a request #retry_delay: 500ms # The initial delay between the retries the client will attempt a request #max_retry_delay: 10s # The max delay between the retries the client will attempt a request - ssl: # if ssl is enabled, this is the ssl configuration used for the sidecar client - enabled: false + #ssl: # if ssl is enabled, this is the ssl configuration used for the sidecar client + # enabled: false # keystore: # type: PKCS12 # Keystore type # path: path/to/client/keystore.p12 # Path to the client keystore file diff --git a/gradle.properties b/gradle.properties index e731b195d..57f87c4d5 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ # limitations under the License. # -version=0.3.99 +version=0.3.0 junitVersion=5.9.2 vertxVersion=4.5.23 nettyVersion=4.1.118.Final diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index 911d5c846..3612a7e3e 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -76,12 +76,6 @@ public class CachingSchemaStore implements SchemaStore this.cdcConfig = cdcConfig; this.sidecarCdcStats = sidecarCdcStats; - ThrowingRunnable configChangeCallback = () -> { - LOGGER.info("Services configuration changed. Reloading publisher..."); - loadPublisher(); - publishSchemas(); - }; - this.cdcConfig.registerConfigChangeListener(configChangeCallback); configureSidecarServerEventListeners(); } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java index c8099574c..1d8dd770a 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java @@ -22,7 +22,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.google.inject.Singleton; @@ -30,7 +29,6 @@ import io.vertx.core.Vertx; import io.vertx.core.eventbus.Message; -import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; import org.apache.cassandra.sidecar.common.server.utils.SecondBoundConfiguration; import org.apache.cassandra.sidecar.db.CdcConfigAccessor; @@ -230,31 +228,11 @@ protected int getInt(String key, Supplier orDefault) return aInt != null ? Integer.valueOf(aInt) : orDefault.get(); } - /** - * Adds a listener to service config changes - * - * @param listener The listener to call - */ - public void registerConfigChangeListener(ThrowingRunnable listener) - { -// this.configRefreshNotifier.registerConfigChangeListener(listener); - } - private Map getAuthConfigs() { return new HashMap<>(); } - @VisibleForTesting - void forceExecuteNotifier() - { -// if (configRefreshNotifier != null && -// configRefreshNotifier.scheduleDecision() == ScheduleDecision.EXECUTE) -// { -// configRefreshNotifier.execute(Promise.promise()); -// } - } - enum ConfigKeys { DATACENTER, diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index 369fed1e7..822952fdf 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -133,7 +133,6 @@ List buildCdcConsumers() return consumers; } - SidecarCdc loadOrBuildCdcConsumer(Integer instanceId, ClusterConfigProvider clusterConfigProvider, EventConsumer eventConsumer, @@ -171,7 +170,6 @@ public void stopConsumers() consumers.forEach(SidecarCdc::stop); } - private Integer getInstanceId(String instanceIp) { for (InstanceMetadata instance : instanceFetcher.allLocalInstances()) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java index 79f56afe8..8988ff02c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/CdcConfigurationImpl.java @@ -37,7 +37,7 @@ public class CdcConfigurationImpl implements CdcConfiguration public static final String CONFIGURATION_REFRESH_TIME_PROPERTY = "config_refresh_time"; public static final String TABLE_SCHEMA_REFRESH_TIME_PROPERTY = "table_schema_refresh_time"; public static final String SEGMENT_HARD_LINK_CACHE_EXPIRY_PROPERTY = "segment_hardlink_cache_expiry"; - public static final boolean DEFAULT_IS_ENABLED = true; + public static final boolean DEFAULT_IS_ENABLED = false; public static final MillisecondBoundConfiguration DEFAULT_CDC_CONFIG_REFRESH_TIME = MillisecondBoundConfiguration.parse("30s"); public static final SecondBoundConfiguration DEFAULT_TABLE_SCHEMA_REFRESH_TIME = diff --git a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java index 867ccbb46..31c0ffaa2 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SchemaKeyspaceConfigurationImpl.java @@ -35,7 +35,7 @@ public class SchemaKeyspaceConfigurationImpl implements SchemaKeyspaceConfiguration { private static final Logger LOGGER = LoggerFactory.getLogger(SchemaKeyspaceConfigurationImpl.class); - public static final boolean DEFAULT_IS_ENABLED = true; + public static final boolean DEFAULT_IS_ENABLED = false; public static final String DEFAULT_KEYSPACE = "sidecar_internal"; public static final String DEFAULT_REPLICATION_STRATEGY = "SimpleStrategy"; public static final int DEFAULT_REPLICATION_FACTOR = 1; @@ -161,7 +161,7 @@ static ConfigurationException leaseTTLConfigurationException(SecondBoundConfigur */ public static class Builder implements DataObjectBuilder { - private boolean isEnabled = true; + private boolean isEnabled = DEFAULT_IS_ENABLED; private String keyspace = DEFAULT_KEYSPACE; private String replicationStrategy = DEFAULT_REPLICATION_STRATEGY; private int replicationFactor = DEFAULT_REPLICATION_FACTOR; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java b/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java index aa19c394b..7060a3b3c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/handlers/AbstractHandler.java @@ -87,8 +87,8 @@ public void handle(RoutingContext context) try { requestParams = extractParamsOrThrow(context); -// logger.debug("{} received request={}, remoteAddress={}, instance={}", -// this.getClass().getSimpleName(), requestParams, remoteAddress, host); + logger.debug("{} received request={}, remoteAddress={}, instance={}", + this.getClass().getSimpleName(), requestParams, remoteAddress, host); handleInternal(context, request, host, remoteAddress, requestParams); } catch (Exception exception) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java index c6d9afc6b..155220c69 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java @@ -152,8 +152,6 @@ public ClusterTopologyMonitor(Vertx vertx, TokenRingProvider tokenRingProvider) } } - // periodic task - /** * Returns the delay between topology refresh cycles. */ @@ -184,8 +182,6 @@ public void execute(Promise promise) } } - // internal methods - /** * Refreshes topology information for all datacenters and publishes change events. */ diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java index 952a84712..19f7a0475 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java @@ -27,25 +27,21 @@ import io.vertx.core.Promise; import io.vertx.core.Vertx; import org.apache.cassandra.sidecar.TestResourceReaper; -import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; +import org.apache.cassandra.sidecar.codecs.CdcConfigMappingsCodec; import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; import org.apache.cassandra.sidecar.common.server.utils.SecondBoundConfiguration; import org.apache.cassandra.sidecar.concurrent.ExecutorPools; -import org.apache.cassandra.sidecar.config.CdcConfiguration; -import org.apache.cassandra.sidecar.config.SchemaKeyspaceConfiguration; -import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; import org.apache.cassandra.sidecar.coordination.ClusterLease; import org.apache.cassandra.sidecar.db.CdcConfigAccessor; -import org.apache.cassandra.sidecar.db.KafkaConfigAccessor; +import org.apache.cassandra.sidecar.tasks.CdcConfigRefresherNotifierTask; import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; import static org.apache.cassandra.testing.utils.AssertionUtils.loopAssert; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; class CdcConfigImplTest @@ -78,7 +74,6 @@ static void teardown() void testIsConfigReadySchemaNotInitialized() { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.isAvailable()).thenReturn(false); CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); @@ -89,7 +84,6 @@ void testIsConfigReadySchemaNotInitialized() void testIsConfigReadyKafkaConfigsEmpty() { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("k1", "v1")); CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.isConfigReady()).isFalse(); @@ -99,7 +93,6 @@ void testIsConfigReadyKafkaConfigsEmpty() void testIsConfigReadyCdcConfigsEmpty() { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("k1", "v1")); CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.isConfigReady()).isFalse(); @@ -109,7 +102,6 @@ void testIsConfigReadyCdcConfigsEmpty() void testReturnDefaultValuesWhenConfigsAreEmpty() { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); assertThat(cdcConfig.datacenter()).isEqualTo(null); assertThat(cdcConfig.env()).isEqualTo(""); @@ -120,21 +112,22 @@ void testReturnDefaultValuesWhenConfigsAreEmpty() } @Test - void testConfigsWhenConfigsAreNotEmpty() + void testConfigsWhenConfigsAreNotEmpty() throws InterruptedException { CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); - when(cdcConfigAccessor.getConfig().getConfigs()) - .thenReturn(Map.of("datacenter", "DC1", - "env", "if", - "log_only", "false", - "topic", "topic1", - "watermark_seconds", "120", - "persist_delay_millis", "5000")); - when(kafkaConfigAccessor.getConfig().getConfigs()) - .thenReturn(Map.of("k1", "v1")); + CdcConfigRefresherNotifierTask.ConfigMappings configMappings = new CdcConfigRefresherNotifierTask.ConfigMappings(); + configMappings.setKafkaConfigMappings(Map.of("k1", "v1")); + configMappings.setCdcConfigMappings(Map.of("datacenter", "DC1", + "env", "if", + "log_only", "false", + "topic", "topic1", + "watermark_seconds", "120", + "persist_delay_millis", "5000")); CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); + vertx.eventBus().registerDefaultCodec(CdcConfigRefresherNotifierTask.ConfigMappings.class, CdcConfigMappingsCodec.INSTANCE); + vertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), configMappings); + loopAssert(5, ()-> assertThat(cdcConfig.isConfigReady()).isTrue()); assertThat(cdcConfig.datacenter()).isEqualTo("DC1"); assertThat(cdcConfig.env()).isEqualTo("if"); @@ -144,44 +137,6 @@ void testConfigsWhenConfigsAreNotEmpty() assertThat(cdcConfig.persistDelay()).isEqualTo(new MillisecondBoundConfiguration(5, TimeUnit.SECONDS)); } - @Test - void testConfigChanged() throws Exception - { - ThrowingRunnable listener = mockRunnable(); - CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); - KafkaConfigAccessor kafkaConfigAccessor = mockKafkaConfigAccessor(); - when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("dc", "DC1", - "env", "if", - "log_only", "false")); - when(kafkaConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("topic", "topic1")); - - CdcConfigImpl cdcConfig = new CdcConfigImpl(vertx, cdcConfigAccessor); - cdcConfig.registerConfigChangeListener(listener); - - // do not wait the periodic task execution, we force running it immediately. - cdcConfig.forceExecuteNotifier(); - verify(listener, times(1)).run(); - - // run the task multiple times, the listener should still be invoked only once - cdcConfig.forceExecuteNotifier(); - cdcConfig.forceExecuteNotifier(); - cdcConfig.forceExecuteNotifier(); - verify(listener, times(1)).run(); - - // update the config. The listener should be called - when(cdcConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of("dc", "DC1", - "env", "if", - "log_only", "true")); - cdcConfig.forceExecuteNotifier(); - verify(listener, times(2)).run(); - - // run the task multiple times, the listener should not be called since no more changes are made - cdcConfig.forceExecuteNotifier(); - cdcConfig.forceExecuteNotifier(); - cdcConfig.forceExecuteNotifier(); - verify(listener, times(2)).run(); - } - private CdcConfigAccessor mockCdcConfigAccessor() { CdcConfigAccessor cdcConfigAccessor = mock(CdcConfigAccessor.class, RETURNS_DEEP_STUBS); @@ -189,40 +144,4 @@ private CdcConfigAccessor mockCdcConfigAccessor() when(cdcConfigAccessor.isAvailable()).thenReturn(true); return cdcConfigAccessor; } - - private KafkaConfigAccessor mockKafkaConfigAccessor() - { - KafkaConfigAccessor kafkaConfigAccessor = mock(KafkaConfigAccessor.class, RETURNS_DEEP_STUBS); - when(kafkaConfigAccessor.getConfig().getConfigs()).thenReturn(Map.of()); - when(kafkaConfigAccessor.isAvailable()).thenReturn(true); - return kafkaConfigAccessor; - } - - private CdcConfiguration mockCdcConfiguration() - { - CdcConfiguration cdcConfiguration = mock(CdcConfiguration.class); - when(cdcConfiguration.isEnabled()).thenReturn(true); - when(cdcConfiguration.cdcConfigRefreshTime()).thenReturn(MillisecondBoundConfiguration.parse("1s")); - return cdcConfiguration; - } - - private SchemaKeyspaceConfiguration mockSchemaKeyspaceConfiguration() - { - SchemaKeyspaceConfiguration schemaKeyspaceConfiguration = mock(SchemaKeyspaceConfiguration.class); - when(schemaKeyspaceConfiguration.isEnabled()).thenReturn(true); - return schemaKeyspaceConfiguration; - } - - private SidecarConfiguration mockSidecarConfiguration() - { - SidecarConfiguration sidecarConfiguration = mock(SidecarConfiguration.class, RETURNS_DEEP_STUBS); - when(sidecarConfiguration.serviceConfiguration().cdcConfiguration()).thenAnswer(invocation -> mockCdcConfiguration()); - when(sidecarConfiguration.serviceConfiguration().schemaKeyspaceConfiguration()).thenAnswer(invocation -> mockSchemaKeyspaceConfiguration()); - return sidecarConfiguration; - } - - private ThrowingRunnable mockRunnable() - { - return mock(ThrowingRunnable.class); - } } From ba0641a9205296d6d0c9855df38df85ce1b85e60 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Mon, 1 Dec 2025 09:58:22 -0800 Subject: [PATCH 03/20] Fix RAT --- .../sidecar/cdc/CachingSchemaStore.java | 18 ++++++++++++++++++ .../sidecar/cdc/CdcAvroSerializer.java | 18 ++++++++++++++++++ .../sidecar/cdc/CdcEventConsumer.java | 18 ++++++++++++++++++ .../cassandra/sidecar/cdc/CdcManager.java | 18 ++++++++++++++++++ .../cassandra/sidecar/cdc/CdcPublisher.java | 18 ++++++++++++++++++ .../sidecar/cdc/CdcSchemaSupplier.java | 18 ++++++++++++++++++ .../sidecar/cdc/ExecutorPoolsExecutor.java | 18 ++++++++++++++++++ .../sidecar/cdc/SidecarCdcOptions.java | 18 ++++++++++++++++++ .../cdc/SidecarCqlToAvroSchemaConverter.java | 18 ++++++++++++++++++ .../cdc/StateSidecarCdcCassandraClient.java | 18 ++++++++++++++++++ .../sidecar/codecs/BigIntegerCodec.java | 18 ++++++++++++++++++ .../sidecar/codecs/CdcConfigMappingsCodec.java | 18 ++++++++++++++++++ .../cassandra/sidecar/codecs/CommonCodecs.java | 18 ++++++++++++++++++ .../DcLocalTopologyChangeEventCodec.java | 18 ++++++++++++++++++ .../sidecar/codecs/RangeChangeEventCodec.java | 18 ++++++++++++++++++ .../cassandra/sidecar/codecs/RangeCodec.java | 18 ++++++++++++++++++ .../sidecar/codecs/RangeMapCodec.java | 18 ++++++++++++++++++ .../ContentionFreeRangeManager.java | 18 ++++++++++++++++++ .../sidecar/coordination/RangeManager.java | 18 ++++++++++++++++++ .../db/VirtualTablesDatabaseAccessor.java | 18 ++++++++++++++++++ .../tasks/CdcConfigRefresherNotifierTask.java | 18 ++++++++++++++++++ .../sidecar/tasks/ClusterTopologyMonitor.java | 18 ++++++++++++++++++ .../sidecar/cdc/CachingSchemaStoreTest.java | 18 ++++++++++++++++++ .../cassandra/sidecar/cdc/CdcManagerTest.java | 18 ++++++++++++++++++ .../sidecar/cdc/CdcPublisherTests.java | 18 ++++++++++++++++++ .../sidecar/cdc/CdcSchemaSupplierTest.java | 18 ++++++++++++++++++ 26 files changed, 468 insertions(+) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index 3612a7e3e..955cdc958 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.nio.charset.StandardCharsets; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java index fa6c7b06d..e73929265 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import org.apache.cassandra.bridge.CassandraBridgeFactory; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java index 87c375a40..f79f46379 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcEventConsumer.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.function.Consumer; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index 822952fdf..dd9515cab 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.io.IOException; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index 4c137f039..bc959e2eb 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.HashMap; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java index 4da13f618..c22fb00e1 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Map; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java index d15ef6251..ae9e7e3f9 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/ExecutorPoolsExecutor.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.concurrent.CompletableFuture; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java index 9cb5a31db..a29687199 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCdcOptions.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Map; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java index f5f17df54..2f07c0be1 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarCqlToAvroSchemaConverter.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java index 9442cd6ed..76d56e52e 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/StateSidecarCdcCassandraClient.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.nio.ByteBuffer; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java index 9f32c7beb..491cda4b6 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import java.math.BigInteger; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java index ca5e30f64..a44d7dc43 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CdcConfigMappingsCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import java.util.HashMap; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java index cca4a408b..d0dcf3edb 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import io.vertx.core.eventbus.impl.codecs.BooleanMessageCodec; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java index 0be449549..775392d87 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/DcLocalTopologyChangeEventCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import java.util.ArrayList; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java index 4d9bcea7c..0987473fd 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeChangeEventCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import org.apache.commons.lang3.mutable.MutableInt; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java index 58e98431d..9bbc7f800 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import java.math.BigInteger; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java index 2369369ee..ae8791be0 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/RangeMapCodec.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.codecs; import java.util.HashMap; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java index 4e8a7ad84..33035d280 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/ContentionFreeRangeManager.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.coordination; import java.util.Map; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java index 64101de20..0a9dff302 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.coordination; import java.util.HashMap; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java index 614f6049b..c814aaaea 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/VirtualTablesDatabaseAccessor.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.db; import com.datastax.driver.core.Row; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java index da4049cb4..7f261015c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.tasks; import java.util.ArrayList; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java index 155220c69..a088d90e6 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.tasks; import java.util.ArrayList; diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java index b12c559af..c11564693 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStoreTest.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Collections; diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java index e518fa6bc..d4f1cb187 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.io.IOException; diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java index 331e57a60..9c04c011e 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Arrays; diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java index 8d2401f5c..b5b7d02a7 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplierTest.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Set; From b1214c135a22842dea5edfda296ff41c97064ff1 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Mon, 1 Dec 2025 10:16:56 -0800 Subject: [PATCH 04/20] Fix checkstyle --- .../org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index 955cdc958..31b81f654 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -45,7 +45,6 @@ import org.apache.cassandra.cdc.schemastore.SchemaStore; import org.apache.cassandra.cdc.schemastore.SchemaStorePublisherFactory; import org.apache.cassandra.cdc.schemastore.TableSchemaPublisher; -import org.apache.cassandra.sidecar.common.server.ThrowingRunnable; import org.apache.cassandra.sidecar.db.TableHistoryDatabaseAccessor; import org.apache.cassandra.sidecar.db.schema.SidecarSchema; import org.apache.cassandra.sidecar.tasks.CassandraClusterSchemaMonitor; From 670619a7696826590fc8a79defba293809b72e02 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Mon, 1 Dec 2025 11:14:20 -0800 Subject: [PATCH 05/20] Fix tests --- .../sidecar/db/CdcDatabaseAccessorTests.java | 6 ++++-- .../sidecar/db/SidecarSchemaTest.java | 18 ++++++++++++++++-- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java b/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java index bba3acd53..2db66cb24 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessorTests.java @@ -134,8 +134,9 @@ void testShrink(int numNodes) Partitioner partitioner = Partitioner.Murmur3Partitioner; MockCdcStateV2 datastore = new MockCdcStateV2(); String jobId = UUID.randomUUID().toString(); - SidecarSchema mockSidecarSchema = mock(SidecarSchema.class, RETURNS_DEEP_STUBS); + SidecarSchema mockSidecarSchema = mock(SidecarSchema.class); CdcStatesSchema mockCdcStatesSchema = mock(CdcStatesSchema.class, RETURNS_DEEP_STUBS); + when(mockSidecarSchema.tableSchema(CdcStatesSchema.class)).thenReturn(mockCdcStatesSchema); List tokensBeforeShrink = TokenSplitUtil.splitTokens(numNodes, partitioner); List tokensAfterShrink = TokenSplitUtil.splitTokens(numNodes / 2, partitioner); TokenSplitUtil tokenSplitUtil = new TokenSplitUtil(numNodes); @@ -183,8 +184,9 @@ void testExpand(int numNodes) Partitioner partitioner = Partitioner.Murmur3Partitioner; MockCdcStateV2 datastore = new MockCdcStateV2(); String jobId = UUID.randomUUID().toString(); - SidecarSchema mockSidecarSchema = mock(SidecarSchema.class, RETURNS_DEEP_STUBS); + SidecarSchema mockSidecarSchema = mock(SidecarSchema.class); CdcStatesSchema mockCdcStatesSchema = mock(CdcStatesSchema.class, RETURNS_DEEP_STUBS); + when(mockSidecarSchema.tableSchema(CdcStatesSchema.class)).thenReturn(mockCdcStatesSchema); List tokensBeforeExpansion = TokenSplitUtil.splitTokens(numNodes, partitioner); List tokensAfterExpansion = TokenSplitUtil.splitTokens(numNodes * 2, partitioner); TokenSplitUtil tokenSplitUtil = new TokenSplitUtil(numNodes); diff --git a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java index 8e21606f9..62a0cde6b 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java @@ -133,7 +133,7 @@ void testSchemaInitOnStartup() { sidecarSchemaInitializer.execute(Promise.promise()); loopAssert(10, 500, () -> { - assertThat(interceptedExecStmts.size()).isEqualTo(7); + assertThat(interceptedExecStmts.size()).isEqualTo(9); assertThat(interceptedExecStmts.get(0)).as("Create keyspace should be executed the first") .contains("CREATE KEYSPACE IF NOT EXISTS sidecar_internal"); assertThat(interceptedExecStmts).as("Create table should be executed for job table") @@ -144,6 +144,14 @@ void testSchemaInitOnStartup() .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.restore_range_v1")); assertThat(interceptedExecStmts).as("Create table should be executed for role_permissions_v1 table") .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.role_permissions_v1")); + assertThat(interceptedExecStmts).as("Create table should be executed for table_schema_history table") + .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.table_schema_history")); + assertThat(interceptedExecStmts).as("Create table should be executed for cdc_state_v2 table") + .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.cdc_state_v2")); + assertThat(interceptedExecStmts).as("Create table should be executed for configs table") + .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.configs")); + assertThat(interceptedExecStmts).as("Create table should be executed for sidecar_lease_v1 table") + .anyMatch(stmt -> stmt.contains("CREATE TABLE IF NOT EXISTS sidecar_internal.sidecar_lease_v1")); List expectedPrepStatements = Arrays.asList( "INSERT INTO sidecar_internal.restore_job_v5 ( created_at, job_id, keyspace_name, table_name, " + @@ -211,7 +219,13 @@ void testSchemaInitOnStartup() "SELECT * FROM system_views.clients", - "SELECT username, COUNT(*) AS connection_count FROM system_views.clients" + "SELECT username, COUNT(*) AS connection_count FROM system_views.clients", + + "INSERT INTO sidecar_internal.table_schema_history (keyspace_name, table_name, version, created_at, table_schema) VALUES (?, ?, ?, NOW(), ?)", + "SELECT table_schema FROM sidecar_internal.table_schema_history WHERE keyspace_name = ? AND table_name = ? AND version = ?", + + "INSERT INTO sidecar_internal.cdc_state_v2 (job_id, split, start, end, state) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP ?", + "SELECT start, end, state FROM sidecar_internal.cdc_state_v2 WHERE job_id = ? AND split = ?" ); assertThat(interceptedPrepStmts).as("Intercepted statements match expected statements") From b94ccc255834b4054776aaa9de83bae7e594ce89 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Mon, 1 Dec 2025 14:21:49 -0800 Subject: [PATCH 06/20] Add readme info --- README.md | 202 ++++++++++++++++++ .../sidecar/db/SidecarSchemaTest.java | 2 +- 2 files changed, 203 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index cfad18f26..7772b1fa4 100644 --- a/README.md +++ b/README.md @@ -84,6 +84,208 @@ Configuring Cassandra Instance While setting up cassandra instance, make sure the data directories of cassandra are in the path stored in sidecar.yaml file, else modify data directories path to point to the correct directories for stream APIs to work. +Change Data Capture (CDC) Configuration +--------------------------------------- + +Apache Cassandra Sidecar supports Change Data Capture (CDC) to stream table mutations to Apache Kafka. This section describes how to configure and run Sidecar with CDC enabled. + +### Prerequisites + +1. Apache Cassandra 4.0+ with CDC support +2. Apache Kafka cluster +3. Sidecar configured with schema management enabled + +### Configuration Steps + +#### 1. Enable CDC in Cassandra + +Edit your `cassandra.yaml` configuration file and enable CDC: + +```yaml +cdc_enabled: true +``` + +Restart your Cassandra instance for this change to take effect. + +#### 2. Configure Sidecar for CDC + +Edit your `sidecar.yaml` configuration file with the following settings: + +```yaml +sidecar: + # Enable schema management (required for CDC) + schema: + is_enabled: true + keyspace: sidecar_internal + replication_strategy: SimpleStrategy + replication_factor: 3 + + # Enable CDC feature + cdc: + enabled: true + config_refresh_time: 10s + table_schema_refresh_time: 60s + segment_hardlink_cache_expiry: 1m +``` + +**Configuration Parameters:** +- `schema.is_enabled`: **Must be `true`** for CDC to function. Creates the `sidecar_internal` keyspace for CDC state management. +- `cdc.enabled`: Enables the CDC feature in Sidecar. +- `cdc.config_refresh_time`: How frequently CDC configuration is refreshed from the database. +- `cdc.table_schema_refresh_time`: How frequently table schemas are refreshed for CDC-enabled tables. +- `cdc.segment_hardlink_cache_expiry`: Cache expiration time for CDC segment hard links. + +#### 3. Enable CDC on Tables + +For each table you want to capture changes from, enable the CDC property using CQL: + +```cql +-- For a new table +CREATE TABLE my_keyspace.my_table ( + id text PRIMARY KEY, + name text, + value int +) WITH cdc = true; + +-- For an existing table +ALTER TABLE my_keyspace.my_table WITH cdc = true; +``` + +#### 4. Configure CDC Service + +Use the CDC configuration API endpoint to set up CDC parameters: + +```bash +curl --request PUT \ + --url http://localhost:9043/api/v1/services/cdc/config \ + --header 'content-type: application/json' \ + --data '{ + "config": { + "datacenter": "datacenter1", + "env": "production", + "topic_format_type": "STATIC", + "topic": "cdc-events" + } +}' +``` + +**CDC Configuration Parameters:** +- `datacenter`: The datacenter name for this Sidecar instance. +- `env`: Environment identifier (e.g., `production`, `staging`, `dev`). +- `topic_format_type`: Determines how Kafka topic names are generated. Options: + - `STATIC`: Use a single fixed topic name specified in `topic` field + - `KEYSPACE`: Format as `{topic}-{keyspace}` + - `KEYSPACETABLE`: Format as `{topic}-{keyspace}-{table}` + - `TABLE`: Format as `{topic}-{table}` + - `MAP`: Use custom topic mapping (advanced) +- `topic`: Base Kafka topic name for CDC events. + +#### 5. Configure Kafka Producer + +Configure the Kafka producer settings using the Kafka configuration API endpoint: + +```bash +curl --request PUT \ + --url http://localhost:9043/api/v1/services/kafka/config \ + --header 'content-type: application/json' \ + --data '{ + "config": { + "bootstrap.servers": "localhost:9092", + "key.serializer": "org.apache.kafka.common.serialization.StringSerializer", + "value.serializer": "org.apache.kafka.common.serialization.ByteArraySerializer", + "acks": "all", + "retries": "3", + "retry.backoff.ms": "200", + "enable.idempotence": "true", + "batch.size": "16384", + "linger.ms": "5", + "buffer.memory": "33554432", + "compression.type": "snappy", + "request.timeout.ms": "30000", + "delivery.timeout.ms": "120000", + "max.in.flight.requests.per.connection": "5", + "client.id": "cdc-producer" + } +}' +``` + +**Key Kafka Producer Parameters:** +- `bootstrap.servers`: Comma-separated list of Kafka broker addresses. +- `key.serializer`: Serializer for the message key (use `StringSerializer`). +- `value.serializer`: Serializer for the message value (use `ByteArraySerializer` for Avro). +- `acks`: Number of acknowledgments the producer requires (`all` for maximum durability). +- `enable.idempotence`: Ensures exactly-once semantics when set to `true`. +- `compression.type`: Compression algorithm (`snappy`, `gzip`, `lz4`, `zstd`, or `none`). + +For a complete list of Kafka producer configurations, see the [Apache Kafka Producer Configuration Documentation](https://kafka.apache.org/documentation/#producerconfigs). + +### Data Format and Serialization + +CDC events are serialized in **Apache Avro** format. Sidecar includes a built-in schema store (`CachingSchemaStore`) that: +- Automatically tracks CDC-enabled table schemas +- Converts CQL schemas to Avro schemas +- Refreshes schemas based on `table_schema_refresh_time` configuration +- Caches Avro schemas for performance + +Each CDC event published to Kafka contains: +- **Key**: Table identifier (keyspace + table name) +- **Value**: Avro-serialized mutation data containing: + - Partition key + - Clustering key (if applicable) + - Mutation type (INSERT, UPDATE, DELETE) + - Column values + - Timestamp + +### Verification + +After completing the configuration: + +1. **Check Sidecar Logs**: Verify CDC is enabled and connected to Kafka: + ``` + grep -i "cdc" /path/to/sidecar.log + ``` + +2. **Verify Configuration**: Retrieve current CDC and Kafka configurations: + ```bash + # Get CDC configuration + curl http://localhost:9043/api/v1/services/cdc/config + + # Get Kafka configuration + curl http://localhost:9043/api/v1/services/kafka/config + + # Get all service configurations + curl http://localhost:9043/api/v1/services + ``` + +### Advanced Configuration + +#### Custom Schema Registry Integration + +While Sidecar includes a built-in schema store, you can integrate with external schema registries by: +1. Implementing a custom `SchemaStore` interface +2. Registering your implementation via Guice dependency injection +3. Configuring your schema registry connection details in the Kafka producer configuration + + +### Troubleshooting + +**CDC not starting:** +- Verify `schema.is_enabled: true` in `sidecar.yaml` +- Check Cassandra has `cdc_enabled: true` +- Ensure `sidecar_internal` keyspace exists and is accessible + +**No messages in Kafka:** +- Verify tables have `cdc = true` property +- Check Kafka connectivity and broker availability +- Review Sidecar logs for errors: `grep -i "kafka\|cdc" /path/to/sidecar.log` +- Verify CDC and Kafka configurations are set via API endpoints + +**Schema errors:** +- Ensure table schemas are stable (avoid frequent schema changes during CDC) +- Check `table_schema_refresh_time` is appropriate for your use case +- Review Sidecar logs for schema conversion errors + + Testing ------- diff --git a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java index 62a0cde6b..9b6800606 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/db/SidecarSchemaTest.java @@ -223,7 +223,7 @@ void testSchemaInitOnStartup() "INSERT INTO sidecar_internal.table_schema_history (keyspace_name, table_name, version, created_at, table_schema) VALUES (?, ?, ?, NOW(), ?)", "SELECT table_schema FROM sidecar_internal.table_schema_history WHERE keyspace_name = ? AND table_name = ? AND version = ?", - + "INSERT INTO sidecar_internal.cdc_state_v2 (job_id, split, start, end, state) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP ?", "SELECT start, end, state FROM sidecar_internal.cdc_state_v2 WHERE job_id = ? AND split = ?" ); From 7c1e81cab0dff210bc9d3a944887f9b213777881 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 10 Dec 2025 14:00:05 -0800 Subject: [PATCH 07/20] Applies feedback --- .../sidecar/cdc/CachingSchemaStore.java | 26 +++----- .../sidecar/cdc/CdcAvroSerializer.java | 2 +- .../cassandra/sidecar/cdc/CdcManager.java | 21 +++++- .../cassandra/sidecar/cdc/CdcPublisher.java | 62 +++++++++--------- .../sidecar/cdc/CdcSchemaSupplier.java | 26 +++++++- .../cdc/SidecarClusterConfigProvider.java | 59 +++++++++++++++++ .../sidecar/db/CdcDatabaseAccessor.java | 34 ---------- .../db/TableHistoryDatabaseAccessor.java | 25 ------- .../cassandra/sidecar/modules/CdcModule.java | 65 +++++-------------- .../tasks/CdcConfigRefresherNotifierTask.java | 2 +- .../sidecar/tasks/ClusterTopologyMonitor.java | 5 +- .../sidecar/cdc/CdcPublisherTests.java | 7 +- 12 files changed, 171 insertions(+), 163 deletions(-) create mode 100644 server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index 31b81f654..31f8475f6 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -56,7 +56,8 @@ import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_SCHEMA_INITIALIZED; /** - * Schemas cache to be used by CDC event serialization. + * Schemas cache to be used by CDC event serialization. It contains a map of table schemas + * using TableIdentifier as key. */ @Singleton public class CachingSchemaStore implements SchemaStore @@ -68,10 +69,13 @@ public class CachingSchemaStore implements SchemaStore private final TableHistoryDatabaseAccessor tableHistoryDatabaseAccessor; private final Vertx vertx; private final CdcConfigImpl cdcConfig; - @Nullable volatile TableSchemaPublisher publisher; + @Nullable private volatile TableSchemaPublisher publisher; private final CqlToAvroSchemaConverter cqlToAvroSchemaConverter; private final SidecarCdcStats sidecarCdcStats; + private static final String METADATA_NAME_KEY = "name"; + private static final String METADATA_NAMESPACE_KEY = "namespace"; + @Inject CachingSchemaStore(Vertx vertx, CassandraClusterSchemaMonitor cassandraClusterSchemaMonitor, @@ -98,13 +102,7 @@ public class CachingSchemaStore implements SchemaStore private void loadPublisher() { - KafkaOptions kafkaOptions = new KafkaOptions() - { - public Map kafkaConfigs() - { - return cdcConfig.kafkaConfigs(); - } - }; + KafkaOptions kafkaOptions = () -> cdcConfig.kafkaConfigs(); this.publisher = SchemaStorePublisherFactory.DEFAULT.buildPublisher(kafkaOptions); } @@ -143,8 +141,8 @@ private void publishSchemas() { Schema schema = cqlToAvroSchemaConverter.convert(cqlTable); TableSchemaPublisher.SchemaPublishMetadata metadata = new TableSchemaPublisher.SchemaPublishMetadata(); - metadata.put("name", cqlTable.table()); - metadata.put("namespace", cqlTable.keyspace()); + metadata.put(METADATA_NAME_KEY, cqlTable.table()); + metadata.put(METADATA_NAMESPACE_KEY, cqlTable.keyspace()); publisher.publishSchema(schema.toString(false), metadata); sidecarCdcStats.capturePublishedSchema(); } @@ -224,12 +222,6 @@ public String getVersion(String namespace, String name) }).schemaUuid; } - public Map getSchemas() - { - return avroSchemasCache.values().stream() - .collect(Collectors.toMap(e -> e.schema.getNamespace(), e -> e.schema)); - } - private Map createSchemaCache(Set cdcTables) { diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java index e73929265..1acc7c4bd 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcAvroSerializer.java @@ -25,7 +25,7 @@ import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; /** - * Taking a schema, this class serializes a CDC into AVRO format + * Serializer to convert Cassandra CDC events into Avro GenericRecord objects. */ public class CdcAvroSerializer extends AvroGenericRecordSerializer { diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index dd9515cab..f4eaae886 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -54,7 +54,26 @@ /** - * Class handling CDC iterators + * Manages the lifecycle and coordination of CDC (Change Data Capture) consumers for processing + * Cassandra change events across distributed token ranges. + * + *

This class is responsible for: + *

    + *
  • Building and configuring {@link SidecarCdc} consumers based on owned token ranges
  • + *
  • Deduplicating consumers by instance ID and token range to prevent duplicate processing
  • + *
  • Managing consumer lifecycle (start/stop operations)
  • + *
  • Integrating with various providers for cluster configuration, schema, and instance metadata
  • + *
  • Coordinating with the range manager to determine token ownership
  • + *
+ * + *

The CDC consumers created by this manager process change events from Cassandra commit logs + * and forward them to configured event consumers (such as Kafka producers) while maintaining + * state persistence and proper token range distribution across the cluster. + * + * @see SidecarCdc + * @see EventConsumer + * @see RangeManager + * @see CdcConfig */ public class CdcManager { diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index bc959e2eb..bbf86538f 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -53,9 +53,7 @@ import org.apache.cassandra.sidecar.config.KeyStoreConfiguration; import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.config.SslConfiguration; -import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; import org.apache.cassandra.sidecar.coordination.RangeManager; -import org.apache.cassandra.sidecar.coordination.TokenRingProvider; import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; import org.apache.cassandra.sidecar.tasks.PeriodicTask; @@ -74,9 +72,22 @@ @Singleton public class CdcPublisher implements Handler>, PeriodicTask { - static final Logger LOGGER = LoggerFactory.getLogger(CdcPublisher.class); - static final long INITIALIZATION_LOOP_DELAY_MILLIS = 1000; - private final Vertx vertx; + private static final Logger LOGGER = LoggerFactory.getLogger(CdcPublisher.class); + private static final long INITIALIZATION_LOOP_DELAY_MILLIS = 1000; + + // SSL Configuration Keys + private static final String SSL_ENABLED_KEY = "enabled"; + private static final String SSL_PREFER_OPENSSL_KEY = "preferOpenSSL"; + private static final String SSL_CLIENT_AUTH_KEY = "clientAuth"; + private static final String SSL_CIPHER_SUITES_KEY = "cipherSuites"; + private static final String SSL_SECURE_TRANSPORT_PROTOCOLS_KEY = "secureTransportProtocols"; + private static final String SSL_HANDSHAKE_TIMEOUT_KEY = "handshakeTimeout"; + private static final String SSL_KEYSTORE_PATH_KEY = "keystorePath"; + private static final String SSL_KEYSTORE_PASSWORD_KEY = "keystorePassword"; + private static final String SSL_KEYSTORE_TYPE_KEY = "keystoreType"; + private static final String SSL_TRUSTSTORE_PATH_KEY = "truststorePath"; + private static final String SSL_TRUSTSTORE_PASSWORD_KEY = "truststorePassword"; + private static final String SSL_TRUSTSTORE_TYPE_KEY = "truststoreType"; private final TaskExecutorPool executorPools; private final CdcConfig conf; private volatile boolean isRunning = false; @@ -86,7 +97,6 @@ public class CdcPublisher implements Handler>, PeriodicTask private final VirtualTablesDatabaseAccessor virtualTables; private final SidecarCdcStats sidecarCdcStats; private RangeManager rangeManager; - private final TokenRingProvider tokenRingProvider; private final SchemaSupplier schemaSupplier; private final CdcSidecarInstancesProvider sidecarInstancesProvider; private final InstanceMetadataFetcher instanceMetadataFetcher; @@ -95,7 +105,7 @@ public class CdcPublisher implements Handler>, PeriodicTask private final ICdcStats cdcStats; private final SidecarConfiguration sidecarConfiguration; private CdcManager cdcManager; - Serializer avroSerializer; + private Serializer avroSerializer; @Inject public CdcPublisher(Vertx vertx, @@ -109,18 +119,15 @@ public CdcPublisher(Vertx vertx, CdcConfig conf, CdcDatabaseAccessor databaseAccessor, ICdcStats cdcStats, - TokenRingProvider tokenRingProvider, VirtualTablesDatabaseAccessor virtualTables, SidecarCdcStats sidecarCdcStats, - Serializer avroSerializer) + Serializer avroSerializer, + RangeManager rangeManager) { - this.vertx = vertx; - this.sidecarCdcStats = sidecarCdcStats; this.executorPools = executorPools.internal(); this.conf = conf; this.databaseAccessor = databaseAccessor; - this.tokenRingProvider = tokenRingProvider; this.virtualTables = virtualTables; this.schemaSupplier = schemaSupplier; @@ -131,6 +138,7 @@ public CdcPublisher(Vertx vertx, this.cdcStats = cdcStats; this.sidecarConfiguration = sidecarConfiguration; this.avroSerializer = avroSerializer; + this.rangeManager = rangeManager; vertx.eventBus().localConsumer(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), this); vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), this); vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), this); @@ -150,27 +158,27 @@ public SecretsProvider secretsProvider() Map sslConfigMap = new HashMap<>(); - sslConfigMap.put("enabled", sslConfiguration.enabled() + ""); - sslConfigMap.put("preferOpenSSL", sslConfiguration.preferOpenSSL() + ""); - sslConfigMap.put("clientAuth", sslConfiguration.clientAuth()); - sslConfigMap.put("cipherSuites", String.join(",", sslConfiguration.cipherSuites())); - sslConfigMap.put("secureTransportProtocols", String.join(",", sslConfiguration.secureTransportProtocols())); - sslConfigMap.put("handshakeTimeout", sslConfiguration.handshakeTimeout().toString()); + sslConfigMap.put(SSL_ENABLED_KEY, sslConfiguration.enabled() + ""); + sslConfigMap.put(SSL_PREFER_OPENSSL_KEY, sslConfiguration.preferOpenSSL() + ""); + sslConfigMap.put(SSL_CLIENT_AUTH_KEY, sslConfiguration.clientAuth()); + sslConfigMap.put(SSL_CIPHER_SUITES_KEY, String.join(",", sslConfiguration.cipherSuites())); + sslConfigMap.put(SSL_SECURE_TRANSPORT_PROTOCOLS_KEY, String.join(",", sslConfiguration.secureTransportProtocols())); + sslConfigMap.put(SSL_HANDSHAKE_TIMEOUT_KEY, sslConfiguration.handshakeTimeout().toString()); if (sslConfiguration.isKeystoreConfigured()) { KeyStoreConfiguration keystore = sslConfiguration.keystore(); - sslConfigMap.put("keystorePath", keystore.path()); - sslConfigMap.put("keystorePassword", keystore.password()); - sslConfigMap.put("keystoreType", keystore.type()); + sslConfigMap.put(SSL_KEYSTORE_PATH_KEY, keystore.path()); + sslConfigMap.put(SSL_KEYSTORE_PASSWORD_KEY, keystore.password()); + sslConfigMap.put(SSL_KEYSTORE_TYPE_KEY, keystore.type()); } if (sslConfiguration.isTrustStoreConfigured()) { KeyStoreConfiguration truststore = sslConfiguration.truststore(); - sslConfigMap.put("truststorePath", truststore.path()); - sslConfigMap.put("truststorePassword", truststore.password()); - sslConfigMap.put("truststoreType", truststore.type()); + sslConfigMap.put(SSL_TRUSTSTORE_PATH_KEY, truststore.path()); + sslConfigMap.put(SSL_TRUSTSTORE_PASSWORD_KEY, truststore.password()); + sslConfigMap.put(SSL_TRUSTSTORE_TYPE_KEY, truststore.type()); } SslConfig sslConfig = SslConfig.create(sslConfigMap); @@ -283,21 +291,15 @@ private void initialize() { try { - if (this.rangeManager == null) - { - this.rangeManager = new ContentionFreeRangeManager(vertx, tokenRingProvider); - } String localDc = rangeManager.getLocalDcSafe(); if (conf.datacenter() != null && !conf.datacenter().isEmpty() && !conf.datacenter().equals(localDc)) { LOGGER.info("Cdc not enabled in this DC localDc={} cdcDc={}", localDc, conf.datacenter()); - return; } else if (virtualTables.isCdcOnRepairEnabled()) { LOGGER.warn("Cannot run CDC while cdc on repair is enabled, disable cdc_on_repair_enabled in the yaml file."); sidecarCdcStats.captureCdcOnRepairEnabled(); - return; } else if (conf.cdcEnabled()) { diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java index c22fb00e1..0c55ab531 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcSchemaSupplier.java @@ -45,7 +45,31 @@ /** - * Class providing the schema for CDC + * Supplies schema information for CDC (Change Data Capture) enabled tables in a Cassandra cluster. + * + *

This class is responsible for: + *

    + *
  • Retrieving and parsing the complete schema from Cassandra instances
  • + *
  • Identifying tables that have CDC enabled
  • + *
  • Building {@link CqlTable} representations with appropriate metadata
  • + *
  • Caching table IDs to optimize repeated lookups
  • + *
  • Extracting User Defined Types (UDTs) and replication factors
  • + *
  • Updating the CDC bridge with schema information
  • + *
+ * + *

The schema supplier works by: + *

    + *
  1. Fetching the full schema export from any available Cassandra instance
  2. + *
  3. Parsing CREATE TABLE statements to identify CDC-enabled tables
  4. + *
  5. Extracting keyspace-specific UDTs and replication settings
  6. + *
  7. Building complete {@link CqlTable} objects with partitioner information
  8. + *
  9. Coordinating with the CDC bridge to maintain schema consistency
  10. + *
+ * + * @see SchemaSupplier + * @see CqlTable + * @see CdcUtil + * @see CassandraBridge */ @Singleton public class CdcSchemaSupplier implements SchemaSupplier diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java new file mode 100644 index 000000000..68c91017b --- /dev/null +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java @@ -0,0 +1,59 @@ +package org.apache.cassandra.sidecar.cdc; + +import java.util.Set; +import java.util.stream.Collectors; + +import com.datastax.driver.core.Host; +import org.apache.cassandra.cdc.sidecar.ClusterConfigProvider; +import org.apache.cassandra.sidecar.common.response.NodeSettings; +import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; +import org.apache.cassandra.spark.data.partitioner.CassandraInstance; +import org.apache.cassandra.spark.data.partitioner.Partitioner; + +/** + * Provides cluster configuration information for CDC (Change Data Capture) operations by + * interfacing with Cassandra sidecar instances. + * + *

This implementation of {@link ClusterConfigProvider} retrieves cluster topology and + * configuration details from available Cassandra instances through the sidecar's metadata + * fetcher. It serves as a bridge between the CDC framework and the sidecar's knowledge + * of the Cassandra cluster. + * + */ +public class SidecarClusterConfigProvider implements ClusterConfigProvider +{ + private final InstanceMetadataFetcher instanceMetadataFetcher; + + public SidecarClusterConfigProvider(InstanceMetadataFetcher instanceMetadataFetcher) + { + this.instanceMetadataFetcher = instanceMetadataFetcher; + } + + public String dc() + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + return nodeSettings.datacenter(); + } + + public Set getCluster() + { + Set hosts = instanceMetadataFetcher.callOnFirstAvailableInstance(instance -> + instance.delegate().metadata().getAllHosts()); + return hosts.stream() + .filter(host -> host.getListenAddress() != null) + .flatMap(host -> host.getTokens().stream() + .map(token -> new CassandraInstance( + token.toString(), + host.getEndPoint().resolve().getHostName(), + host.getDatacenter() + )) + ).collect(Collectors.toSet()); + } + + public Partitioner partitioner() + { + NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); + String[] parts = nodeSettings.partitioner().split("\\."); + return Partitioner.valueOf(parts[parts.length - 1]); + } +} diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java index 62b70956b..472a4cf8a 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/CdcDatabaseAccessor.java @@ -20,10 +20,8 @@ package org.apache.cassandra.sidecar.db; import java.nio.ByteBuffer; -import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Optional; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.stream.Collectors; @@ -40,14 +38,8 @@ import com.google.inject.Provider; import com.google.inject.ProvisionException; import com.google.inject.Singleton; -import org.apache.cassandra.bridge.CassandraBridge; -import org.apache.cassandra.bridge.CassandraBridgeFactory; import org.apache.cassandra.bridge.TokenRange; -import org.apache.cassandra.cdc.CdcKryoRegister; -import org.apache.cassandra.cdc.state.CdcState; -import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; -import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; import org.apache.cassandra.sidecar.db.schema.CdcStatesSchema; import org.apache.cassandra.sidecar.db.schema.SidecarSchema; @@ -138,32 +130,6 @@ public List storeStateAsync(String jobId, .collect(Collectors.toList()); } - /** - * Load cdc state for a given jobId and token range and merge into canonical view - * - * @param stats SidecarCdcStats to publish metrics - * @param jobId Cdc job id - * @param range token range - * @return merged SidecarCdcState object that merges previous state objects that overlap with token range to given canonical view of Cdc state. - */ - public Optional loadSidecarCdcState(SidecarCdcStats stats, String jobId, TokenRange range) - { - NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); - CassandraBridge cassandraBridge = new CassandraBridgeFactory().get(nodeSettings.releaseVersion()); - List sizes = new ArrayList<>(); - // deserialize and merge the CDC state objects into canonical view - Optional result = loadStateForRange(jobId, range) - .peek(bytes -> sizes.add(bytes.length)) - .map((byte[] compressed) -> CdcState.deserialize(CdcKryoRegister.kryo(), cassandraBridge.compressionUtil(), compressed)) - .reduce((s1, s2) -> s1.merge(range, s2)); - int count = sizes.size(); - int len = sizes.stream().mapToInt(i -> i).sum(); - LOGGER.debug("Read CDC state from Cassandra jobId={} start={} end={} stateCount={} stateSize={}", - jobId, range.lowerEndpoint(), range.upperEndpoint(), count, len); - stats.captureCdcConsumerReadFromState(count, len); - return result; - } - /** * @param jobId Cdc job id * @param range token range diff --git a/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java b/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java index eae5b4f8b..be80e386c 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/db/TableHistoryDatabaseAccessor.java @@ -22,11 +22,7 @@ import java.nio.charset.StandardCharsets; import java.util.UUID; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import com.datastax.driver.core.ResultSetFuture; -import com.datastax.driver.core.Row; import com.google.inject.Inject; import com.google.inject.Singleton; import org.apache.cassandra.sidecar.common.server.CQLSessionProvider; @@ -40,8 +36,6 @@ @Singleton public class TableHistoryDatabaseAccessor extends DatabaseAccessor { - private static final Logger LOGGER = LoggerFactory.getLogger(TableHistoryDatabaseAccessor.class); - @Inject public TableHistoryDatabaseAccessor(SidecarSchema sidecarSchema, CQLSessionProvider sessionProvider) @@ -56,23 +50,4 @@ public ResultSetFuture insertTableSchemaHistory(String keyspace, String tableNam .insertTableSchema() .bind(keyspace, tableName, schemaUuid, schema)); } - - public String tableSchemaFromVersion(String keyspace, String tableName, String version) - { - UUID schemaUuid = UUID.fromString(version); - try - { - Row row = session() - .executeAsync(tableSchema - .selectVersionTableSchema() - .bind(keyspace, tableName, schemaUuid)) - .get() - .one(); - return row == null ? null : row.getString("table_schema"); - } - catch (Exception e) - { - throw new RuntimeException(e); - } - } } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java index ab68f050c..a667288b1 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java @@ -18,13 +18,6 @@ package org.apache.cassandra.sidecar.modules; -import java.util.Set; -import java.util.stream.Collectors; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import com.datastax.driver.core.Host; import com.google.inject.AbstractModule; import com.google.inject.Provides; import com.google.inject.Singleton; @@ -52,20 +45,22 @@ import org.apache.cassandra.sidecar.cdc.CdcPublisher; import org.apache.cassandra.sidecar.cdc.CdcSchemaSupplier; import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.cdc.SidecarClusterConfigProvider; import org.apache.cassandra.sidecar.cdc.SidecarCqlToAvroSchemaConverter; import org.apache.cassandra.sidecar.client.SidecarInstancesProvider; import org.apache.cassandra.sidecar.cluster.InstancesMetadata; import org.apache.cassandra.sidecar.common.ApiEndpointsV1; import org.apache.cassandra.sidecar.common.request.data.AllServicesConfigPayload; import org.apache.cassandra.sidecar.common.response.ListCdcSegmentsResponse; -import org.apache.cassandra.sidecar.common.response.NodeSettings; import org.apache.cassandra.sidecar.concurrent.ExecutorPools; import org.apache.cassandra.sidecar.config.ServiceConfiguration; import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.coordination.CassandraClientTokenRingProvider; +import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; import org.apache.cassandra.sidecar.coordination.DynamicSidecarInstancesProvider; import org.apache.cassandra.sidecar.coordination.InnerDcTokenAdjacentPeerProvider; +import org.apache.cassandra.sidecar.coordination.RangeManager; import org.apache.cassandra.sidecar.coordination.SidecarHttpHealthProvider; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; @@ -99,8 +94,6 @@ import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; import org.apache.cassandra.sidecar.utils.SidecarClientProvider; import org.apache.cassandra.sidecar.utils.TokenSplitUtil; -import org.apache.cassandra.spark.data.partitioner.CassandraInstance; -import org.apache.cassandra.spark.data.partitioner.Partitioner; import org.apache.kafka.common.serialization.Serializer; import org.eclipse.microprofile.openapi.annotations.Operation; import org.eclipse.microprofile.openapi.annotations.enums.SchemaType; @@ -114,9 +107,6 @@ @Path("/") public class CdcModule extends AbstractModule { - - private static final Logger LOGGER = LoggerFactory.getLogger(CdcModule.class); - @ProvidesIntoMap @KeyClassMapKey(PeriodicTaskMapKeys.SidecarPeerHealthMonitorTaskKey.class) PeriodicTask sidecarPeerHealthMonitorTask(SidecarPeerHealthMonitorTask task) @@ -347,38 +337,9 @@ public SchemaSupplier schemaSupplier(InstanceMetadataFetcher instanceMetadataFet @Provides @Singleton - public ClusterConfigProvider clusterConfigProvider(InstanceMetadataFetcher instanceMetadataFetcher, CassandraBridgeFactory cassandraBridgeFactory) + public ClusterConfigProvider clusterConfigProvider(InstanceMetadataFetcher instanceMetadataFetcher) { - return new ClusterConfigProvider() - { - public String dc() - { - NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); - return nodeSettings.datacenter(); - } - - public Set getCluster() - { - Set hosts = instanceMetadataFetcher.callOnFirstAvailableInstance(instance -> - instance.delegate().metadata().getAllHosts()); - return hosts.stream() - .filter(host -> host.getListenAddress() != null) - .flatMap(host -> host.getTokens().stream() - .map(token -> new CassandraInstance( - token.toString(), - host.getEndPoint().resolve().getHostName(), - host.getDatacenter() - )) - ).collect(Collectors.toSet()); - } - - public Partitioner partitioner() - { - NodeSettings nodeSettings = instanceMetadataFetcher.callOnFirstAvailableInstance(instance-> instance.delegate().nodeSettings()); - String[] parts = nodeSettings.partitioner().split("\\."); - return Partitioner.valueOf(parts[parts.length - 1]); - } - }; + return new SidecarClusterConfigProvider(instanceMetadataFetcher); } @Provides @@ -414,6 +375,14 @@ public TableSchema virtualTablesDatabaseAccessor(ServiceConfiguration configurat return new CdcStatesSchema(configuration); } + @Provides + @Singleton + public RangeManager rangeManager(Vertx vertx, + TokenRingProvider tokenRingProvider) + { + return new ContentionFreeRangeManager(vertx, tokenRingProvider); + } + @ProvidesIntoMap @KeyClassMapKey(PeriodicTaskMapKeys.CdcPublisherTaskKey.class) PeriodicTask cdcPublisherTask(Vertx vertx, @@ -427,10 +396,10 @@ PeriodicTask cdcPublisherTask(Vertx vertx, CdcConfig conf, CdcDatabaseAccessor databaseAccessor, ICdcStats cdcStats, - TokenRingProvider tokenRingProvider, VirtualTablesDatabaseAccessor virtualTables, SidecarCdcStats sidecarCdcStats, - Serializer avroSerializer) + Serializer avroSerializer, + RangeManager rangeManager) { return new CdcPublisher(vertx, sidecarConfiguration, @@ -443,10 +412,10 @@ PeriodicTask cdcPublisherTask(Vertx vertx, conf, databaseAccessor, cdcStats, - tokenRingProvider, virtualTables, sidecarCdcStats, - avroSerializer); + avroSerializer, + rangeManager); } @Singleton diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java index 7f261015c..19f1d0b08 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/CdcConfigRefresherNotifierTask.java @@ -53,7 +53,7 @@ public class CdcConfigRefresherNotifierTask implements PeriodicTask private final KafkaConfigAccessor kafkaConfigAccessor; private final Vertx vertx; private final List configChangeListeners = Collections.synchronizedList(new ArrayList<>()); - public ConfigMappings configMappings = new ConfigMappings(); + private final ConfigMappings configMappings = new ConfigMappings(); @Inject public CdcConfigRefresherNotifierTask(Vertx vertx, diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java index a088d90e6..01d93e077 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java @@ -176,7 +176,6 @@ public ClusterTopologyMonitor(Vertx vertx, TokenRingProvider tokenRingProvider) @Override public DurationSpec delay() { -// return MillisecondBoundConfiguration.parse(sidecarConfiguration.serviceConfiguration().refreshClusterTopology()); return MillisecondBoundConfiguration.parse("1000"); } @@ -207,9 +206,9 @@ protected void refresh() { for (String dc : tokenRingProvider.dcs()) { - @Nullable final Map> prev = perDcRanges.get(dc); + Map> prev = perDcRanges.get(dc); - final Map> curr = tokenRingProvider.getPrimaryTokenRanges(dc); + Map> curr = tokenRingProvider.getPrimaryTokenRanges(dc); if (!curr.equals(prev) && changeInInstanceTopology(prev, curr) && update(dc, prev, curr)) { LOGGER.info("Publishing ON_DC_TOPOLOGY_CHANGE event"); diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java index 9c04c011e..63142592f 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java @@ -41,6 +41,7 @@ import org.apache.cassandra.sidecar.config.KeyStoreConfiguration; import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.config.SslConfiguration; +import org.apache.cassandra.sidecar.coordination.RangeManager; import org.apache.cassandra.sidecar.coordination.TokenRingProvider; import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; @@ -87,6 +88,8 @@ public class CdcPublisherTests private SidecarCdcStats sidecarCdcStats; @Mock private Serializer avroSerializer; + @Mock + private RangeManager rangeManager; private SidecarConfiguration sidecarConfiguration; private CdcConfig cdcConfig; @@ -119,10 +122,10 @@ void setUp() cdcConfig, databaseAccessor, cdcStats, - tokenRingProvider, virtualTables, sidecarCdcStats, - avroSerializer + avroSerializer, + rangeManager ); } From 6865f60838fcefb9eeb2695ff913d7573e329484 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 10 Dec 2025 14:38:20 -0800 Subject: [PATCH 08/20] Add missing license --- .../cdc/SidecarClusterConfigProvider.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java index 68c91017b..7a4c7df49 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/SidecarClusterConfigProvider.java @@ -1,3 +1,22 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + package org.apache.cassandra.sidecar.cdc; import java.util.Set; From 1c8f4b0bbcc044c887cd7697c8b92dd57ab4c90f Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 10 Dec 2025 15:24:43 -0800 Subject: [PATCH 09/20] Fix deferred injection --- .../cassandra/sidecar/cdc/CdcPublisher.java | 11 ++++++----- .../cassandra/sidecar/modules/CdcModule.java | 15 ++++----------- .../cassandra/sidecar/cdc/CdcPublisherTests.java | 6 ++---- 3 files changed, 12 insertions(+), 20 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index bbf86538f..9ff67c3f3 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -27,6 +27,7 @@ import com.google.inject.Inject; +import com.google.inject.Provider; import com.google.inject.Singleton; import io.vertx.core.Handler; import io.vertx.core.Promise; @@ -96,7 +97,6 @@ public class CdcPublisher implements Handler>, PeriodicTask private final CdcDatabaseAccessor databaseAccessor; private final VirtualTablesDatabaseAccessor virtualTables; private final SidecarCdcStats sidecarCdcStats; - private RangeManager rangeManager; private final SchemaSupplier schemaSupplier; private final CdcSidecarInstancesProvider sidecarInstancesProvider; private final InstanceMetadataFetcher instanceMetadataFetcher; @@ -106,6 +106,7 @@ public class CdcPublisher implements Handler>, PeriodicTask private final SidecarConfiguration sidecarConfiguration; private CdcManager cdcManager; private Serializer avroSerializer; + private Provider rangeManagerProvider; @Inject public CdcPublisher(Vertx vertx, @@ -122,7 +123,7 @@ public CdcPublisher(Vertx vertx, VirtualTablesDatabaseAccessor virtualTables, SidecarCdcStats sidecarCdcStats, Serializer avroSerializer, - RangeManager rangeManager) + Provider rangeManagerProvider) { this.sidecarCdcStats = sidecarCdcStats; this.executorPools = executorPools.internal(); @@ -138,7 +139,7 @@ public CdcPublisher(Vertx vertx, this.cdcStats = cdcStats; this.sidecarConfiguration = sidecarConfiguration; this.avroSerializer = avroSerializer; - this.rangeManager = rangeManager; + this.rangeManagerProvider = rangeManagerProvider; vertx.eventBus().localConsumer(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), this); vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), this); vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), this); @@ -224,7 +225,7 @@ private synchronized void run() throws IllegalStateException cdcManager = new CdcManager(eventConsumer(conf, avroSerializer), schemaSupplier, conf, - rangeManager, + rangeManagerProvider.get(), instanceMetadataFetcher, clusterConfigProvider, sidecarInstancesProvider, @@ -291,7 +292,7 @@ private void initialize() { try { - String localDc = rangeManager.getLocalDcSafe(); + String localDc = rangeManagerProvider.get().getLocalDcSafe(); if (conf.datacenter() != null && !conf.datacenter().isEmpty() && !conf.datacenter().equals(localDc)) { LOGGER.info("Cdc not enabled in this DC localDc={} cdcDc={}", localDc, conf.datacenter()); diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java index a667288b1..e88f015a0 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java @@ -19,6 +19,7 @@ package org.apache.cassandra.sidecar.modules; import com.google.inject.AbstractModule; +import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.Singleton; import com.google.inject.multibindings.ProvidesIntoMap; @@ -375,14 +376,6 @@ public TableSchema virtualTablesDatabaseAccessor(ServiceConfiguration configurat return new CdcStatesSchema(configuration); } - @Provides - @Singleton - public RangeManager rangeManager(Vertx vertx, - TokenRingProvider tokenRingProvider) - { - return new ContentionFreeRangeManager(vertx, tokenRingProvider); - } - @ProvidesIntoMap @KeyClassMapKey(PeriodicTaskMapKeys.CdcPublisherTaskKey.class) PeriodicTask cdcPublisherTask(Vertx vertx, @@ -395,11 +388,11 @@ PeriodicTask cdcPublisherTask(Vertx vertx, InstanceMetadataFetcher instanceMetadataFetcher, CdcConfig conf, CdcDatabaseAccessor databaseAccessor, + TokenRingProvider tokenRingProvider, ICdcStats cdcStats, VirtualTablesDatabaseAccessor virtualTables, SidecarCdcStats sidecarCdcStats, - Serializer avroSerializer, - RangeManager rangeManager) + Serializer avroSerializer) { return new CdcPublisher(vertx, sidecarConfiguration, @@ -415,7 +408,7 @@ PeriodicTask cdcPublisherTask(Vertx vertx, virtualTables, sidecarCdcStats, avroSerializer, - rangeManager); + () -> new ContentionFreeRangeManager(vertx, tokenRingProvider)); } @Singleton diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java index 63142592f..f6eecf10a 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcPublisherTests.java @@ -26,6 +26,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import com.google.inject.Provider; import io.vertx.core.Vertx; import org.apache.cassandra.cdc.api.EventConsumer; import org.apache.cassandra.cdc.api.SchemaSupplier; @@ -42,7 +43,6 @@ import org.apache.cassandra.sidecar.config.SidecarConfiguration; import org.apache.cassandra.sidecar.config.SslConfiguration; import org.apache.cassandra.sidecar.coordination.RangeManager; -import org.apache.cassandra.sidecar.coordination.TokenRingProvider; import org.apache.cassandra.sidecar.db.CdcDatabaseAccessor; import org.apache.cassandra.sidecar.db.VirtualTablesDatabaseAccessor; import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher; @@ -81,15 +81,13 @@ public class CdcPublisherTests @Mock private ICdcStats cdcStats; @Mock - private TokenRingProvider tokenRingProvider; - @Mock private VirtualTablesDatabaseAccessor virtualTables; @Mock private SidecarCdcStats sidecarCdcStats; @Mock private Serializer avroSerializer; @Mock - private RangeManager rangeManager; + private Provider rangeManager; private SidecarConfiguration sidecarConfiguration; private CdcConfig cdcConfig; From 02bc3994a45de36dd3895250c14107ff1165a115 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 10 Dec 2025 15:36:17 -0800 Subject: [PATCH 10/20] Re add config changed test --- .../sidecar/cdc/CdcConfigImplTest.java | 78 +++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java index 19f7a0475..56cdabfad 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java @@ -20,12 +20,15 @@ import java.util.Map; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import io.vertx.core.Promise; import io.vertx.core.Vertx; +import io.vertx.core.eventbus.Message; +import io.vertx.core.eventbus.MessageConsumer; import org.apache.cassandra.sidecar.TestResourceReaper; import org.apache.cassandra.sidecar.codecs.CdcConfigMappingsCodec; import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; @@ -38,6 +41,7 @@ import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIGURATION_CHANGED; import static org.apache.cassandra.testing.utils.AssertionUtils.loopAssert; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; @@ -137,6 +141,80 @@ void testConfigsWhenConfigsAreNotEmpty() throws InterruptedException assertThat(cdcConfig.persistDelay()).isEqualTo(new MillisecondBoundConfiguration(5, TimeUnit.SECONDS)); } + @Test + void testConfigChanged() throws InterruptedException + { + // Create a dedicated Vertx instance for this test to avoid interference from other tests + Vertx testVertx = Vertx.vertx(); + try + { + // Set up a listener to count configuration change signals + AtomicInteger configChangeCount = new AtomicInteger(0); + MessageConsumer consumer = testVertx.eventBus().localConsumer(ON_CDC_CONFIGURATION_CHANGED.address(), + (Message message) -> configChangeCount.incrementAndGet()); + + CdcConfigAccessor cdcConfigAccessor = mockCdcConfigAccessor(); + + // Initial configuration + CdcConfigRefresherNotifierTask.ConfigMappings initialConfig = new CdcConfigRefresherNotifierTask.ConfigMappings(); + initialConfig.setKafkaConfigMappings(Map.of("topic", "topic1")); + initialConfig.setCdcConfigMappings(Map.of("datacenter", "DC1", + "env", "if", + "log_only", "false")); + + // Create CdcConfigImpl with the dedicated Vertx instance + CdcConfigImpl cdcConfig = new CdcConfigImpl(testVertx, cdcConfigAccessor); + testVertx.eventBus().registerDefaultCodec(CdcConfigRefresherNotifierTask.ConfigMappings.class, CdcConfigMappingsCodec.INSTANCE); + + // Publish initial config - this should trigger the first configuration changed signal + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), initialConfig); + + // Wait for the configuration to be processed and change signal to be sent + loopAssert(5, () -> assertThat(configChangeCount.get()).isEqualTo(1)); + + // Publish the same config multiple times - current implementation will send signals each time + // (This is expected behavior since the implementation doesn't check for actual changes) + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), initialConfig); + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), initialConfig); + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), initialConfig); + + // Wait for all signals to be processed - expecting 4 total (1 + 3) + loopAssert(5, () -> assertThat(configChangeCount.get()).isEqualTo(4)); + + // Create updated configuration with a change + CdcConfigRefresherNotifierTask.ConfigMappings updatedConfig = new CdcConfigRefresherNotifierTask.ConfigMappings(); + updatedConfig.setKafkaConfigMappings(Map.of("topic", "topic1")); + updatedConfig.setCdcConfigMappings(Map.of("datacenter", "DC1", + "env", "if", + "log_only", "true")); // Changed from false to true + + // Publish updated config - this should trigger another configuration changed signal + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), updatedConfig); + + // Wait for the updated configuration to be processed and change signal to be sent (5 total) + loopAssert(5, () -> assertThat(configChangeCount.get()).isEqualTo(5)); + + // Verify the configuration was actually updated + assertThat(cdcConfig.logOnly()).isTrue(); + + // Publish the same updated config multiple times - will send signals each time + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), updatedConfig); + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), updatedConfig); + testVertx.eventBus().publish(ON_CDC_CONFIG_MAPPINGS_CHANGED.address(), updatedConfig); + + // Wait for all signals to be processed - expecting 8 total (5 + 3) + loopAssert(5, () -> assertThat(configChangeCount.get()).isEqualTo(8)); + + // Clean up the consumer + consumer.unregister(); + } + finally + { + // Clean up the dedicated Vertx instance + testVertx.close(); + } + } + private CdcConfigAccessor mockCdcConfigAccessor() { CdcConfigAccessor cdcConfigAccessor = mock(CdcConfigAccessor.class, RETURNS_DEEP_STUBS); From cb4f58169eb667eff79d31ccec16302e16534f51 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 19 Dec 2025 16:47:05 +0100 Subject: [PATCH 11/20] Fix checkstyle --- .../java/org/apache/cassandra/sidecar/modules/CdcModule.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java index e88f015a0..e7d8f3d51 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java @@ -19,7 +19,6 @@ package org.apache.cassandra.sidecar.modules; import com.google.inject.AbstractModule; -import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.Singleton; import com.google.inject.multibindings.ProvidesIntoMap; @@ -61,7 +60,6 @@ import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; import org.apache.cassandra.sidecar.coordination.DynamicSidecarInstancesProvider; import org.apache.cassandra.sidecar.coordination.InnerDcTokenAdjacentPeerProvider; -import org.apache.cassandra.sidecar.coordination.RangeManager; import org.apache.cassandra.sidecar.coordination.SidecarHttpHealthProvider; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; From bbcc22eaf763ed91904b6f33bc48451e4ae4736b Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 19 Dec 2025 17:04:27 +0100 Subject: [PATCH 12/20] Fix checkstyle --- .../org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java index 56cdabfad..c53cd08b5 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcConfigImplTest.java @@ -40,8 +40,8 @@ import org.apache.cassandra.sidecar.tasks.CdcConfigRefresherNotifierTask; import org.apache.cassandra.sidecar.tasks.PeriodicTaskExecutor; -import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIGURATION_CHANGED; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_CDC_CONFIG_MAPPINGS_CHANGED; import static org.apache.cassandra.testing.utils.AssertionUtils.loopAssert; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.RETURNS_DEEP_STUBS; From d60ed0d5fc1cb0c0300d3794f409dfc29befe86e Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 19 Dec 2025 19:21:59 +0100 Subject: [PATCH 13/20] Add Changes.txt --- CHANGES.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/CHANGES.txt b/CHANGES.txt index 2c8d7ebb4..d2baf2aa1 100644 --- a/CHANGES.txt +++ b/CHANGES.txt @@ -1,5 +1,6 @@ 0.3.0 ----- + * Implementation of CDCPublisher (CASSSIDECAR-243) * Sidecar endpoint for moving a node to a new token (CASSSIDECAR-344) * Returning JSON responses for live migration status endpoints in case of errors (CASSSIDECAR-395) * Upgrade vertx to 4.5.23 (CASSSIDECAR-391) From 881f55c50cfed250b041a19d60c801308e228d6b Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 7 Jan 2026 17:35:59 +0100 Subject: [PATCH 14/20] Adds feedback --- .../cassandra/sidecar/cdc/CdcConfigImpl.java | 4 ++-- .../cassandra/sidecar/cdc/CdcManager.java | 18 ++++++++---------- .../cassandra/sidecar/cdc/CdcPublisher.java | 4 ++-- .../sidecar/codecs/BigIntegerCodec.java | 2 +- .../cassandra/sidecar/cdc/CdcManagerTest.java | 16 ++++++++++++++++ 5 files changed, 29 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java index 1d8dd770a..fa4d7469a 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcConfigImpl.java @@ -54,8 +54,8 @@ public class CdcConfigImpl implements CdcConfig private final Vertx vertx; private final CdcConfigAccessor cdcConfigAccessor; - private Map kafkaConfigMappings = Map.of(); - private Map cdcConfigMappings = Map.of(); + private volatile Map kafkaConfigMappings = Map.of(); + private volatile Map cdcConfigMappings = Map.of(); @Inject public CdcConfigImpl(Vertx vertx, diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index f4eaae886..b72d821f0 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -26,7 +26,6 @@ import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,7 +128,7 @@ List buildCdcConsumers() } // NEW: Deduplicate by (instanceId, tokenRange) to prevent duplicate consumers - Map uniqueConsumers = new HashMap<>(); + Map uniqueConsumers = new HashMap<>(ownedRanges.values().stream().mapToInt(Set::size).sum()); ownedRanges.entrySet().stream() .flatMap(entry -> @@ -163,8 +162,7 @@ List buildCdcConsumers() throw new RuntimeException(e); } }); - })) - .collect(Collectors.toList()); + })); consumers = new ArrayList<>(uniqueConsumers.values()); return consumers; @@ -211,10 +209,10 @@ private Integer getInstanceId(String instanceIp) { for (InstanceMetadata instance : instanceFetcher.allLocalInstances()) { - String configuredHost = instance.ipAddress(); + String configuredIpAddress = instance.ipAddress(); // Option 1a: Normalize both to InetAddress and compare - if (resolveToSameAddress(instanceIp, configuredHost)) + if (resolveToSameAddress(instanceIp, configuredIpAddress)) { return instance.id(); } @@ -223,18 +221,18 @@ private Integer getInstanceId(String instanceIp) return -1; } - private boolean resolveToSameAddress(String host1, String host2) + public static boolean resolveToSameAddress(String address1, String address2) { try { - InetAddress addr1 = InetAddress.getByName(host1); - InetAddress addr2 = InetAddress.getByName(host2); + InetAddress addr1 = InetAddress.getByName(address1); + InetAddress addr2 = InetAddress.getByName(address2); return addr1.equals(addr2); } catch (UnknownHostException e) { LOGGER.warn("Could not resolve hostname: {}", e.getMessage()); - return host1.equals(host2); // Fallback to string comparison + return address1.equals(address2); // Fallback to string comparison } } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index 9ff67c3f3..9342bd6f5 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -105,8 +105,8 @@ public class CdcPublisher implements Handler>, PeriodicTask private final ICdcStats cdcStats; private final SidecarConfiguration sidecarConfiguration; private CdcManager cdcManager; - private Serializer avroSerializer; - private Provider rangeManagerProvider; + private final Serializer avroSerializer; + private final Provider rangeManagerProvider; @Inject public CdcPublisher(Vertx vertx, diff --git a/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java index 491cda4b6..5e679278b 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/codecs/BigIntegerCodec.java @@ -55,7 +55,7 @@ public BigInteger decodeFromWire(MutableInt pos, Buffer buf) */ public BigInteger decodeFromWire(int pos, Buffer buf) { - return decodeFromWire(new MutableInt(pos), buf); + return new BigInteger(CommonCodecs.BYTE_ARRAY.decodeFromWire(pos, buf)); } /** diff --git a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java index d4f1cb187..3fd958ef3 100644 --- a/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java +++ b/server/src/test/java/org/apache/cassandra/sidecar/cdc/CdcManagerTest.java @@ -295,6 +295,22 @@ void testUnknownInstanceHandlesGracefully() throws IOException assertThat(consumers).hasSize(1); } + @Test + void testResolveToSameAddressTrue() + { + String address1 = "127.0.0.1"; + String address2 = "localhost"; + assertThat(CdcManager.resolveToSameAddress(address1, address2)).isTrue(); + } + + @Test + void testResolveToSameAddressFalse() + { + String address1 = "127.0.0.1"; + String address2 = "127.0.0.2"; + assertThat(CdcManager.resolveToSameAddress(address1, address2)).isFalse(); + } + // Helper methods private TokenRange mockTokenRange(BigInteger start, BigInteger end) From 3b5fc7659df00dca3b26c481db0b3849a02ca749 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Wed, 7 Jan 2026 19:16:57 +0100 Subject: [PATCH 15/20] Fix tests --- .../java/org/apache/cassandra/sidecar/cdc/CdcManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index b72d821f0..ff2a612be 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -26,6 +26,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -162,7 +163,8 @@ List buildCdcConsumers() throw new RuntimeException(e); } }); - })); + })) + .collect(Collectors.toList());; consumers = new ArrayList<>(uniqueConsumers.values()); return consumers; From ca6faee28ff28e2ddb6200076f09616bf36cb0ef Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Thu, 8 Jan 2026 23:21:04 +0100 Subject: [PATCH 16/20] Revert sidecar.yaml changes --- conf/sidecar.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/conf/sidecar.yaml b/conf/sidecar.yaml index 645aee33c..57be2e787 100644 --- a/conf/sidecar.yaml +++ b/conf/sidecar.yaml @@ -42,11 +42,11 @@ cassandra_instances: # Directory where Cassandra stores mutations. If not set, the default directory is # /cdc_raw. - cdc_dir: /var/lib/cassandra/cdc_raw + #cdc_dir: /var/lib/cassandra/cdc_raw # Directory where Cassandra stores the commit logs. If not set, the default directory # is /commitlog - commitlog_dir: /var/lib/cassandra/commitlog + #commitlog_dir: /var/lib/cassandra/commitlog # Directory where Cassandra stores hints. If not set, the default directory is # /hints. From 296853d86dffc845912d48f24e1f35e1dc19946a Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 9 Jan 2026 01:08:55 +0100 Subject: [PATCH 17/20] Remove not needed todo comment --- .../cassandra/sidecar/coordination/RangeManager.java | 7 +------ .../cassandra/sidecar/tasks/ClusterTopologyMonitor.java | 7 +------ 2 files changed, 2 insertions(+), 12 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java index 0a9dff302..47e0005a7 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/coordination/RangeManager.java @@ -142,12 +142,7 @@ public RangeManager(Vertx vertx, TokenRingProvider tokenRingProvider) } catch (IllegalStateException ise) { - if (ise.getMessage().contains("Already a default codec registered")) - { - // TODO: do not register codec in the constructor when OSS; All codecs should be managed in a central repository - // ignore - } - else + if (!ise.getMessage().contains("Already a default codec registered")) { throw ise; } diff --git a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java index 01d93e077..4b77155e4 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/tasks/ClusterTopologyMonitor.java @@ -158,12 +158,7 @@ public ClusterTopologyMonitor(Vertx vertx, TokenRingProvider tokenRingProvider) } catch (IllegalStateException ise) { - if (ise.getMessage().contains("Already a default codec registered")) - { - // TODO: do not register codec in the constructor when OSS; All codecs should be managed in a central repository - // ignore - } - else + if (!ise.getMessage().contains("Already a default codec registered")) { throw ise; } From 5ab573285861fce0352a89111878272d97f99015 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 9 Jan 2026 17:39:59 +0100 Subject: [PATCH 18/20] Close objects --- .../cassandra/sidecar/cdc/CachingSchemaStore.java | 11 +++++++---- .../apache/cassandra/sidecar/cdc/CdcManager.java | 2 +- .../apache/cassandra/sidecar/cdc/CdcPublisher.java | 14 ++++++++++++-- .../cassandra/sidecar/modules/CdcModule.java | 4 ++-- .../sidecar/server/SidecarServerEvents.java | 14 ++++++++++++++ 5 files changed, 36 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index 31f8475f6..f10141b02 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -85,7 +85,6 @@ public class CachingSchemaStore implements SchemaStore SidecarSchema sidecarSchema, CqlToAvroSchemaConverter cqlToAvroSchemaConverter) { - super(); this.cassandraClusterSchemaMonitor = cassandraClusterSchemaMonitor; this.tableHistoryDatabaseAccessor = tableHistoryDatabaseAccessor; this.sidecarSchema = sidecarSchema; @@ -102,7 +101,11 @@ public class CachingSchemaStore implements SchemaStore private void loadPublisher() { - KafkaOptions kafkaOptions = () -> cdcConfig.kafkaConfigs(); + KafkaOptions kafkaOptions = cdcConfig::kafkaConfigs; + if (this.publisher != null) + { + this.publisher.close(); + } this.publisher = SchemaStorePublisherFactory.DEFAULT.buildPublisher(kafkaOptions); } @@ -171,9 +174,9 @@ void onSchemaChanged() } return v; }); - loadPublisher(); - publishSchemas(); } + loadPublisher(); + publishSchemas(); // Remove any old schema entries for deleted tables, this operation can be done in the end as this is // only for removing stale entries and no one is going to use these entries once the table is removed. // This doesn't have to be an atomic operation. diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java index ff2a612be..215913540 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcManager.java @@ -164,7 +164,7 @@ List buildCdcConsumers() } }); })) - .collect(Collectors.toList());; + .collect(Collectors.toList()); consumers = new ArrayList<>(uniqueConsumers.values()); return consumers; diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index 9342bd6f5..f40ac49dc 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -107,6 +107,8 @@ public class CdcPublisher implements Handler>, PeriodicTask private CdcManager cdcManager; private final Serializer avroSerializer; private final Provider rangeManagerProvider; + KafkaProducer producer; + KafkaPublisher kafkaPublisher; @Inject public CdcPublisher(Vertx vertx, @@ -189,8 +191,16 @@ public SecretsProvider secretsProvider() public EventConsumer eventConsumer(CdcConfig conf, Serializer avroSerializer) { - KafkaProducer producer = new KafkaProducer<>(conf.kafkaConfigs()); - KafkaPublisher kafkaPublisher = new KafkaPublisher(TopicSupplier.staticTopicSupplier(conf.kafkaTopic()), + if (this.producer != null) + { + this.producer.close(); + } + if (this.kafkaPublisher != null) + { + this.kafkaPublisher.close(); + } + this.producer = new KafkaProducer<>(conf.kafkaConfigs()); + this.kafkaPublisher = new KafkaPublisher(TopicSupplier.staticTopicSupplier(conf.kafkaTopic()), producer, avroSerializer, conf.maxRecordSizeBytes(), diff --git a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java index e7d8f3d51..56d5c61f6 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/modules/CdcModule.java @@ -414,8 +414,8 @@ PeriodicTask cdcPublisherTask(Vertx vertx, @KeyClassMapKey(PeriodicTaskMapKeys.CdcConfigRefresherNotifierKey.class) PeriodicTask cdcConfigRefresherNotifier(Vertx vertx, SidecarConfiguration sidecarConfiguration, - KafkaConfigAccessor kafkaConfigAccessor, - CdcConfigAccessor cdcConfigAccessor) + KafkaConfigAccessor kafkaConfigAccessor, + CdcConfigAccessor cdcConfigAccessor) { return new CdcConfigRefresherNotifierTask(vertx, sidecarConfiguration, diff --git a/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java b/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java index 9c876ac79..bc57aa564 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/server/SidecarServerEvents.java @@ -114,8 +114,22 @@ public enum SidecarServerEvents */ ON_SIDECAR_GLOBAL_LEASE_LOST, + /** + * The {@link io.vertx.core.eventbus.EventBus} address where events will be published when the CDC + * (Change Data Capture) configuration has been changed or updated. + */ ON_CDC_CONFIGURATION_CHANGED, + + /** + * The {@link io.vertx.core.eventbus.EventBus} address where events will be published when the CDC + * cache has been warmed up and is ready for use. + */ ON_CDC_CACHE_WARMED_UP, + + /** + * The {@link io.vertx.core.eventbus.EventBus} address where events will be published when the CDC + * configuration mappings have been changed or reloaded. + */ ON_CDC_CONFIG_MAPPINGS_CHANGED, ; From 17825d27c667c4263174da5ebdbd51cb3289b439 Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 9 Jan 2026 18:45:42 +0100 Subject: [PATCH 19/20] Skip event registration when CDC is not enabled --- .../cassandra/sidecar/cdc/CdcPublisher.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java index f40ac49dc..38c8bdd43 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CdcPublisher.java @@ -142,12 +142,16 @@ public CdcPublisher(Vertx vertx, this.sidecarConfiguration = sidecarConfiguration; this.avroSerializer = avroSerializer; this.rangeManagerProvider = rangeManagerProvider; - vertx.eventBus().localConsumer(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), this); - vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), this); - vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), this); - vertx.eventBus().localConsumer(ON_SERVER_STOP.address(), this); - vertx.eventBus().localConsumer(ON_CDC_CACHE_WARMED_UP.address(), this); - vertx.eventBus().localConsumer(ON_CDC_CONFIGURATION_CHANGED.address(), new ConfigChangedHandler()); + + if (conf.cdcEnabled()) + { + vertx.eventBus().localConsumer(RangeManager.RangeManagerEvents.ON_TOKEN_RANGE_CHANGED.address(), this); + vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_GAINED.address(), this); + vertx.eventBus().localConsumer(RangeManager.LeadershipEvents.ON_TOKEN_RANGE_LOST.address(), this); + vertx.eventBus().localConsumer(ON_SERVER_STOP.address(), this); + vertx.eventBus().localConsumer(ON_CDC_CACHE_WARMED_UP.address(), this); + vertx.eventBus().localConsumer(ON_CDC_CONFIGURATION_CHANGED.address(), new ConfigChangedHandler()); + } } public SecretsProvider secretsProvider() From 8c2550415f6b2445b496bade63577948bdc5f99a Mon Sep 17 00:00:00 2001 From: Bernardo Botella Corbi Date: Fri, 9 Jan 2026 22:02:17 +0100 Subject: [PATCH 20/20] Avoid final event registration if CDC is not enabled --- .../org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java index f10141b02..9b24de8c4 100644 --- a/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java +++ b/server/src/main/java/org/apache/cassandra/sidecar/cdc/CachingSchemaStore.java @@ -96,7 +96,10 @@ public class CachingSchemaStore implements SchemaStore this.cdcConfig = cdcConfig; this.sidecarCdcStats = sidecarCdcStats; - configureSidecarServerEventListeners(); + if (cdcConfig.cdcEnabled()) + { + configureSidecarServerEventListeners(); + } } private void loadPublisher()