You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/11/29 21:17:54 UTC

[GitHub] [cassandra] xgerman opened a new pull request, #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

xgerman opened a new pull request, #2033:
URL: https://github.com/apache/cassandra/pull/2033

   This changes the creation of the default role to use write time 0 - thus ensuring, if it runs twice in a cluster, repairs won't overwrite the password a user might have assigned on a different node to the cassandra role.
   
   Thanks for sending a pull request! Here are some tips if you're new here:
    
    * Ensure you have added or run the [appropriate tests](https://cassandra.apache.org/_/development/testing.html) for your PR.
    * Be sure to keep the PR description updated to reflect all changes.
    * Write your PR title to summarize what this PR proposes.
    * If possible, provide a concise example to reproduce the issue for a faster review.
    * Read our [contributor guidelines](https://cassandra.apache.org/_/development/index.html)
    * If you're making a documentation change, see our [guide to documentation contribution](https://cassandra.apache.org/_/development/documentation.html)
    
   Commit messages should follow the following format:
   
   ```
   <One sentence description, usually Jira title or CHANGES.txt summary>
   
   <Optional lengthier description (context on patch)>
   
   patch by <Authors>; reviewed by <Reviewers> for CASSANDRA-#####
   
   Co-authored-by: Name1 <email1>
   Co-authored-by: Name2 <email2>
   
   ```
   
   The [Cassandra Jira](https://issues.apache.org/jira/projects/CASSANDRA/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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038808235


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException

Review Comment:
   I would rename this test method. I do not think it appropriately reflects what this test is doing. It should be like "testWhat"



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807559


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)

Review Comment:
   formatting plus missing atMost



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807974


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup
+            Long time2 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+            // we don't do this here but if the user changed the Cassandra user password it will be (read) reapired if the second node has a later
+            // write timsestamp - check that this is not the case
+            assertTrue(time1 >= time2);

Review Comment:
   why there is `>=` instead of just `=`? How can ever be `time1` bigger than `time2` anyway?



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic closed pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials
URL: https://github.com/apache/cassandra/pull/2033


-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807219


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)

Review Comment:
   `(Long) cluster`



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807028


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)

Review Comment:
   this does not have any explicit "timeout", does it? what about adding `atMost`?
   
   also, formatting would be nice to have like this:
   
   ````
   await().pollDelay(1, SECONDS)
          .pollInterval(1, SECONDS)
          .atMost(...)
          .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
   ````



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038604417


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -52,4 +64,60 @@ public void authSetupIsCalledAfterStartup() throws IOException
             assertTrue(setupCalled);
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            boolean defaultRoleSetup = cluster.get(1).callOnInstance(() -> {
+                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
+                long start = System.nanoTime();
+                while (!CassandraRoleManager.hasExistingRoles() && System.nanoTime() - start < maxWait)
+                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
+
+                return CassandraRoleManager.hasExistingRoles();
+            });
+            assertTrue(defaultRoleSetup);
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1).execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",

Review Comment:
   formating might be better, `execute` bellow `coordinator`, `ConsistencyLevel` below `SELECT` .... you feel me



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807790


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup
+            Long time2 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+            // we don't do this here but if the user changed the Cassandra user password it will be (read) reapired if the second node has a later
+            // write timsestamp - check that this is not the case
+            assertTrue(time1 >= time2);
+        }
+    }
+

Review Comment:
   redundant new lines



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807611


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup

Review Comment:
   titme -> time
   frome -> from
   second -> the second



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807185


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)

Review Comment:
   ````
   await().pollDelay(1, SECONDS)
          .pollInterval(1, SECONDS)
          .atMost(12, SECONDS)
          .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
   ````



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] xgerman commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
xgerman commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1039805801


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup
+            Long time2 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+            // we don't do this here but if the user changed the Cassandra user password it will be (read) reapired if the second node has a later
+            // write timsestamp - check that this is not the case
+            assertTrue(time1 >= time2);

