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 2021/04/04 22:32:03 UTC

[GitHub] [cassandra] fibersel opened a new pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

fibersel opened a new pull request #952:
URL: https://github.com/apache/cassandra/pull/952


   This pull request closes [CASSANDRA-14582](https://issues.apache.org/jira/browse/CASSANDRA-14582)
   
   @pauloricardomg Can you guide me please?
   Code in this PR is incomplete.


-- 
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.

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] pauloricardomg commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r607462357



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +97,38 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+    	int nodeCount = 3;
+    	try (Cluster cluster = builder().withNodes(nodeCount).start()) {
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.config().set("cassandra.cassandra.host_id_first_boot", String.valueOf(i));
+    		} 		
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));
+    		}
+    	
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.shutdown().wait();
+    		}
+    		
+    		
+    		cluster.startup();
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));

Review comment:
       yes, it's on `system.local` table.




-- 
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.

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] fibersel edited a comment on pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel edited a comment on pull request #952:
URL: https://github.com/apache/cassandra/pull/952#issuecomment-817385584


   > > About last one.If I provide invalid host_id, cassandra throws an exception and does not start.So, how can I catch that exception?
   > 
   > How about using [AssertJ exception assertion](https://www.baeldung.com/assertj-exception-assertion) ?
   
   The main problem is that this exception is thrown in another thread,that runs Cassandra node,so I can’t just catch it


-- 
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.

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 #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic closed pull request #952:
URL: https://github.com/apache/cassandra/pull/952


   


-- 
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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r611421184



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -110,15 +112,31 @@ public void hostIdOverrideTest() throws Throwable
             populate(cluster,0, 100);
             IInstanceConfig config = cluster.newInstanceConfig();
             IInvokableInstance newInstance = cluster.bootstrap(config);
+
+            
             withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
                          () -> newInstance.startup(cluster));
 
             newInstance.shutdown().get();
             newInstance.startup();
             String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
             Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
-    	}
-    	
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",
+                    () -> newInstance.startup());
+            host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // check if another cluster member is aware of proper host id
+            IInvokableInstance member = cluster.get(1);
+            Object[][] result = member.executeInternal("SELECT host_id FROM system.peers");

Review comment:
       done, now both tables are checked




-- 
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.

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 change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r638076473



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);
+            IInstanceConfig config = cluster.newInstanceConfig();
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            String peer = newInstance.broadcastAddress().getHostString();
+            
+            withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
+                         () -> newInstance.startup(cluster));
+            String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",

Review comment:
       @fibersel just call it "invalid-host-id"
   
   edit: aha I got what you mean here ... so just call it "setting-new-host-id-first-boot" or something more descriptive




-- 
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.

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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r611421285



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -110,15 +112,31 @@ public void hostIdOverrideTest() throws Throwable
             populate(cluster,0, 100);
             IInstanceConfig config = cluster.newInstanceConfig();
             IInvokableInstance newInstance = cluster.bootstrap(config);
+
+            
             withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
                          () -> newInstance.startup(cluster));
 
             newInstance.shutdown().get();

Review comment:
       removed




-- 
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.

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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r611421638



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +95,32 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable

Review comment:
       added additional test with assertion




-- 
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.

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] fibersel commented on pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on pull request #952:
URL: https://github.com/apache/cassandra/pull/952#issuecomment-817385584


   > > About last one.If I provide invalid host_id, cassandra throws an exception and does not start.So, how can I catch that exception?
   > 
   > How about using [AssertJ exception assertion](https://www.baeldung.com/assertj-exception-assertion) ?
   
   The main problem is that this exception is thrown in another thread,that runs Cassandra node.


-- 
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.

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] pauloricardomg commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r607415877



##########
File path: src/java/org/apache/cassandra/db/SystemKeyspace.java
##########
@@ -1114,7 +1114,8 @@ public static UUID getLocalHostId()
             return result.one().getUUID("host_id");
 
         // ID not found, generate a new one, persist, and then return it.
-        UUID hostId = UUID.randomUUID();
+        UUID hostId = UUID.fromString(System.getProperty("cassandra.cassandra.host_id_first_boot", UUID.randomUUID().toString()));

Review comment:
       should be `cassandra.host_id_first_boot`

##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +97,38 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+    	int nodeCount = 3;
+    	try (Cluster cluster = builder().withNodes(nodeCount).start()) {
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.config().set("cassandra.cassandra.host_id_first_boot", String.valueOf(i));
+    		} 		
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));
+    		}
+    	
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.shutdown().wait();
+    		}
+    		
+    		
+    		cluster.startup();
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));

