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 2020/09/16 03:12:19 UTC

[GitHub] [cassandra] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

dcapwell commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r489135340



##########
File path: test/distributed/org/apache/cassandra/distributed/test/ClientNetworkStopStartTest.java
##########
@@ -0,0 +1,189 @@
+package org.apache.cassandra.distributed.test;
+
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Objects;
+
+import org.junit.Assert;
+import org.junit.Test;
+
+import com.datastax.driver.core.Session;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.ConsistencyLevel;
+import org.apache.cassandra.distributed.api.Feature;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.QueryResults;
+import org.apache.cassandra.distributed.api.SimpleQueryResult;
+import org.apache.cassandra.distributed.shared.AssertUtils;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.thrift.ColumnOrSuperColumn;
+import org.apache.cassandra.thrift.Mutation;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.thrift.TException;
+import org.hamcrest.BaseMatcher;
+import org.hamcrest.Description;
+
+public class ClientNetworkStopStartTest extends TestBaseImpl
+{
+    /**
+     * @see <a href="https://issues.apache.org/jira/browse/CASSANDRA-16127">CASSANDRA-16127</a>
+     */
+    @Test
+    public void stopStartThrift() throws IOException, TException
+    {
+        try (Cluster cluster = init(Cluster.build(1).withConfig(c -> c.with(Feature.NATIVE_PROTOCOL)).start()))
+        {
+            IInvokableInstance node = cluster.get(1);
+            assertBinaryRunning(node);
+            assertThriftRunning(node);
+            node.nodetoolResult("disablethrift").asserts().success();
+            assertBinaryRunning(node);
+            assertThriftNotRunning(node);
+            node.nodetoolResult("enablethrift").asserts().success();
+            assertBinaryRunning(node);
+            assertThriftRunning(node);
+
+            // now use it to make sure it still works!
+            cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, value int, PRIMARY KEY (pk))");
+
+            ThriftClientUtils.thriftClient(node, thrift -> {
+                thrift.set_keyspace(KEYSPACE);
+                Mutation mutation = new Mutation();
+                ColumnOrSuperColumn csoc = new ColumnOrSuperColumn();
+                Column column = new Column();
+                column.setName(CompositeType.build(ByteBufferUtil.bytes("value")));
+                column.setValue(ByteBufferUtil.bytes(0));
+                column.setTimestamp(System.currentTimeMillis());
+                csoc.setColumn(column);
+                mutation.setColumn_or_supercolumn(csoc);
+
+                thrift.batch_mutate(Collections.singletonMap(ByteBufferUtil.bytes(0),
+                                                             Collections.singletonMap("tbl", Arrays.asList(mutation))),
+                                    org.apache.cassandra.thrift.ConsistencyLevel.ALL);
+            });
+
+            SimpleQueryResult qr = cluster.coordinator(1).executeWithResult("SELECT * FROM " + KEYSPACE + ".tbl", ConsistencyLevel.ALL);
+            AssertUtils.assertRows(qr, QueryResults.builder().row(0, 0).build());
+        }
+    }
+
+    /**
+     * @see <a href="https://issues.apache.org/jira/browse/CASSANDRA-16127">CASSANDRA-16127</a>
+     */
+    @Test
+    public void stopStartNative() throws IOException
+    {
+        try (Cluster cluster = init(Cluster.build(1).withConfig(c -> c.with(Feature.NATIVE_PROTOCOL)).start()))
+        {
+            IInvokableInstance node = cluster.get(1);
+            assertBinaryRunning(node);
+            assertThriftRunning(node);
+            node.nodetoolResult("disablebinary").asserts().success();
+            assertBinaryNotRunning(node);
+            assertThriftRunning(node);
+            node.nodetoolResult("enablebinary").asserts().success();
+            assertBinaryRunning(node);
+            assertThriftRunning(node);
+
+            // now use it to make sure it still works!
+            cluster.schemaChange("CREATE TABLE " + KEYSPACE + ".tbl (pk int, value int, PRIMARY KEY (pk))");
+
+            try (com.datastax.driver.core.Cluster client = com.datastax.driver.core.Cluster.builder().addContactPoints(node.broadcastAddress().getAddress()).build();
+                 Session session = client.connect())
+            {
+                session.execute("INSERT INTO " + KEYSPACE + ".tbl (pk, value) VALUES (?, ?)", 0, 0);
+            }
+
+            SimpleQueryResult qr = cluster.coordinator(1).executeWithResult("SELECT * FROM " + KEYSPACE + ".tbl", ConsistencyLevel.ALL);
+            AssertUtils.assertRows(qr, QueryResults.builder().row(0, 0).build());
+        }
+    }
+
+    private static void assertBinaryRunning(IInvokableInstance node)
+    {
+        assertNodetoolStdout(node, "running", "not running", "statusbinary");
+    }
+
+    private static void assertBinaryNotRunning(IInvokableInstance node)
+    {
+        assertNodetoolStdout(node, "not running", null, "statusbinary");
+    }
+
+    private static void assertThriftRunning(IInvokableInstance node)
+    {
+        assertNodetoolStdout(node, "running", "not running", "statusthrift");
+    }
+
+    private static void assertThriftNotRunning(IInvokableInstance node)
+    {
+        assertNodetoolStdout(node, "not running", null, "statusthrift");
+    }
+
+    private static void assertNodetoolStdout(IInvokableInstance node, String expectedStatus, String notExpected, String... nodetool)
+    {
+        // without CASSANDRA-16057 need this hack
+        PrintStream previousStdout = System.out;
+        try
+        {
+            ByteArrayOutputStream out = new ByteArrayOutputStream();
+            PrintStream stdout = new PrintStream(out, true);
+            System.setOut(stdout);
+
+            node.nodetoolResult(nodetool).asserts().success();
+
+            stdout.flush();
+            String output = out.toString();
+            Assert.assertThat(output, new StringContains(expectedStatus));

Review comment:
       trunk uses a different version of junit and that has `CoreMatchers.containsString` but this one doesn't, so implemented my own.




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