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 01:08:33 UTC

[GitHub] [cassandra] dcapwell opened a new pull request #752: Bug/cassandra 16127 3.11

dcapwell opened a new pull request #752:
URL: https://github.com/apache/cassandra/pull/752


   


----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r492329110



##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -802,4 +805,28 @@ private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn
         }
         return accumulate;
     }
+
+    @FunctionalInterface
+    private interface IgnoreThrowingRunnable extends ThrowingRunnable
+    {
+        void doRun() throws Throwable;
+
+        @Override
+        default void run()
+        {
+            try
+            {
+                doRun();
+            }
+            catch (Throwable e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+            }
+        }
+
+        static IgnoreThrowingRunnable ignore(IgnoreThrowingRunnable fn)

Review comment:
       The static method only hints the compiler about the input type of `IgnoreThrowingRunnable`. 
   I think it is easier to read to explicitly hint the type at the call-sites and remove this method. 
   
   ```java
   // equivalent
   //  IgnoreThrowingRunnable.ignore(MessagingService.instance()::shutdown)
   (IgnoreThrowingRunnable) MessagingService.instance()::shutdown
   ```




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r489777232



##########
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");
+    }

Review comment:
       nit: maybe combine those 4 helper methods into 1? 
   ```java
       private static void assertTransportStatus(IInvokableInstance node, String transportService, boolean running)
       {
           assertNodetoolStdout(node, running ? "running" : "not running", running ? "not running" : null, "status" + transportService);
       }
   
       // usage
       assertTransportStatus(node, "binary", true); // same as assertBinaryRunning(node);
   ```

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       The `validateTransportsCanStart()` before this line seems no longer needed? Because both `startNativeTransport()` and `startThriftServer()` does the same validation before starting the service. 
   If removing the duplicated validation, the `start()` method barely contains anything other than `startClientTransports()`. 




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -802,4 +805,28 @@ private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn
         }
         return accumulate;
     }
+
+    @FunctionalInterface
+    private interface IgnoreThrowingRunnable extends ThrowingRunnable
+    {
+        void doRun() throws Throwable;
+
+        @Override
+        default void run()
+        {
+            try
+            {
+                doRun();
+            }
+            catch (Throwable e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+            }
+        }
+
+        static IgnoreThrowingRunnable ignore(IgnoreThrowingRunnable fn)

Review comment:
       > The static method only hints the compiler about the input type of IgnoreThrowingRunnable.
   
   Correct.
   
   > I think it is easier to read to explicitly hint the type at the call-sites and remove this method.
   
   Sure, will change.




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r492317741



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       If the instance is not going to start both native and thrift, would the validation be useless? It is to `validateTransportsCanStart`. 

##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -802,4 +805,28 @@ private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn
         }
         return accumulate;
     }
+
+    @FunctionalInterface
+    private interface IgnoreThrowingRunnable extends ThrowingRunnable
+    {
+        void doRun() throws Throwable;
+
+        @Override
+        default void run()
+        {
+            try
+            {
+                doRun();
+            }
+            catch (Throwable e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+            }
+        }
+
+        static IgnoreThrowingRunnable ignore(IgnoreThrowingRunnable fn)

Review comment:
       The static method only hints the compiler about the input type of `IgnoreThrowingRunnable`. 
   I think it is easier to read to explicitly hint the type at the call-sites and remove this method. 
   
   ```java
   // equivalent
   //  IgnoreThrowingRunnable.ignore(MessagingService.instance()::shutdown)
   (IgnoreThrowingRunnable) MessagingService.instance()::shutdown
   ```

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;

Review comment:
       it can also be simplified. 

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+    }
+
+    public void startThriftServer()
+    {
+        validateTransportsCanStart();
+
+        if (thriftServer == null)
+            throw new IllegalStateException("setup() must be called first for CassandraDaemon");
+        thriftServer.start();
+    }
+
+    public void stopThriftServer()
+    {
         if (thriftServer != null)
         {
             thriftServer.stop();
-            thriftServer = null;
         }
     }
 
-    public boolean isNativeTransportRunning()
+    public boolean isThriftServerRunning()
     {
-        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+        return thriftServer != null ? thriftServer.isRunning() : false;

Review comment:
       It can be simplified to 
   ```java
   thriftServer != null && thriftServer.isRunning();
   ```

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       Should the condition be `!isNativeTransportRunning() || !isRPCServerRunning()`




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       if both native and thrift are disabled, then the check doesn't happen in the start method, so the above check would be the only place that calls it; if either are enabled, then you are right that it is duplicate.




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       if you choose to not start either protocol then validation only causes a log, but we will reject starting based off validation (was setup in https://issues.apache.org/jira/browse/CASSANDRA-14525), so I think its kinda useless.  It also causes a edge case where my new log doesn't happen since it would return.
   
   I will remove and rerun python dtests.




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r489760214



##########
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:
       Yep. It is annoying that the other branches other than trunk are still in `junit 4.6`. Should have port https://issues.apache.org/jira/browse/CASSANDRA-13360 to other branches. 




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r492332726



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;

Review comment:
       it can also be simplified. 

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+    }
+
+    public void startThriftServer()
+    {
+        validateTransportsCanStart();
+
+        if (thriftServer == null)
+            throw new IllegalStateException("setup() must be called first for CassandraDaemon");
+        thriftServer.start();
+    }
+
+    public void stopThriftServer()
+    {
         if (thriftServer != null)
         {
             thriftServer.stop();
-            thriftServer = null;
         }
     }
 
-    public boolean isNativeTransportRunning()
+    public boolean isThriftServerRunning()
     {
-        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+        return thriftServer != null ? thriftServer.isRunning() : false;

Review comment:
       It can be simplified to 
   ```java
   thriftServer != null && thriftServer.isRunning();
   ```

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       Should the condition be `!isNativeTransportRunning() || !isRPCServerRunning()`




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -802,4 +805,28 @@ private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn
         }
         return accumulate;
     }
