You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by br...@apache.org on 2020/10/16 14:38:14 UTC

[cassandra] branch trunk updated: Nodetool ring testing

This is an automated email from the ASF dual-hosted git repository.

brandonwilliams pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new c54da9e  Nodetool ring testing
c54da9e is described below

commit c54da9e243f03daf92514db06d9ecfcb802e127e
Author: Bereng <be...@gmail.com>
AuthorDate: Thu Oct 8 03:35:20 2020 +0200

    Nodetool ring testing
    
    Patch by Berenguer Blasi, reviewed by brandonwilliams for
    CASSANDRA-16200
---
 build.xml                                          |  16 +-
 .../org/apache/cassandra/tools/nodetool/Ring.java  |   4 +-
 .../distributed/test/NodeToolRingTest.java         | 170 +++++++++++++++++++++
 .../org/apache/cassandra/tools/ToolRunner.java     |  88 +++++++++--
 4 files changed, 262 insertions(+), 16 deletions(-)

diff --git a/build.xml b/build.xml
index 44cd965..519d8d9 100644
--- a/build.xml
+++ b/build.xml
@@ -18,7 +18,9 @@
  ~ under the License.
  -->
 <project basedir="." default="jar" name="apache-cassandra"
-         xmlns:artifact="antlib:org.apache.maven.artifact.ant">
+         xmlns:artifact="antlib:org.apache.maven.artifact.ant"
+         xmlns:if="ant:if"
+         xmlns:unless="ant:unless">
     <property environment="env"/>
     <property file="build.properties" />
     <property file="build.properties.default" />
@@ -1610,7 +1612,8 @@
   -->
   <target name="testsome" depends="build-test" description="Execute specific unit tests" >
     <testmacro inputdir="${test.unit.src}" timeout="${test.timeout}">
-      <test name="${test.name}" methods="${test.methods}" outfile="build/test/output/TEST-${test.name}-${test.methods}"/>
+      <test unless:blank="${test.methods}" name="${test.name}" methods="${test.methods}" outfile="build/test/output/TEST-${test.name}-${test.methods}"/>
+      <test if:blank="${test.methods}" name="${test.name}" outfile="build/test/output/TEST-${test.name}"/>
       <jvmarg value="-Dlegacy-sstable-root=${test.data}/legacy-sstables"/>
       <jvmarg value="-Dinvalid-legacy-sstable-root=${test.data}/invalid-legacy-sstables"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
@@ -1624,7 +1627,8 @@
   -->
   <target name="long-testsome" depends="build-test" description="Execute specific long unit tests" >
     <testmacro inputdir="${test.long.src}" timeout="${test.long.timeout}">
-      <test name="${test.name}" methods="${test.methods}"/>
+      <test unless:blank="${test.methods}" name="${test.name}" methods="${test.methods}"/>
+      <test if:blank="${test.methods}" name="${test.name}"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=1000"/>
       <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
     </testmacro>
@@ -1635,7 +1639,8 @@
   -->
   <target name="burn-testsome" depends="build-test" description="Execute specific burn unit tests" >
     <testmacro inputdir="${test.burn.src}" timeout="${test.burn.timeout}">
-      <test name="${test.name}" methods="${test.methods}"/>
+      <test unless:blank="${test.methods}" name="${test.name}" methods="${test.methods}"/>
+      <test if:blank="${test.methods}" name="${test.name}"/>
       <jvmarg value="-Dlogback.configurationFile=test/conf/logback-burntest.xml"/>
     </testmacro>
   </target>
@@ -2127,7 +2132,8 @@
     -->
   <target name="test-jvm-dtest-some" depends="build-test" description="Execute some in-jvm dtests">
     <testmacro inputdir="${test.distributed.src}" timeout="${test.distributed.timeout}" forkmode="once" showoutput="true">
