You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2021/11/10 17:18:09 UTC

[GitHub] [cassandra] dcapwell commented on a change in pull request #1314: CASSANDRA-17072-4.0: Fix ignored client warnings in DebuggableThreadPoolExecutor

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



##########
File path: test/unit/org/apache/cassandra/concurrent/DebuggableThreadPoolExecutorTest.java
##########
@@ -78,6 +82,34 @@ public void runMayThrow() throws InterruptedException
         assert delta >= 9 * 50 : delta;
     }
 
+    @Test
+    public void testExecuteFutureTaskWhileCapturingClientWarnings() throws InterruptedException
+    {
+        testClientWarnings(executor -> executor.execute(() -> ClientWarn.instance.warn("msg")));
+        testClientWarnings(executor -> executor.submit(() -> ClientWarn.instance.warn("msg")));
+        testClientWarnings(executor -> executor.submit(() -> ClientWarn.instance.warn("msg"), null));
+        testClientWarnings(executor -> executor.submit((Callable<Void>) () -> {
+            ClientWarn.instance.warn("msg");
+            return null;
+        }));
+    }
+
+    private void testClientWarnings(Consumer<DebuggableThreadPoolExecutor> schedulingTask) throws InterruptedException
+    {
+        LinkedBlockingQueue<Runnable> q = new LinkedBlockingQueue<Runnable>(1);
+        DebuggableThreadPoolExecutor executor = new DebuggableThreadPoolExecutor(1,

Review comment:
       nit: can reuse the executor rather than start/stop for each test case

##########
File path: src/java/org/apache/cassandra/cql3/statements/schema/CreateKeyspaceStatement.java
##########
@@ -87,10 +83,6 @@ public Keyspaces apply(Keyspaces schema)
         keyspace.params.validate(keyspaceName);
         Keyspaces keyspaces = schema.withAddedOrUpdated(keyspace);
 
-        int newNumWarnings = ClientWarn.instance.numWarnings();
-        if (newNumWarnings > previousNumWarnings)
-            clientWarnings.addAll(ClientWarn.instance.getWarnings().subList(previousNumWarnings, newNumWarnings));
-

Review comment:
       I know you are removing but I am trying to figure out the justification for ignoring previous warnings...  Tried reading CASSANDRA-16296 and this comes off as a work around to a bug rather than fix the bug...  https://issues.apache.org/jira/browse/CASSANDRA-16296?focusedCommentId=17277213&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17277213
   
   @adelapena can I get your eyes here?

##########
File path: test/unit/org/apache/cassandra/schema/SchemaTest.java
##########
@@ -78,4 +92,81 @@ public void testTransKsMigration() throws IOException
         }
     }
 
+    @Test
+    @BMRules(rules = { @BMRule(name = "client warning 1",
+                               targetClass = "CreateKeyspaceStatement",
+                               targetMethod = "apply",
+                               targetLocation = "AT INVOKE KeyspaceParams.validate",
+                               action = "org.apache.cassandra.schema.SchemaTest.addWarn()"),
+                       @BMRule(name = "client warning 2",
+                               targetClass = "CreateKeyspaceStatement",
+                               targetMethod = "clientWarnings",
+                               targetLocation = "AT EXIT",
+                               action = "return org.apache.cassandra.schema.SchemaTest.addWarnToList($!)"),
+                       @BMRule(name = "client warning 3",
+                               targetClass = "AlterSchemaStatement",
+                               targetMethod = "clientWarnings",
+                               targetLocation = "AT EXIT",
+                               action = "return org.apache.cassandra.schema.SchemaTest.addWarnToList($!)")})
+    public void testClientWarningsOnCreateKeyspace() throws Throwable
+    {
+        ClientWarn.instance.captureWarnings();
+        injectedWarnings.clear();
+        createKeyspace("CREATE KEYSPACE %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}");
+
+        Assertions.assertThat(injectedWarnings).contains(msg1, msg2); // failure here means the bm rules need to be updated
+        Assertions.assertThat(ClientWarn.instance.getWarnings()).containsExactlyInAnyOrder(msg1, msg2);
+    }
+
+    @Test
+    @BMRules(rules = { @BMRule(name = "client warning 1",
+                               targetClass = "CreateTableStatement",
+                               targetMethod = "apply",
+                               targetLocation = "AT INVOKE TableMetadata.validate",
+                               action = "org.apache.cassandra.schema.SchemaTest.addWarn()"),
+                       @BMRule(name = "client warning 2",
+                               targetClass = "CreateTableStatement",
+                               targetMethod = "clientWarnings",
+                               targetLocation = "AT EXIT",
+                               action = "return org.apache.cassandra.schema.SchemaTest.addWarnToList($!)"),
+                       @BMRule(name = "client warning 3",
+                               targetClass = "AlterSchemaStatement",
+                               targetMethod = "clientWarnings",
+                               targetLocation = "AT EXIT",
+                               action = "return org.apache.cassandra.schema.SchemaTest.addWarnToList($!)"),

Review comment:
       missing case for `alter`

##########
File path: src/java/org/apache/cassandra/cql3/statements/schema/AlterKeyspaceStatement.java
##########
@@ -84,10 +81,6 @@ public Keyspaces apply(Keyspaces schema)
 
         Keyspaces res = schema.withAddedOrUpdated(newKeyspace);
 
-        int newNumWarnings = ClientWarn.instance.numWarnings();

Review comment:
       I know you are removing but I am trying to figure out the justification for ignoring previous warnings...  Tried reading CASSANDRA-16296 and this comes off as a work around to a bug rather than fix the bug...  https://issues.apache.org/jira/browse/CASSANDRA-16296?focusedCommentId=17277213&page=com.atlassian.jira.plugin.system.issuetabpanels%3Acomment-tabpanel#comment-17277213
   
   @adelapena can I get your eyes here?

##########
File path: src/java/org/apache/cassandra/service/ClientWarn.java
##########
@@ -74,6 +74,11 @@ public void resetWarnings()
         warnLocal.remove();
     }
 
+    public static boolean isCapturingClientWarnings()

Review comment:
       dead code?




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