Review comment:
       The test looks good, but this only checks that the `cassandra.host_id_first_boot` property was defined, but does not verify that the node actually picked the provided ID to use as host identifier. How is this value being used? Is there any other way you can check that the cassandra node is using the property?




-- 
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.

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] pauloricardomg commented on pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on pull request #952:
URL: https://github.com/apache/cassandra/pull/952#issuecomment-817403457


   I was able to get the exception with:
   ```java
               withProperty("cassandra.host_id_first_boot", "BAD_HOST_ID",
                            () -> {
                                try {
                                    newInstance.startup(cluster);
                                } catch (Exception e) {
                                    fail("SHOULD NOT THROW EXCEPTION");
                                }
                            });
   ``` 
   
   With this you can use assertJ to verify the exception.


-- 
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.

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 change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r638076199



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))

Review comment:
       @fibersel nit, could you please just do it like "dc1" and "rack1" ? I do not remember I have ever seen "dc0" and "rack0" ever (nor in tests ...)




-- 
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.

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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r607427606



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +97,38 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+    	int nodeCount = 3;
+    	try (Cluster cluster = builder().withNodes(nodeCount).start()) {
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.config().set("cassandra.cassandra.host_id_first_boot", String.valueOf(i));
+    		} 		
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));
+    		}
+    	
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.shutdown().wait();
+    		}
+    		
+    		
+    		cluster.startup();
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));

Review comment:
       Can I retrieve Host ID from cassandra node?




-- 
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.

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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r608159094



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +97,38 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+    	int nodeCount = 3;
+    	try (Cluster cluster = builder().withNodes(nodeCount).start()) {
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.config().set("cassandra.cassandra.host_id_first_boot", String.valueOf(i));
+    		} 		
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));
+    		}
+    	
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			instance.shutdown().wait();
+    		}
+    		
+    		
+    		cluster.startup();
+    		
+    		for (int i = 1; i <= cluster.size(); ++i) {
+    			IInvokableInstance instance = cluster.get(i);
+    			Assert.assertEquals(instance.config().getString("cassandra.cassandra.host_id_first_boot"), String.valueOf(i));

Review comment:
       done!




-- 
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.

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] pauloricardomg commented on pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on pull request #952:
URL: https://github.com/apache/cassandra/pull/952#issuecomment-817388717


   > The main problem is that this exception is thrown in another thread,that runs Cassandra node,so I can’t just catch it
   
   Maybe you can check the `system.local` table that the `host_id=null` and `bootstrap_state=null`, indicating the node did not start successfully? Also check on `newInstance.logs()` that an exception was thrown.


-- 
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.

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] fibersel commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
fibersel commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r611042944



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +95,32 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable

Review comment:
       I have done two first points.
   
   About last one.If I provide invalid host_id, cassandra throws an exception and does not start.So, how can I catch that exception?




-- 
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.

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 change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r638076473



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);
+            IInstanceConfig config = cluster.newInstanceConfig();
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            String peer = newInstance.broadcastAddress().getHostString();
+            
+            withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
+                         () -> newInstance.startup(cluster));
+            String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",

Review comment:
       @fibersel just call it "invalid-host-id"




-- 
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.

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 change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r638075093



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +95,32 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable

Review comment:
       hey @fibersel , if something is done, just click the "resolved" button, it just makes less visual clutter ...




-- 
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.

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 change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
smiklosovic commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r638077528



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);

Review comment:
       @fibersel , do you use the correct formatter? I know it is a detail but there should be space after a comma (before "0").




-- 
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.

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] pauloricardomg commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r637566154



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);
+            IInstanceConfig config = cluster.newInstanceConfig();
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            String peer = newInstance.broadcastAddress().getHostString();
+            
+            withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",

Review comment:
       Maybe extract `06fc931f-33b5-4e22-0001-000000000001` to constant since it's used in multiple places?

##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);
+            IInstanceConfig config = cluster.newInstanceConfig();
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            String peer = newInstance.broadcastAddress().getHostString();
+            
+            withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
+                         () -> newInstance.startup(cluster));
+            String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",
+                    () -> newInstance.startup());
+            host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // check if another cluster member is aware of proper host id
+            IInvokableInstance member = cluster.get(1);
+            Object[][] result = member.executeInternal("SELECT host_id FROM system.peers WHERE peer = '" + peer + "'");
+            Assert.assertEquals(result[0][0].toString(), "06fc931f-33b5-4e22-0001-000000000001");
+            
+            result = member.executeInternal("SELECT host_id FROM system.peers_v2 WHERE peer = '" + peer + "'");
+            Assert.assertEquals(result[0][0].toString(), "06fc931f-33b5-4e22-0001-000000000001");
+    	}	
+    }
+    
 