+
+    @FunctionalInterface
+    private interface IgnoreThrowingRunnable extends ThrowingRunnable
+    {
+        void doRun() throws Throwable;
+
+        @Override
+        default void run()
+        {
+            try
+            {
+                doRun();
+            }
+            catch (Throwable e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+            }
+        }
+
+        static IgnoreThrowingRunnable ignore(IgnoreThrowingRunnable fn)

Review comment:
       > The static method only hints the compiler about the input type of IgnoreThrowingRunnable.
   
   Correct.
   
   > I think it is easier to read to explicitly hint the type at the call-sites and remove this method.
   
   Sure, will change.

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       I didn't want to change this as it was unrelated to the bug.  Its also a specific feature about resuming bootstrap so feel its best to leave as is and only init if native isn't running.




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       I didn't want to change this as it was unrelated to the bug.  Its also a specific feature about resuming bootstrap so feel its best to leave as is and only init if native isn't running.




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r492332726



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;

Review comment:
       it can also be simplified. 

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -707,30 +712,41 @@ public void startNativeTransport()
             throw new IllegalStateException("setup() must be called first for CassandraDaemon");
 
         nativeTransportService.start();
-
-        if (thriftServer == null)
-            throw new IllegalStateException("thrift transport should be set up before it can be started");
-        thriftServer.start();
     }
 
     public void stopNativeTransport()
     {
         if (nativeTransportService != null)
         {
             nativeTransportService.stop();
-            nativeTransportService = null;
         }
+    }
 
+    public boolean isNativeTransportRunning()
+    {
+        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+    }
+
+    public void startThriftServer()
+    {
+        validateTransportsCanStart();
+
+        if (thriftServer == null)
+            throw new IllegalStateException("setup() must be called first for CassandraDaemon");
+        thriftServer.start();
+    }
+
+    public void stopThriftServer()
+    {
         if (thriftServer != null)
         {
             thriftServer.stop();
-            thriftServer = null;
         }
     }
 
-    public boolean isNativeTransportRunning()
+    public boolean isThriftServerRunning()
     {
-        return nativeTransportService != null ? nativeTransportService.isRunning() : false;
+        return thriftServer != null ? thriftServer.isRunning() : false;

Review comment:
       It can be simplified to 
   ```java
   thriftServer != null && thriftServer.isRunning();
   ```

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       Should the condition be `!isNativeTransportRunning() || !isRPCServerRunning()`




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
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");
+    }

Review comment:
       looks cleaner, switched.




----------------------------------------------------------------
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] yifan-c commented on a change in pull request #752: Bug/cassandra 16127 3.11

Posted by GitBox <gi...@apache.org>.
yifan-c commented on a change in pull request #752:
URL: https://github.com/apache/cassandra/pull/752#discussion_r492317741



##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       If the instance is not going to start both native and thrift, would the validation be useless? It is to `validateTransportsCanStart`. 




----------------------------------------------------------------
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] dcapwell commented on a change in pull request #752: Bug/cassandra 16127 3.11

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



##########
File path: test/distributed/org/apache/cassandra/distributed/impl/Instance.java
##########
@@ -802,4 +805,28 @@ private static Throwable parallelRun(Throwable accumulate, ExecutorService runOn
         }
         return accumulate;
     }
+
+    @FunctionalInterface
+    private interface IgnoreThrowingRunnable extends ThrowingRunnable
+    {
+        void doRun() throws Throwable;
+
+        @Override
+        default void run()
+        {
+            try
+            {
+                doRun();
+            }
+            catch (Throwable e)
+            {
+                JVMStabilityInspector.inspectThrowable(e);
+            }
+        }
+
+        static IgnoreThrowingRunnable ignore(IgnoreThrowingRunnable fn)

Review comment:
       > The static method only hints the compiler about the input type of IgnoreThrowingRunnable.
   
   Correct.
   
   > I think it is easier to read to explicitly hint the type at the call-sites and remove this method.
   
   Sure, will change.

##########
File path: src/java/org/apache/cassandra/service/StorageService.java
##########
@@ -1613,7 +1612,7 @@ public void onSuccess(StreamState streamState)
                         }
                         progressSupport.progress("bootstrap", new ProgressEvent(ProgressEventType.COMPLETE, 1, 1, "Resume bootstrap complete"));
                         if (!isNativeTransportRunning())

Review comment:
       I didn't want to change this as it was unrelated to the bug.  Its also a specific feature about resuming bootstrap so feel its best to leave as is and only init if native isn't running.

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -531,7 +540,12 @@ public void start()
             logger.info(isx.getMessage());
             return;
         }
+        startClientTransports();

Review comment:
       if you choose to not start either protocol then validation only causes a log, but we will reject starting based off validation (was setup in https://issues.apache.org/jira/browse/CASSANDRA-14525), so I think its kinda useless.  It also causes a edge case where my new log doesn't happen since it would return.
   
   I will remove and rerun python dtests.




----------------------------------------------------------------
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 #752: Bug/cassandra 16127 3.11

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


   


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