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/07/17 07:57:05 UTC

[GitHub] [cassandra] smiklosovic opened a new pull request #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

smiklosovic opened a new pull request #684:
URL: https://github.com/apache/cassandra/pull/684


   for trunk


----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
##########
@@ -58,11 +59,32 @@ private JVMStabilityInspector() {}
      */
     public static void inspectThrowable(Throwable t) throws OutOfMemoryError
     {
-        inspectThrowable(t, true);
+        inspectThrowable(t, null, true, true);

Review comment:
       most examples I see calling this method happen in the current thread, so thinking we should use the current thread as the default rather than null.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       These tests don't replicate the bad logic reported in JIRA.  I took these tests and slightly modified them to not throw but update the thread error handler and both passed.  
   
   Can we update the tests to replicate the behavior reported?  Since the tests call `JVMStabilityInspector` directly they don't actually match the change made to AbstractLocalAwareExecutorService

##########
File path: src/java/org/apache/cassandra/service/CassandraDaemon.java
##########
@@ -463,14 +463,12 @@ public static void uncaughtException(Thread t, Throwable e)
             {
                 if (e2 != e) // make sure FSError gets logged exactly once.
                     logger.error("Exception in thread " + t, e2);
-                FileUtils.handleFSError((FSError) e2);

Review comment:
       since this is now handled by `JVMStabilityInspector.inspectThrowable` the logging should also be handled there.  Also, inspect is recursive so we don't need to walk the cause change anymore




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell sorry I am not getting what you are trying to say.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell I have added the second commit where I am testing that executor service so I hope it helps.
   
   I am little bit lost on your first comment as I do not know how to proceed there. The looping and the logging over these exceptions there is done only for CorruptSSTableException and FSError exception but not for the rest, should I move this behavior to inspect method or we should take more general approach? If I am about to make this happen for CSE and FSE, to exactly copy the existing behaviour, I would have to propagate Thread into that inspect method too which would change method signature etc ... 




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
##########
@@ -58,11 +59,32 @@ private JVMStabilityInspector() {}
      */
     public static void inspectThrowable(Throwable t) throws OutOfMemoryError
     {
-        inspectThrowable(t, true);
+        inspectThrowable(t, null, true, true);
     }
 
+
     public static void inspectThrowable(Throwable t, boolean propagateOutOfMemory) throws OutOfMemoryError
     {
+        inspectThrowable(t, null, propagateOutOfMemory, true);
+    }
+
+    public static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory)
+    {
+        inspectThrowable(t, thread, propagateOutOfMemory, true);
+    }
+
+    private static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory, boolean shouldLog) throws OutOfMemoryError
+    {
+        if (shouldLog)
+        {
+            if (thread != null)
+                logger.error(String.format("Uncaught exception in thread %s", thread), t);

Review comment:
       with logger you can use `logger.error("Uncaught exception in thread {}", thread, t);`.  Don't need String.format




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
##########
@@ -58,11 +59,32 @@ private JVMStabilityInspector() {}
      */
     public static void inspectThrowable(Throwable t) throws OutOfMemoryError
     {
-        inspectThrowable(t, true);
+        inspectThrowable(t, null, true, true);
     }
 
+
     public static void inspectThrowable(Throwable t, boolean propagateOutOfMemory) throws OutOfMemoryError
     {
+        inspectThrowable(t, null, propagateOutOfMemory, true);

Review comment:
       this looks mostly used by networking, and the exception thrown is in a different thread... sadly null makes sense given the current usage =(




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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


   


----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell sorry I am not getting what you are trying to say, especially the first paragraph. How can I replicate the "bad behavior" when I just fixed it?
   
   I am not sure what is wrong with your tests but when I comment out `JVMStabilityInspector.inspectThrowable` in that callable and I run the test, it fails on assertions in test method.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell good stuff, I wanted to do something similar but I am not so strong in Cassandra test framework itself and I couldnt express that idea into the code as you wrote it. I ll incorporate this into the PR and we should be ready to go I guess ... 




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell sorry I am not getting what you are trying to say, especially the first paragraph. How can I replicate the "bad behavior" when I just fixed 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] dcapwell commented on a change in pull request #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/unit/org/apache/cassandra/utils/JVMStabilityInspectorTest.java
##########
@@ -60,6 +62,14 @@ public void testKill() throws Exception
             JVMStabilityInspector.inspectThrowable(new FSReadError(new IOException(), "blah"));
             assertTrue(killerForTests.wasKilled());
 
+            killerForTests.reset();
+            JVMStabilityInspector.inspectThrowable(new FSWriteError(new IOException(), "blah"));
+            assertTrue(killerForTests.wasKilled());
+
+            killerForTests.reset();
+            JVMStabilityInspector.inspectThrowable(new CorruptSSTableException(new IOException(), "blah"));

Review comment:
       can you also add a second one where the corrupt sstable is wrapped in a runnable?




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
##########
@@ -58,11 +59,32 @@ private JVMStabilityInspector() {}
      */
     public static void inspectThrowable(Throwable t) throws OutOfMemoryError
     {
-        inspectThrowable(t, true);
+        inspectThrowable(t, null, true, true);
     }
 
+
     public static void inspectThrowable(Throwable t, boolean propagateOutOfMemory) throws OutOfMemoryError
     {
+        inspectThrowable(t, null, propagateOutOfMemory, true);
+    }
+
+    public static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory)
+    {
+        inspectThrowable(t, thread, propagateOutOfMemory, true);
+    }
+
+    private static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory, boolean shouldLog) throws OutOfMemoryError
+    {
+        if (shouldLog)
+        {
+            if (thread != null)
+                logger.error(String.format("Uncaught exception in thread %s", thread), t);

Review comment:
       thanks, fixed.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell If I have this method
   
   ```
       private static final class CorruptSSTableCallable implements SerializableCallable<Boolean[]>
       {
           public Boolean[] call()
           {
               final CassandraDaemon cassandraDaemon = CassandraDaemon.getInstanceForTesting();
               cassandraDaemon.completeSetup();
               StorageService.instance.registerDaemon(cassandraDaemon);
               FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
   
               JVMStabilityInspector.inspectThrowable(new CorruptSSTableException(new RuntimeException(""), "sstablepath"));
   
               return new Boolean[]{
               StorageService.instance.isNativeTransportRunning(),
               Gossiper.instance.isEnabled(),
               };
           }
       }
   ```
   And I comment out `JVMStabilityInspector.inspectThrowable....`
   
   my tests fails, which is right.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.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.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+
+import org.junit.Test;
+
+import org.apache.cassandra.concurrent.LocalAwareExecutorService;
+import org.apache.cassandra.concurrent.SharedExecutorPool;
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl
+{
+    @Test
+    public void testAbstractLocalAwareExecutorService()
+    {
+        try (final Cluster cluster = init(getCluster(DiskFailurePolicy.ignore).start()))
+        {
+            final Boolean[] results = cluster.get(1).callsOnInstance(new CorruptSSTableInExecutorCallable()).call();
+
+            assertTrue(results[0]);
+            assertTrue(results[1]);
+            assertTrue(results[2]);
+            assertTrue(results[3]);
+            assertTrue(results[4]);
+        }
+        catch (final Exception ex)
+        {
+            fail("Threw exception:" + ex);
+        }
+    }
+
+    @Test
+    public void corruptSSTAbleOnIgnoreDiskFailurePolicyTest()
+    {
+        try (final Cluster cluster = init(getCluster(DiskFailurePolicy.ignore).start()))
+        {
+            final Boolean[] enabledServices = cluster.get(1).callsOnInstance(new CorruptSSTableCallable()).call();
+
+            // here we expect that Gossip and transport is NOT stopped as disk failure policy is ignore
+            assertTrue(enabledServices[0]);
+            assertTrue(enabledServices[1]);
+        }
+        catch (final Exception ex)
+        {
+            fail("Threw exception:" + ex);
+        }
+    }
+
+    @Test
+    public void corruptSSTableOnStopParanoidDiskFailurePolicyTest()
+    {
+        try (final Cluster cluster = init(getCluster(DiskFailurePolicy.stop_paranoid).start()))
+        {
+            final Boolean[] enabledServices = cluster.get(1).callsOnInstance(new CorruptSSTableCallable()).call();
+
+            // here we expect that Gossip and transport ARE stopped as disk failure policy is stop_paranoid
+            assertFalse(enabledServices[0]);
+            assertFalse(enabledServices[1]);
+        }
+        catch (final Exception ex)
+        {
+            fail("Threw exception: " + ex);
+        }
+    }
+
+    private static final class CorruptSSTableCallable implements SerializableCallable<Boolean[]>
+    {
+        public Boolean[] call()
+        {
+            final CassandraDaemon cassandraDaemon = CassandraDaemon.getInstanceForTesting();
+            cassandraDaemon.completeSetup();
+            StorageService.instance.registerDaemon(cassandraDaemon);
+            FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
+
+            JVMStabilityInspector.inspectThrowable(new CorruptSSTableException(new RuntimeException(""), "sstablepath"));
+
+            return new Boolean[]{
+            StorageService.instance.isNativeTransportRunning(),
+            Gossiper.instance.isEnabled(),
+            };
+        }
+    }
+
+    private static final class CorruptSSTableInExecutorCallable implements SerializableCallable<Boolean[]>

Review comment:
       @dcapwell newly added, the first test uses that.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: src/java/org/apache/cassandra/utils/JVMStabilityInspector.java
##########
@@ -58,11 +59,32 @@ private JVMStabilityInspector() {}
      */
     public static void inspectThrowable(Throwable t) throws OutOfMemoryError
     {
-        inspectThrowable(t, true);
+        inspectThrowable(t, null, true, true);
     }
 
+
     public static void inspectThrowable(Throwable t, boolean propagateOutOfMemory) throws OutOfMemoryError
     {
+        inspectThrowable(t, null, propagateOutOfMemory, true);
+    }
+
+    public static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory)
+    {
+        inspectThrowable(t, thread, propagateOutOfMemory, true);
+    }
+
+    private static void inspectThrowable(Throwable t, Thread thread, boolean propagateOutOfMemory, boolean shouldLog) throws OutOfMemoryError
+    {
+        if (shouldLog)
+        {
+            if (thread != null)
+                logger.error(String.format("Uncaught exception in thread %s", thread), t);

Review comment:
       with logger you can use `logger.error("Uncaught exception in thread {}", thread, t);`




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       sorry that I was not clear.  
   
   What I wanted to say was that these tests test the new behavior but didn't show that the old behavior was problematic.  As the JIRA calls out, the issue is that RuntimeException is used, but this is controlled by `org.apache.cassandra.service.StorageProxy.DroppableRunnable#run`, which isn't leveraged in the test.
   
   To help, here is a patch which rewrites the tests in the model used by FailingRepairTest; they cause corruption at the sstable so a read will fail
   
   ```
   /*
    * 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.HashSet;
   import java.util.Set;
   
   import org.junit.Test;
   
   import org.apache.cassandra.config.Config.DiskFailurePolicy;
   import org.apache.cassandra.db.ColumnFamilyStore;
   import org.apache.cassandra.db.DecoratedKey;
   import org.apache.cassandra.db.Keyspace;
   import org.apache.cassandra.db.RowIndexEntry;
   import org.apache.cassandra.db.Slices;
   import org.apache.cassandra.db.filter.ColumnFilter;
   import org.apache.cassandra.db.rows.UnfilteredRowIterator;
   import org.apache.cassandra.distributed.Cluster;
   import org.apache.cassandra.distributed.api.ConsistencyLevel;
   import org.apache.cassandra.distributed.api.IInvokableInstance;
   import org.apache.cassandra.distributed.shared.AbstractBuilder;
   import org.apache.cassandra.distributed.shared.NetworkTopology;
   import org.apache.cassandra.gms.Gossiper;
   import org.apache.cassandra.io.sstable.CorruptSSTableException;
   import org.apache.cassandra.io.sstable.format.ForwardingSSTableReader;
   import org.apache.cassandra.io.sstable.format.SSTableReader;
   import org.apache.cassandra.io.sstable.format.SSTableReadsListener;
   import org.apache.cassandra.io.util.FileDataInput;
   import org.apache.cassandra.service.CassandraDaemon;
   import org.apache.cassandra.service.StorageService;
   import org.assertj.core.api.Assertions;
   
   import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
   import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
   import static org.apache.cassandra.distributed.api.Feature.NETWORK;
   
   public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl
   {
       @Test
       public void testAbstractLocalAwareExecutorServiceOnIgnoredDiskFailurePolicy() throws IOException
       {
           test(DiskFailurePolicy.ignore, true, true);
       }
   
       @Test
       public void testAbstractLocalAwareExecutorServiceOnStopParanoidDiskFailurePolicy() throws IOException
       {
           test(DiskFailurePolicy.stop_paranoid, false, false);
       }
   
       private static void test(DiskFailurePolicy policy, boolean expectNativeTransportRunning, boolean expectGossiperEnabled) throws IOException
       {
           String table = policy.name();
           try (final Cluster cluster = init(getCluster(policy).start()))
           {
               IInvokableInstance node = cluster.get(1);
               node.runOnInstance(() -> {
                   //TODO why is this not working properly in jvm-dtest?  StorageService.instance.isNativeTransportRunning() never gets set to true even with native enabled
                   StorageService.instance.registerDaemon(CassandraDaemon.getInstanceForTesting());
               });
   
               cluster.schemaChange("CREATE TABLE " + KEYSPACE + "." + table + " (id bigint PRIMARY KEY)");
               node.executeInternal("INSERT INTO " + KEYSPACE + "." + table + " (id) VALUES (?)", 0L);
               corruptTable(node, KEYSPACE, table);
   
               // make sure environment is setup propertly
               Assertions.assertThat(node.callOnInstance(() -> new boolean[]{ StorageService.instance.isNativeTransportRunning(), Gossiper.instance.isEnabled() }))
                         .isEqualTo(new boolean[]{ true, true });
   
               // query should see corrupt sstable and should fail the query
               Assertions.assertThatThrownBy(() -> cluster.coordinator(1).execute("SELECT * FROM " + KEYSPACE + "." + table + " WHERE id=?", ConsistencyLevel.ONE, 0L));
   
               Assertions.assertThat(node.callOnInstance(() -> new boolean[]{ StorageService.instance.isNativeTransportRunning(), Gossiper.instance.isEnabled() }))
                         .isEqualTo(new boolean[]{ expectNativeTransportRunning, expectGossiperEnabled });
           }
       }
   
       private static void corruptTable(IInvokableInstance node, String keyspace, String table)
       {
           node.runOnInstance(() -> {
               ColumnFamilyStore cf = Keyspace.open(keyspace).getColumnFamilyStore(table);
               cf.forceBlockingFlush();
   
               Set<SSTableReader> remove = cf.getLiveSSTables();
               Set<SSTableReader> replace = new HashSet<>();
               for (SSTableReader r : remove)
                   replace.add(new CorruptedSSTableReader(r));
   
               cf.getTracker().removeUnsafe(remove);
               cf.addSSTables(replace);
           });
       }
   
       private static AbstractBuilder<IInvokableInstance, Cluster, Cluster.Builder> getCluster(DiskFailurePolicy diskFailurePolicy)
       {
           return Cluster.build()
                         .withNodeIdTopology(NetworkTopology.singleDcNetworkTopology(1, "dc0", "rack0"))
                         .withConfig(config -> config.with(NETWORK, GOSSIP, NATIVE_PROTOCOL)
                                                     .set("disk_failure_policy", diskFailurePolicy.name()));
       }
   
       private static final class CorruptedSSTableReader extends ForwardingSSTableReader
       {
           public CorruptedSSTableReader(SSTableReader delegate)
           {
               super(delegate);
           }
   
           public UnfilteredRowIterator iterator(DecoratedKey key, Slices slices, ColumnFilter selectedColumns, boolean reversed, SSTableReadsListener listener)
           {
               throw throwCorrupted();
           }
   
           public UnfilteredRowIterator iterator(FileDataInput file, DecoratedKey key, RowIndexEntry indexEntry, Slices slices, ColumnFilter selectedColumns, boolean reversed)
           {
               throw throwCorrupted();
           }
   
           private CorruptSSTableException throwCorrupted()
           {
               throw new CorruptSSTableException(new IOException("failed to get position"), descriptor.baseFilename());
           }
       }
   }
   ```
   
   This test replicates the behavior reported in the JIRA and shows that a corrupt sstable on a read will not trigger the disk failure policy and that your patch fixes it.  This problem is localized to the read/write stages as compaction actually handles this behavior (as shown by the FailingRepairTest).




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell I have added the second commit where I am testing that executor service so I hope it helps.
   
   I am little bit lost on your first comment as I do not know how to proceed there. The looping and the logging over these exceptions there is done only for CorruptSSTableException and FSError exception but not for the rest, should I move this behavior to inspect method or we should take more general approach? If I am about to make this happen for CSE and FSE, to exactly copy the existing behaviour, I would have to propagate Thread into that inspect method too which would change method signature etc ... 




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell If I have this method
   
   ```
       private static final class CorruptSSTableCallable implements SerializableCallable<Boolean[]>
       {
           public Boolean[] call()
           {
               final CassandraDaemon cassandraDaemon = CassandraDaemon.getInstanceForTesting();
               cassandraDaemon.completeSetup();
               StorageService.instance.registerDaemon(cassandraDaemon);
               FileUtils.setFSErrorHandler(new DefaultFSErrorHandler());
   
               JVMStabilityInspector.inspectThrowable(new CorruptSSTableException(new RuntimeException(""), "sstablepath"));
   
               return new Boolean[]{
               StorageService.instance.isNativeTransportRunning(),
               Gossiper.instance.isEnabled(),
               };
           }
       }
   ```
   And I comment out `JVMStabilityInspector.inspectThrowable....`
   
   my tests fail, which is right.




----------------------------------------------------------------
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 #684: CASSANDRA-15191 - stop gossip and transport on stop_paranoid disk failure policy

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



##########
File path: test/distributed/org/apache/cassandra/distributed/test/JVMStabilityInspectorCorruptSSTableExceptionTest.java
##########
@@ -0,0 +1,105 @@
+/*
+ * 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 org.junit.Test;
+
+import org.apache.cassandra.config.Config.DiskFailurePolicy;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.api.IInvokableInstance;
+import org.apache.cassandra.distributed.api.IIsolatedExecutor.SerializableCallable;
+import org.apache.cassandra.distributed.shared.AbstractBuilder;
+import org.apache.cassandra.distributed.shared.NetworkTopology;
+import org.apache.cassandra.gms.Gossiper;
+import org.apache.cassandra.io.sstable.CorruptSSTableException;
+import org.apache.cassandra.io.util.FileUtils;
+import org.apache.cassandra.service.CassandraDaemon;
+import org.apache.cassandra.service.DefaultFSErrorHandler;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.JVMStabilityInspector;
+
+import static org.apache.cassandra.distributed.api.Feature.GOSSIP;
+import static org.apache.cassandra.distributed.api.Feature.NATIVE_PROTOCOL;
+import static org.apache.cassandra.distributed.api.Feature.NETWORK;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class JVMStabilityInspectorCorruptSSTableExceptionTest extends TestBaseImpl

Review comment:
       @dcapwell I have added the second commit where I am testing that executor service so I hope it helps.
   
   I am little bit lost on your first comment as I do not know how to proceed there. The looping over these exceptions there is done only for CorruptSSTableException and FSError exception but not for the rest, should I move this behavior to inspect method or we should take more general approach? If I am about to make this happen for CSE and FSE, to exactly copy the existing behaviour, I would have to propagate Thread into that inspect method too which would change method signature etc ... 




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