-      <test name="${test.name}" methods="${test.methods}" outfile="build/test/output/TEST-${test.name}-${test.methods}"/>
+      <test unless:blank="${test.methods}" name="${test.name}" methods="${test.methods}" outfile="build/test/output/TEST-${test.name}-${test.methods}"/>
+      <test if:blank="${test.methods}" name="${test.name}" outfile="build/test/output/TEST-${test.name}"/>
       <jvmarg value="-Dlogback.configurationFile=test/conf/logback-dtest.xml"/>
       <jvmarg value="-Dcassandra.ring_delay_ms=10000"/>
       <jvmarg value="-Dcassandra.tolerate_sstable_size=true"/>
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Ring.java b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
index 8b23717..134c85d 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Ring.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Ring.java
@@ -244,7 +244,9 @@ public class Ring extends NodeToolCmd
 
         for (HostStatWithPort stat : hoststats)
         {
-            tokens.addAll(endpointsToTokens.get(stat.endpoint.toString()));
+            // Remove extra '/' from address
+            String addressNPort = stat.endpointWithPort.toString().replaceAll("^/", "");
+            tokens.addAll(endpointsToTokens.get(addressNPort));
             lastToken = tokens.get(tokens.size() - 1);
         }
 
diff --git a/test/distributed/org/apache/cassandra/distributed/test/NodeToolRingTest.java b/test/distributed/org/apache/cassandra/distributed/test/NodeToolRingTest.java
new file mode 100644
index 0000000..a248110
--- /dev/null
+++ b/test/distributed/org/apache/cassandra/distributed/test/NodeToolRingTest.java
@@ -0,0 +1,170 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.distributed.test;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.ICluster;
+import org.apache.cassandra.tools.ToolRunner;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.assertj.core.api.Assertions;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class NodeToolRingTest extends TestBaseImpl
+{
+    private static ICluster cluster;
+
+    @Before
+    public void setupEnv() throws IOException
+    {
+        if (cluster == null)
+            cluster = init(builder().withNodes(1).start());
+    }
+
+    @AfterClass
+    public static void teardownEnv() throws Exception
+    {
+        cluster.close();
+    }
+
+    @Test
+    public void testWrongArgFailsAndPrintsHelp()
+    {
+        ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "--wrongarg", "ring");
+        Assertions.assertThat(tool.getStdout()).containsIgnoringCase("nodetool help");
+        assertEquals(1, tool.getExitCode());
+        assertTrue(tool.getCleanedStderr().isEmpty());
+    }
+
+    @Test
+    public void testMaybeChangeDocs()
+    {
+        // If you added, modified options or help, please update docs if necessary
+
+        ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "help", "ring");
+        String help = "NAME\n" + "        nodetool ring - Print information about the token ring\n"
+                      + "\n"
+                      + "SYNOPSIS\n"
+                      + "        nodetool [(-h <host> | --host <host>)] [(-p <port> | --port <port>)]\n"
+                      + "                [(-pp | --print-port)] [(-pw <password> | --password <password>)]\n"
+                      + "                [(-pwf <passwordFilePath> | --password-file <passwordFilePath>)]\n"
+                      + "                [(-u <username> | --username <username>)] ring [(-r | --resolve-ip)]\n"
+                      + "                [--] [<keyspace>]\n"
+                      + "\n"
+                      + "OPTIONS\n"
+                      + "        -h <host>, --host <host>\n"
+                      + "            Node hostname or ip address\n"
+                      + "\n"
+                      + "        -p <port>, --port <port>\n"
+                      + "            Remote jmx agent port number\n"
+                      + "\n"
+                      + "        -pp, --print-port\n"
+                      + "            Operate in 4.0 mode with hosts disambiguated by port number\n"
+                      + "\n"
+                      + "        -pw <password>, --password <password>\n"
+                      + "            Remote jmx agent password\n"
+                      + "\n"
+                      + "        -pwf <passwordFilePath>, --password-file <passwordFilePath>\n"
+                      + "            Path to the JMX password file\n"
+                      + "\n"
+                      + "        -r, --resolve-ip\n"
+                      + "            Show node domain names instead of IPs\n"
+                      + "\n"
+                      + "        -u <username>, --username <username>\n"
+                      + "            Remote jmx agent username\n"
+                      + "\n"
+                      + "        --\n"
+                      + "            This option can be used to separate command-line options from the\n"
+                      + "            list of argument, (useful when arguments might be mistaken for\n"
+                      + "            command-line options\n"
+                      + "\n"
+                      + "        <keyspace>\n"
+                      + "            Specify a keyspace for accurate ownership information (topology\n"
+                      + "            awareness)\n"
+                      + "\n"
+                      + "\n";
+        Assertions.assertThat(tool.getStdout()).isEqualTo(help);
+    }
+
+    @Test
+    public void testRing()
+    {
+        ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "ring");
+        Assertions.assertThat(tool.getStdout())
+                  .contains("Datacenter: datacenter0")
+                  .contains("Address    Rack        Status State   Load            Owns                Token")
+                  .contains("127.0.0.1  rack0       Up     Normal")
+                  .contains("100.00%             9223372036854775807");
+        assertEquals(0, tool.getExitCode());
+        assertTrue(tool.getCleanedStderr().isEmpty());
+    }
+
+    @Test
+    public void testRingPrintPort()
+    {
+        Arrays.asList("-pp", "--print-port").forEach(arg -> {
+            ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), arg, "ring");
+            Assertions.assertThat(tool.getStdout())
+                      .contains("Datacenter: datacenter0")
+                      .contains("Address         Rack        Status State   Load            Owns                Token")
+                      .contains("Unknown")
+                      .contains("100.00%             9223372036854775807");
+            assertEquals(0, tool.getExitCode());
+            assertTrue(tool.getCleanedStderr().isEmpty());
+        });
+    }
+
+    @Test
+    public void testRingResolve()
+    {
+        Arrays.asList("-r", "--resolve-ip").forEach(arg -> {
+            ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "ring", arg);
+            Assertions.assertThat(tool.getStdout())
+                      .contains("Datacenter: datacenter0")
+                      .contains("Address    Rack        Status State   Load            Owns                Token")
+                      .contains("localhost  rack0       Up     Normal")
+                      .contains("100.00%             9223372036854775807");
+            assertEquals(0, tool.getExitCode());
+            assertTrue(tool.getCleanedStderr().isEmpty());
+        });
+    }
+
+    @Test
+    public void testRingKeyspace()
+    {
+        // Bad KS
+        ToolResult tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "ring", "mockks");
+        Assertions.assertThat(tool.getStdout()).contains("The keyspace mockks, does not exist");
+        assertEquals(0, tool.getExitCode());
+        assertTrue(tool.getCleanedStderr().isEmpty());
+
+        // Good KS
+        tool = ToolRunner.invokeNodetoolJvmDtest(cluster.get(1), "ring", "system_schema");
+        Assertions.assertThat(tool.getStdout()).contains("Datacenter: datacenter0");
+        assertEquals(0, tool.getExitCode());
+        assertTrue(tool.getCleanedStderr().isEmpty());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/tools/ToolRunner.java b/test/unit/org/apache/cassandra/tools/ToolRunner.java
index 98fdaed..bcc0179 100644
--- a/test/unit/org/apache/cassandra/tools/ToolRunner.java
+++ b/test/unit/org/apache/cassandra/tools/ToolRunner.java
@@ -33,6 +33,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.concurrent.CompletableFuture;
+import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
 import com.google.common.base.Throwables;