+    @Test
+    public void invalidHostIdTest() throws Throwable {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) {
+    	
+            populate(cluster,0, 100);

Review comment:
       I don't think this is needed.

##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,7 +96,69 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable
+    {
+        int originalNodeCount = 2;
+        int expandedNodeCount = originalNodeCount + 1;
+        try (Cluster cluster = builder().withNodes(originalNodeCount)
+    									.withTokenSupplier(TokenSupplier.evenlyDistributedTokens(expandedNodeCount))
+    									.withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(expandedNodeCount, "dc0", "rack0"))
+    									.withConfig(config -> config.with(NETWORK, GOSSIP))
+    									.start()) 
+    	{
+            populate(cluster,0, 100);
+            IInstanceConfig config = cluster.newInstanceConfig();
+            IInvokableInstance newInstance = cluster.bootstrap(config);
+            String peer = newInstance.broadcastAddress().getHostString();
+            
+            withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
+                         () -> newInstance.startup(cluster));
+            String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",
+                    () -> newInstance.startup());
+            host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");

Review comment:
       The project is moving from Junit to AssertJ-style fluent assertions, so we encourage using AssertJ instead for new tests moving forward. See [this link](https://joel-costigliola.github.io/assertj/assertj-core-converting-junit-assertions-to-assertj.html) for reference.




-- 
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.

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] pauloricardomg commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r610958750



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -95,6 +95,32 @@ public void autoBootstrapTest() throws Throwable
                 Assert.assertEquals("Node " + e.getKey() + " has incorrect row state", e.getValue().longValue(), 100L);
         }
     }
+    
+    @Test
+    public void hostIdOverrideTest() throws Throwable

Review comment:
       This is looking great, good job! I think we can still improve it:
   a) check that original node contains host id of second node on `system.peers` table
   b) after the node starts with the provided `host_id_first_boot`, stop the node and start with a different `host_id_first_boot` - check the id should not change since only the first one should be used.
   c) what happens when `host_id_first_boot` is invalid? node startup should fail with error message and we should test this
   




-- 
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.

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] pauloricardomg commented on a change in pull request #952: [CASSANDRA-14582] Add a system property to set the cassandra hostId if not yet initialized

Posted by GitBox <gi...@apache.org>.
pauloricardomg commented on a change in pull request #952:
URL: https://github.com/apache/cassandra/pull/952#discussion_r611253637



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -110,15 +112,31 @@ public void hostIdOverrideTest() throws Throwable
             populate(cluster,0, 100);
             IInstanceConfig config = cluster.newInstanceConfig();
             IInvokableInstance newInstance = cluster.bootstrap(config);
+
+            
             withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
                          () -> newInstance.startup(cluster));
 
             newInstance.shutdown().get();

Review comment:
       no need to stop instance here before doing assertion

##########
File path: test/distributed/org/apache/cassandra/distributed/test/ring/BootstrapTest.java
##########
@@ -110,15 +112,31 @@ public void hostIdOverrideTest() throws Throwable
             populate(cluster,0, 100);
             IInstanceConfig config = cluster.newInstanceConfig();
             IInvokableInstance newInstance = cluster.bootstrap(config);
+
+            
             withProperty("cassandra.host_id_first_boot", "06fc931f-33b5-4e22-0001-000000000001",
                          () -> newInstance.startup(cluster));
 
             newInstance.shutdown().get();
             newInstance.startup();
             String host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
             Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
-    	}
-    	
+            
+            // stop and run with another host id, check if value is being cached
+            newInstance.shutdown().get();
+            withProperty("cassandra.host_id_first_boot", "stoopid-host-id",
+                    () -> newInstance.startup());
+            host_id = newInstance.executeInternal("SELECT host_id FROM system.local")[0][0].toString();
+            Assert.assertEquals(host_id, "06fc931f-33b5-4e22-0001-000000000001");
+            
+            // check if another cluster member is aware of proper host id
+            IInvokableInstance member = cluster.get(1);
+            Object[][] result = member.executeInternal("SELECT host_id FROM system.peers");

Review comment:
       Why are you querying all peers here? You can restrict the query with `peer = newInstance.broadcastAddress()` and use `assertEquals`. You should also check the table `peers_v2` which is the new table for peers.




-- 
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.

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