Skip to content

Conversation

@JeetKunDoug
Copy link
Contributor

@JeetKunDoug JeetKunDoug commented Dec 10, 2025

This PR adds support for post-TCM Cassandra in our integration testing. This is accomplished by:

  • Adding additional interception points to ByteBuddy where behavior has moved in Cassandra
  • Rewrote all tests that depended on unsafe range movements as TCM no longer allows them, and removed tests that could not be run in a safe manner (doubling/halving clusters all at once, for example)

Additional changes unrelated to the trunk support:

  • Updated to latest in-jvm dtest API and removed some now-duplicate classes that moved to that library as part of CASSANDRA-20884. This was the original motivation for the PR, but led to the broader work to get things working on trunk again.
  • Updated maven-shade plugin so support multi-release dependencies
  • Fixed CI (lots of smaller changes, all test-related)

@JeetKunDoug JeetKunDoug force-pushed the dtest-trunk-fixes branch 5 times, most recently from a73db32 to 4bbd37b Compare December 16, 2025 17:35
* Handler for enabling or disabling live migration APIs based on instance role and migration status.
* <p>
* <h3>Handler Methods:</h3>
* <h2>Handler Methods:</h2>
Copy link
Contributor

@yifan-c yifan-c Dec 17, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we not change those docs? They are not relevant.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They are relevant in that different JDKs fail javadoc generation if they are the "wrong" level, which is why they were changed here. I removed some of them from another class for the same reason, so maybe we just remove them all together to avoid the issue?