@@ -42,7 +43,11 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.distributed.api.IInstance;
+import org.apache.cassandra.distributed.api.NodeToolResult;
 import org.apache.cassandra.tools.OfflineToolUtils.SystemExitException;
+import org.apache.cassandra.utils.Pair;
+import org.assertj.core.util.Lists;
 
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -238,6 +243,51 @@ public class ToolRunner
         allArgs.add(klass);
         allArgs.addAll(Arrays.asList(args));
         
+        Supplier<Integer> runMe = new Supplier<Integer>()
+        {
+            @Override
+            public Integer get()
+            {
+                return runClassAsTool(klass, args);
+            }
+        };
+
+        Pair<Integer, ToolResult> res = invokeSupplier(runMe, stdin);
+        return new ToolResult(allArgs,
+                              res.right.getExitCode() == -1 ? -1 : res.left,
+                              res.right.getStdout(),
+                              res.right.getStderr(),
+                              res.right.getException());
+
+    }
+    
+    public static ToolResult invokeNodetoolJvmDtest(IInstance node, String... args)
+    {
+        Supplier<NodeToolResult> runMe = new Supplier<NodeToolResult>()
+        {
+            @Override
+            public NodeToolResult get()
+            {
+                return node.nodetoolResult(args);
+            }
+        };
+
+        Pair<NodeToolResult, ToolResult> res = invokeSupplier(runMe);
+        return new ToolResult(Arrays.asList(args),
+                              res.left,
+                              res.right.getExitCode() == -1 ? -1 : res.left.getRc(),
+                              res.right.getStdout(),
+                              res.right.getStderr(),
+                              res.right.getException());
+    }
+
+    public static <T> Pair<T, ToolResult> invokeSupplier(Supplier<T> runMe)
+    {
+        return invokeSupplier(runMe, null);
+    }
+
+    public static <T> Pair<T, ToolResult> invokeSupplier(Supplier<T> runMe, InputStream stdin)
+    {
         PrintStream originalSysOut = System.out;
         PrintStream originalSysErr = System.err;
         InputStream originalSysIn = System.in;
@@ -248,23 +298,24 @@ public class ToolRunner
 
         System.setIn(stdin == null ? originalSysIn : stdin);
 
-        try (PrintStream newOut = new PrintStream(out);
-             PrintStream newErr = new PrintStream(err))
+        T res = null;
+        try(PrintStream newOut = new PrintStream(out); PrintStream newErr = new PrintStream(err))
         {
             System.setOut(newOut);
             System.setErr(newErr);
-            int rc = runClassAsTool(klass, args);
+            res = runMe.get();
             out.flush();
             err.flush();
-            return new ToolResult(allArgs, rc, out.toString(), err.toString(), null);
+            return Pair.create(res, new ToolResult(Lists.emptyList(), 0, out.toString(), err.toString(), null));
         }
-        catch (Exception e)
+        catch(Exception e)
         {
-            return new ToolResult(allArgs,
-                                  -1,
-                                  out.toString(),
-                                  err.toString() + "\n" + Throwables.getStackTraceAsString(e),
-                                  e);
+            return Pair.create(res,
+                               new ToolResult(Lists.emptyList(),
+                                              -1,
+                                              out.toString(),
+                                              err.toString() + "\n" + Throwables.getStackTraceAsString(e),
+                                              e));
         }
         finally
         {
@@ -286,6 +337,7 @@ public class ToolRunner
         private final String stdout;
         private final String stderr;
         private final Exception e;
+        private final NodeToolResult ntRes;
 
         private ToolResult(List<String> allArgs, int exitCode, String stdout, String stderr, Exception e)
         {
@@ -294,6 +346,22 @@ public class ToolRunner
             this.stdout = stdout;
             this.stderr = stderr;
             this.e = e;
+            this.ntRes = null;
+        }
+        
+        private ToolResult(List<String> allArgs, NodeToolResult ntRes, int exitCode, String stdout, String stderr, Exception e)
+        {
+            this.allArgs = allArgs;
+            this.exitCode = exitCode;
+            this.stdout = stdout;
+            this.stderr = stderr;
+            this.e = e;
+            this.ntRes = ntRes;
+        }
+        
+        public NodeToolResult getNodeToolResult()
+        {
+            return ntRes;
         }
 
         public int getExitCode()


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