CASSANALYTICS-60: CDC support for Cassandra 5.0 commit logs#175
CASSANALYTICS-60: CDC support for Cassandra 5.0 commit logs#175lukasz-antoniak wants to merge 2 commits intoapache:trunkfrom
Conversation
| protected final KafkaStats kafkaStats; | ||
|
|
||
| public KafkaPublisher(TopicSupplier topicSupplier, | ||
| public KafkaPublisher(String version, |
There was a problem hiding this comment.
API change between Sidecar and Analytics.
There was a problem hiding this comment.
Thanks for calling it out.
Thinking from API design perspective, how about passing CassandraVersion, instead of String version value? It, then, ensures the version is always valid. Sidecar anyways has access to CassandraVersion. It would be clear for Sidecar to create the version object.
| * @param <T> | ||
| */ | ||
| public abstract class RangeTombstoneBuilder<T> | ||
| public interface RangeTombstoneBuilder<T> |
There was a problem hiding this comment.
Classloader clash when FourZeroRangeTombstoneBuilder is loaded by bridge classloader and RangeTombstoneBuilder (abstract parent class) by application classloader.
There was a problem hiding this comment.
maybe add the note in code so it stays as interface.
| protected final KafkaStats kafkaStats; | ||
|
|
||
| public KafkaPublisher(TopicSupplier topicSupplier, | ||
| public KafkaPublisher(String version, |
There was a problem hiding this comment.
Thanks for calling it out.
Thinking from API design perspective, how about passing CassandraVersion, instead of String version value? It, then, ensures the version is always valid. Sidecar anyways has access to CassandraVersion. It would be clear for Sidecar to create the version object.
| } | ||
| } | ||
|
|
||
| public static <T> T executeActionOnBridgeClassLoader(@NotNull CassandraVersion version, Throwing.Function<ClassLoader, T> action) |
There was a problem hiding this comment.
nit: add @VisibleForTesting
| * @param <T> | ||
| */ | ||
| public abstract class RangeTombstoneBuilder<T> | ||
| public interface RangeTombstoneBuilder<T> |
There was a problem hiding this comment.
maybe add the note in code so it stays as interface.
yifan-c
left a comment
There was a problem hiding this comment.
Noticed the PR is marked as Draft. I will re-review if there are new changes.
Please share the CI link too.
Fixes Analytics reader of Cassandra 5.0 commit logs (part of CASSANALYTICS-60).
Using environment variable
CASSANDRA_VERSION=5.0.0(or4.0.0), developers can run CDC unit tests with different Cassandra versions.Refactoring all tests to
@ParameterizedTestrequires more efforts, and will be performed in follow-up PR. Static variables likeCdcTests.BRIDGEare references from multiple places in the code.