jmckenzie-dev commented on code in PR #317: URL: https://github.com/apache/cassandra-sidecar/pull/317#discussion_r2818828270
########## gradle.properties: ########## @@ -46,5 +46,5 @@ swaggerVersion=2.2.21 kryoVersion=4.0.2 # OSHI dependencies oshiVersion=6.9.0 -analyticsVersion=0.2.0 +analyticsVersion=0.100 Review Comment: As discussed offline, this needs to be updated now that the new analytics version is out. ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/cdc/CdcIntegrationTest.java: ########## @@ -0,0 +1,97 @@ +/* + * 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; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.sidecar.testing.SharedClusterCdcSidecarIntegrationTestBase; +import org.apache.cassandra.sidecar.testing.TestCdcEventConsumer; +import org.junit.jupiter.api.Test; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF1; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration test for CDC functionality. + * Tests that mutations on CDC-enabled tables are captured and published to TestCdcEventConsumer. + */ +public class CdcIntegrationTest extends SharedClusterCdcSidecarIntegrationTestBase Review Comment: Also - when we go that route we'll probably want to refactor out some shared helpers from this test to use in both. ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/testing/SharedClusterCdcSidecarIntegrationTestBase.java: ########## @@ -0,0 +1,197 @@ +/* + * 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.testing; + +import java.util.Map; +import java.util.function.Function; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import io.vertx.core.Vertx; +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.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.cdc.CdcConfig; +import org.apache.cassandra.sidecar.cdc.CdcPublisher; +import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.concurrent.ExecutorPools; +import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; +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.cassandra.sidecar.utils.SimpleCassandraVersion; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.kafka.common.serialization.Serializer; +import org.junit.jupiter.api.AfterEach; + +import static org.assertj.core.api.Assumptions.assumeThat; + +/** + * Base class for CDC integration tests. Extends SharedClusterIntegrationTestBase with + * CDC-specific configuration and setup, including: + * - CDC-enabled Cassandra cluster configuration + * - TestCdcPublisher with TestCdcEventConsumer + * - Cassandra 4.1 version requirement + * - Helper methods to access CDC components + */ +public abstract class SharedClusterCdcSidecarIntegrationTestBase extends SharedClusterIntegrationTestBase +{ + @AfterEach + void cleanupCdcConsumerAfterEachTest() + { + TestCdcPublisher testCdcPublisher = (TestCdcPublisher) serverWrapper.injector.getInstance(CdcPublisher.class); + if (testCdcPublisher != null) + { + TestCdcEventConsumer consumer = testCdcPublisher.getTestEventConsumer(); + if (consumer != null) + { + consumer.clear(); + } + } + } + + @Override + protected void beforeClusterProvisioning() + { + // CDC integration test is only compatible with Cassandra 4.1 due to cassandra-analytics library version + SimpleCassandraVersion version = SimpleCassandraVersion.create(testVersion.version()); + assumeThat(version.major) + .as("CDC test requires Cassandra 4.x") + .isEqualTo(4); + assumeThat(version.minor) + .as("CDC test requires Cassandra 4.1") + .isEqualTo(1); + } + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return super.testClusterConfiguration() + .dcCount(1) + .nodesPerDc(1) + .additionalInstanceConfig(Map.of("cdc_enabled", true)); + } + + @Override + protected Function<SidecarConfigurationImpl.Builder, SidecarConfigurationImpl.Builder> configurationOverrides() + { + return builder -> { + // Override service configuration to use specific port for CDC tests + ServiceConfiguration existingConfig = builder.build().serviceConfiguration(); + ServiceConfiguration cdcServiceConfig = ServiceConfigurationImpl.builder() + .host(existingConfig.host()) + .port(9043) // TODO: Make this port dynamically allocated Review Comment: About this... should we go ahead and do this now? Or, if CI is very stable w/the current, have a follow up JIRA for a helper method that'll find an open port and then use it? ########## integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/SharedClusterIntegrationTestBase.java: ########## @@ -775,48 +796,23 @@ public LifecycleProvider lifecycleProvider() } public static SidecarConfigurationImpl.Builder defaultConfigurationBuilder( - MtlsTestHelper mtlsTestHelper, Function<SidecarConfigurationImpl.Builder, SidecarConfigurationImpl.Builder> configurationOverrides) + MtlsTestHelper mtlsTestHelper, Review Comment: The indentation here makes me twitch. :) Do we normally do front-of-line like this in this repo? I see it in the prior line you're refactoring so see that you inherit it, but it's definitely not my favorite. Not a blocker, just a question / observation. ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/testing/SharedClusterCdcSidecarIntegrationTestBase.java: ########## @@ -0,0 +1,197 @@ +/* + * 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.testing; + +import java.util.Map; +import java.util.function.Function; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import io.vertx.core.Vertx; +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.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.cdc.CdcConfig; +import org.apache.cassandra.sidecar.cdc.CdcPublisher; +import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.concurrent.ExecutorPools; +import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; +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.cassandra.sidecar.utils.SimpleCassandraVersion; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.kafka.common.serialization.Serializer; +import org.junit.jupiter.api.AfterEach; + +import static org.assertj.core.api.Assumptions.assumeThat; + +/** + * Base class for CDC integration tests. Extends SharedClusterIntegrationTestBase with + * CDC-specific configuration and setup, including: + * - CDC-enabled Cassandra cluster configuration + * - TestCdcPublisher with TestCdcEventConsumer + * - Cassandra 4.1 version requirement + * - Helper methods to access CDC components + */ +public abstract class SharedClusterCdcSidecarIntegrationTestBase extends SharedClusterIntegrationTestBase +{ + @AfterEach + void cleanupCdcConsumerAfterEachTest() + { + TestCdcPublisher testCdcPublisher = (TestCdcPublisher) serverWrapper.injector.getInstance(CdcPublisher.class); + if (testCdcPublisher != null) + { + TestCdcEventConsumer consumer = testCdcPublisher.getTestEventConsumer(); + if (consumer != null) + { + consumer.clear(); + } + } + } + + @Override + protected void beforeClusterProvisioning() + { + // CDC integration test is only compatible with Cassandra 4.1 due to cassandra-analytics library version + SimpleCassandraVersion version = SimpleCassandraVersion.create(testVersion.version()); + assumeThat(version.major) + .as("CDC test requires Cassandra 4.x") + .isEqualTo(4); Review Comment: Greater than or equal to? ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/cdc/CdcIntegrationTest.java: ########## @@ -0,0 +1,97 @@ +/* + * 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; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.cdc.msg.CdcEvent; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.sidecar.testing.SharedClusterCdcSidecarIntegrationTestBase; +import org.apache.cassandra.sidecar.testing.TestCdcEventConsumer; +import org.junit.jupiter.api.Test; + +import static org.apache.cassandra.testing.TestUtils.DC1_RF1; +import static org.assertj.core.api.Assertions.assertThat; + +/** + * Integration test for CDC functionality. + * Tests that mutations on CDC-enabled tables are captured and published to TestCdcEventConsumer. + */ +public class CdcIntegrationTest extends SharedClusterCdcSidecarIntegrationTestBase Review Comment: This looks like solid "happy path does this actually work" testing. Do we have tests that randomly generate data, form a model of what we would expect the kafka messages to look like from that data, then run through the CDC pipeline and then confirm the end state to the modeled data? If not - be a good follow up JIRA. :) ########## integration-framework/src/main/java/org/apache/cassandra/sidecar/testing/TestCdcConfig.java: ########## @@ -0,0 +1,154 @@ +/* + * 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.testing; + +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.cassandra.sidecar.cdc.CdcConfig; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.common.server.utils.SecondBoundConfiguration; + +/** + * Test implementation of {@link CdcConfig} for integration tests. + * Provides hardcoded configuration values suitable for testing CDC functionality + * without requiring external Kafka infrastructure. + */ +public class TestCdcConfig implements CdcConfig +{ + @Override + public String env() + { + return "test"; + } + + @Override + public String kafkaTopic() + { + return "test-topic"; + } + + @Override + public TopicFormatType topicFormat() + { + return TopicFormatType.KEYSPACETABLE; + } + + @Override + public boolean cdcEnabled() + { + return true; + } + + @Override + public String jobId() + { + return "test-job-id"; + } + + @Override + public Map<String, Object> kafkaConfigs() + { + return new HashMap<>(); + } + + @Override + public Map<String, Object> cdcConfigs() + { + return new HashMap<>(); + } + + @Override + public boolean logOnly() + { + return true; + } + + @Override + public String datacenter() + { + return "datacenter1"; + } + + @Override + public SecondBoundConfiguration watermarkWindow() + { + return new SecondBoundConfiguration(259200, TimeUnit.SECONDS); Review Comment: Total nit, but why use a massive # of seconds here instead of `72, TimeUnit.HOURS` or `3, TimeUnit.DAYS`? ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/testing/SharedClusterCdcSidecarIntegrationTestBase.java: ########## @@ -0,0 +1,197 @@ +/* + * 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.testing; + +import java.util.Map; +import java.util.function.Function; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import io.vertx.core.Vertx; +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.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.cdc.CdcConfig; +import org.apache.cassandra.sidecar.cdc.CdcPublisher; +import org.apache.cassandra.sidecar.cdc.SidecarCdcStats; +import org.apache.cassandra.sidecar.concurrent.ExecutorPools; +import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.ContentionFreeRangeManager; +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.cassandra.sidecar.utils.SimpleCassandraVersion; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; +import org.apache.kafka.common.serialization.Serializer; +import org.junit.jupiter.api.AfterEach; + +import static org.assertj.core.api.Assumptions.assumeThat; + +/** + * Base class for CDC integration tests. Extends SharedClusterIntegrationTestBase with + * CDC-specific configuration and setup, including: + * - CDC-enabled Cassandra cluster configuration + * - TestCdcPublisher with TestCdcEventConsumer + * - Cassandra 4.1 version requirement + * - Helper methods to access CDC components + */ +public abstract class SharedClusterCdcSidecarIntegrationTestBase extends SharedClusterIntegrationTestBase +{ + @AfterEach + void cleanupCdcConsumerAfterEachTest() + { + TestCdcPublisher testCdcPublisher = (TestCdcPublisher) serverWrapper.injector.getInstance(CdcPublisher.class); + if (testCdcPublisher != null) + { + TestCdcEventConsumer consumer = testCdcPublisher.getTestEventConsumer(); + if (consumer != null) + { + consumer.clear(); + } + } + } + + @Override + protected void beforeClusterProvisioning() + { + // CDC integration test is only compatible with Cassandra 4.1 due to cassandra-analytics library version + SimpleCassandraVersion version = SimpleCassandraVersion.create(testVersion.version()); + assumeThat(version.major) + .as("CDC test requires Cassandra 4.x") + .isEqualTo(4); + assumeThat(version.minor) + .as("CDC test requires Cassandra 4.1") + .isEqualTo(1); Review Comment: Along w/the above comment, we should probably check that overall we're >= 4.1. Probably need some slightly more complex logic here but not much. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