Review Comment:
   The error case is node2 overwriting node1's entry - that's why I made it `>=` but you are right it'd now always 0 so `==` is more appropriate



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] xgerman commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
xgerman commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1039798035


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)

Review Comment:
   Default for atMost is 10s but I will make it explicit



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807639


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup
+            Long time2 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+            // we don't do this here but if the user changed the Cassandra user password it will be (read) reapired if the second node has a later

Review Comment:
   reapired -> repaired
   timesestamp -> timestamp



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807974


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS).atMost(12, SECONDS)
+                .until(() -> instance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            //get the time from the the first role setup
+            Long time1 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+
+            IInstanceConfig config = cluster.newInstanceConfig();
+            // set boostrap to false to simulate a seed node
+            config.set("auto_bootstrap", false);
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            withProperty(BOOTSTRAP_SCHEMA_DELAY_MS.getKey(), Integer.toString(90 * 1000),
+                         () -> withProperty("cassandra.join_ring", false, () -> newInstance.startup(cluster)));
+            newInstance.nodetoolResult("join").asserts().success();
+
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                     .until(() -> newInstance.callOnInstance(() -> CassandraRoleManager.hasExistingRoles()));
+
+            // get write titme frome second role setup
+            Long time2 = (Long)cluster.coordinator(1)
+                                      .execute("SELECT WRITETIME (salted_hash) from system_auth.roles where role = 'cassandra'",
+                                               ConsistencyLevel.ONE)[0][0];
+            // we don't do this here but if the user changed the Cassandra user password it will be (read) reapired if the second node has a later
+            // write timsestamp - check that this is not the case
+            assertTrue(time1 >= time2);

Review Comment:
   why is there `>=` instead of just `==`? How can ever be `time1` bigger than `time2` anyway?



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038807252


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -41,15 +56,53 @@ public void authSetupIsCalledAfterStartup() throws IOException
     {
         try (Cluster cluster = Cluster.build().withNodes(1).start())
         {
-            boolean setupCalled = cluster.get(1).callOnInstance(() -> {
-                long maxWait = TimeUnit.NANOSECONDS.convert(10, TimeUnit.SECONDS);
-                long start = System.nanoTime();
-                while (!StorageService.instance.authSetupCalled() && System.nanoTime() - start < maxWait)
-                    Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
-
-                return StorageService.instance.authSetupCalled();
-            });
-            assertTrue(setupCalled);
+            IInvokableInstance instance = cluster.get(1);
+            await().pollDelay(1, SECONDS).pollInterval(1, SECONDS)
+                   .until(() -> instance.callOnInstance(() -> StorageService.instance.authSetupCalled()));
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice

Review Comment:
   Sows -> Shows



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org


[GitHub] [cassandra] smiklosovic commented on a diff in pull request #2033: CASSANDRA-12525 When adding new nodes to a cluster which has authentication enabled, we end up losing cassandra user's current crendentials and they get reverted back to default cassandra/cassandra crendetials

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on code in PR #2033:
URL: https://github.com/apache/cassandra/pull/2033#discussion_r1038603599


##########
test/distributed/org/apache/cassandra/distributed/test/AuthTest.java:
##########
@@ -52,4 +64,60 @@ public void authSetupIsCalledAfterStartup() throws IOException
             assertTrue(setupCalled);
         }
     }
+
+    /**
+     * Sows that in some circumstances CassandraRoleManager will create the cassandra role twice
+     */
+    @Test
+    public void authSetupIsCalledTwice() throws IOException
+    {
+        try (Cluster cluster = builder().withNodes(1)
+                                        .withTokenSupplier(TokenSupplier.evenlyDistributedTokens(3))
+                                        .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(3, "dc0", "rack0"))
+                                        .withConfig(config -> config.with(NETWORK, GOSSIP).set("authenticator", "PasswordAuthenticator"))
+                                        .start())
+        {
+            boolean defaultRoleSetup = cluster.get(1).callOnInstance(() -> {

Review Comment:
   This whole logic can be replaced by Awaitility library. We would just poll for 10s to get true on hasExistingRoles and it would hide all lowlevel complexity here.



-- 
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: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org