michaelsembwever commented on code in PR #3619:
URL: https://github.com/apache/cassandra/pull/3619#discussion_r1806152791
##########
test/distributed/org/apache/cassandra/distributed/test/GossipTest.java:
##########
@@ -141,6 +151,129 @@ public void nodeDownDuringMove() throws Throwable
}
}
+ public static class BBGossiper
+ {
+ public static final AtomicBoolean disableSendGossip = new
AtomicBoolean();
+ public static final AtomicBoolean blockGossipStageQueue = new
AtomicBoolean();
+ public static void install(ClassLoader cl, Integer i)
+ {
+ new ByteBuddy().rebase(Gossiper.class)
+ .method(named("sendGossip"))
+ .intercept(MethodDelegation.to(BBGossiper.class))
+ .make()
+ .load(cl, ClassLoadingStrategy.Default.INJECTION);
+ }
+
+ public static boolean sendGossip(Message<GossipDigestSyn> message,
Set<InetAddressAndPort> epSet, @SuperCall Callable<Boolean> zuper) throws
Exception
+ {
+ if (disableSendGossip.get())
+ {
+ logger.info("Send gossip disabled");
+ return true;
+ }
+ return zuper.call();
+ }
+ }
+
+ @Test
+ public void
testBusyGossipBusyShouldNotCreateParitialGossipInfoOnOtherNodes() throws
Exception
+ {
+ int originalNodeCount = 2;
+ int expandedNodeCount = originalNodeCount + 1;
+ ExecutorService es = Executors.newFixedThreadPool(1);
+ // set 5s as ring delay so fatclient will be removed soon
+ System.setProperty("cassandra.ring_delay_ms", "5000");
+ try (Cluster cluster = builder().withNodes(originalNodeCount)
+
.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+
.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount,
"dc0", "rack0"))
+ .withConfig(config ->
config.with(NETWORK, GOSSIP))
+
.withInstanceInitializer(BBGossiper::install)
+ .start())
+ {
+ IInstanceConfig config = cluster.newInstanceConfig();
+ config.set("auto_bootstrap", true);
+ IInvokableInstance newInstance = cluster.bootstrap(config);
+ withProperty("cassandra.join_ring", false, () ->
newInstance.startup(cluster));
+
+ // wait for the new node to show in existings gossip map, HOST_ID
should be there
+ InetSocketAddress newNodeAddress = newInstance.broadcastAddress();
+ for (int i = 1 ; i <= originalNodeCount ; ++i)
+ {
+ cluster.get(i).acceptsOnInstance((InetSocketAddress address)
-> {
+ EndpointState ep;
+ InetAddressAndPort endpoint =
toCassandraInetAddressAndPort(address);
+ while (null == (ep =
Gossiper.instance.getEndpointStateForEndpoint(endpoint))
+ || ep.getApplicationState(ApplicationState.HOST_ID)
== null)
+
LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(10L));
+ }).accept(newNodeAddress);
+ }
+
+ Runnable busyNewNode = newInstance.runsOnInstance(
+ ()-> {
+ // below code will trigger the simulation for large C*
cluster, this new node may only send gossip to some nodes
+ // here we disable sned gossip so that no other nodes will be
contacted
Review Comment:
i think this can happen on small clusters too, given any network issues…
--
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]