for i in {2..20}; do sudo ifconfig lo0 alias "127.0.0.${i}"; done
```

Note that this does not persist across reboots, so you'll have to run it every time you restart.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

👍

This commit adds support for post-TCM Cassandra in our integration
testing. This is accomplished by:
- Adding additional interception points to ByteBuddy where behavior has
  moved in Cassandra
- Rewrote all tests that depended on unsafe range movements as TCM no
  longer allows them, and removed tests that could not be run in a safe
  manner (doubling/halving clusters all at once, for example)

Additional changes unrelated to TCM:
- Updated to latest in-jvm dtest API and removed some now-duplicate
  classes that moved to that library as part of CASSANDRA-20884.
  This was the original motivation for the PR, but led to the broader
  work to get things working on trunk again.
- Updated maven-shade plugin so support multi-release dependencies
- Fixed hang of Server shutdown, which was caused by double-closing the server.
  The second call to close caused all kinds of errors in logs and never completed.
- Fix issue where snapshot builds don't have version numbers in gossip that quite match the version the dtest framework returns.
- Several tests require streaming to start in 5.1, which requires the `network` feature to be enabled.
- Some jmx methods that used to throw IllegalArgumentExceptions (dealing with intervals) now wrap those in IOExceptions.
  Attempt to unwrap those before throwing a generic RuntimeException, which breaks the handling of the exception downstream.
- Handle newer Snapshot Already Exists messaging.
- Add export for JMX test in server-common (jdk17)
- exports/opens additions (jdk17)
- Javadoc Fixes
- Update CI workflows based on actually-supported jdks and some additonal jdk11 opens
- Remove headers from some javadoc as different java versions expect different header levels

CI Related changes
- Remove jdk17 from 4.0/4.1 heavyweight
- Add entries to /etc/hosts for localhost network aliases
- Increase timeout for schema ready in SystemAuthDatabaseAccessorIntTest
- Handle ConfigurationException where issue is bind failure.
- Fix async issues in StreamCdcSegmentHandlerTest
- Add cluster rebuild retry if CMS seeds can't be found
Copy link
Contributor

@jyothsnakonisa jyothsnakonisa left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good overall a few minor comments. I have noticed unused imports, methods in few places. Please remove all of them. Also there are formatting issues with the code in a few places, please correct them.

File cdcTempDir = new File(CDC_RAW_TEMP_DIR);
assertThat(cdcTempDir.exists()).isTrue();
int attempts = 10;
int attempts = 20;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I am assuming that you are increasing attempts, to surface the flakiness

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ends up that this was caused by some incorrect asynchronous tests and the underlying problem is now resolved so we can put it back to 10 (and, in theory, it may be unnecessary but haven’t had time to try to figure that out).

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

use loopAssert instead?

{
System.setProperty("cassandra.ring_delay_ms", "5000"); // down from 30s default; this change has no effect if GOSSIP feature is enabled
System.setProperty("cassandra.consistent.rangemovement", "false");
// System.setProperty("cassandra.consistent.rangemovement", "false");
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Commented code

return !isNullOrEmpty(string);
}

public static boolean contains(@NotNull String string, @NotNull String target) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I do not see a reason why we should have the StringUtils class.

For all other methods in this class can be removed and Guava StringUtils could be used, for this method can use String.contains() since the strings are not null.

If you think that the strings could be null, then there is a mismatch in the annotation for parameters

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Short answer to "why we should have the StringUtils class" is that we do not want to add the guava dependency in this package. Analytics depends on client-common

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

guava could cause some issues when pulling it into spark for example. client-common needs to be lean and have the minimum number of dependencies.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a javadoc for this method?

{
// Test case involves adding 2 nodes to a 10 node cluster (5 per DC)
// We intercept the bootstrap of nodes (11,12) to validate token ranges
// Test case involves adding 2 nodes to a 12 node cluster (5 per DC)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
// Test case involves adding 2 nodes to a 12 node cluster (5 per DC)
// Test case involves adding 2 nodes to a 10 node cluster (5 per DC)

This comment is copied in several places, please correct it everywhere

// Range 12
dc1Mapping.put(expectedRanges.get(11), Arrays.asList("127.0.0.5", "127.0.0.1", "127.0.0.3"));

return new HashMap<>()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Suggested change
return new HashMap<>()
return Map.of("datacenter1", dc1Mapping);

return tokens[nodeId - 1];
}

public void dup(int src, int dst)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

May be rename it to copyTokens?


import net.bytebuddy.implementation.bind.annotation.SuperCall;
import net.bytebuddy.implementation.bind.annotation.*;
// import net.bytebuddy.implementation.bind.annotation.SuperCall;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove the commented import

}
}

@SuppressWarnings("unused")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

how about adding private constructor for the utils class? few other helper classes also have this missing can you please update everywhere

import net.bytebuddy.implementation.MethodDelegation;
import net.bytebuddy.implementation.bind.annotation.RuntimeType;
import net.bytebuddy.implementation.bind.annotation.SuperCall;
import net.bytebuddy.pool.TypePool;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please remove unused imports.

new ByteBuddy().rebase(description, ClassFileLocator.ForClassLoader.of(cl))
.method(named("stream"))

if (installTcm(cl, typePool)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

curly bracket in the next line? I noticed a few in other places too. Can you please reformat them according to the coding conventions of this repo?

Copy link
Contributor

@frankgh frankgh left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

looks good in general

// the original cause, which we will process correctly in CreateSnapshotHandler
// if the exception was an IllegalArgumentException
IllegalArgumentException iex = ThrowableUtils.getCause(e, IllegalArgumentException.class);
if (iex != null) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

the formatting seems off in some places

Suggested change
if (iex != null) {
if (iex != null)
{

return !isNullOrEmpty(string);
}

public static boolean contains(@NotNull String string, @NotNull String target) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

guava could cause some issues when pulling it into spark for example. client-common needs to be lean and have the minimum number of dependencies.

return !isNullOrEmpty(string);
}

public static boolean contains(@NotNull String string, @NotNull String target) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add a javadoc for this method?

'--add-exports', 'java.management.rmi/com.sun.jmx.remote.internal.rmi=ALL-UNNAMED',
'--add-exports', 'java.rmi/sun.rmi.registry=ALL-UNNAMED',
'--add-exports', 'java.rmi/sun.rmi.server=ALL-UNNAMED',
'--add-exports', 'java.rmi/sun.rmi.transport=ALL-UNNAMED',
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it'd be nice to document which flags are needed for JDK 11 and which ones are needed for JDK 17

= new PeriodicTaskConfigurationImpl(true,
MillisecondBoundConfiguration.parse("50ms"),
MillisecondBoundConfiguration.parse("500ms"));
MillisecondBoundConfiguration.parse("5000ms"));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

any reason we reduce the periodic health check frequency?

import org.apache.cassandra.testing.CassandraIntegrationTest;
import org.apache.cassandra.testing.ConfigurableCassandraTestContext;
import org.apache.cassandra.testing.IClusterExtension;
import org.jetbrains.annotations.NotNull;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please remove unused imports

return;
}

if (installTcm(cl, typePool)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

formatting?


import io.vertx.junit5.VertxExtension;
import io.vertx.junit5.VertxTestContext;
import net.bytebuddy.implementation.bind.annotation.RuntimeType;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can you remove unused imports?

public class TestTokenSupplier implements TokenSupplier
{

private List<String>[] tokens;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

NIT: make it final

Suggested change
private List<String>[] tokens;
private final List<String>[] tokens;

public static void intercept(ClassLoader cl, Class<?> delegateClass)
{
TypePool typePool = TypePool.Default.of(cl);
if (installTCM(cl, typePool, delegateClass) || installPreTCM(cl, typePool, delegateClass)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

formatting

@yifan-c
Copy link
Contributor

yifan-c commented Dec 18, 2025

@jyothsnakonisa and @frankgh thanks for reviewing.
I am continuing Doug's work on this patch. A new PR is created with all the comment addressed and checkstyle errors fixed. Please review #305 instead.
I am closing this PR.

@yifan-c yifan-c closed this Dec 18, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants