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

benedict pushed a commit to branch cep-15-accord
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit 16f1067426241dc07c16fc854225934ab6255a90
Author: David Capwell <[email protected]>
AuthorDate: Tue Dec 3 20:23:38 2024 -0800

    fixed a bug with accords log parsing logic for error detection
---
 .../AbstractPairOfSequencesPaxosSimulation.java    | 92 ++++++++++----------
 .../paxos/PairOfSequencesAccordSimulation.java     |  9 ++
 ...AbstractPairOfSequencesPaxosSimulationTest.java | 98 ++++++++++++++++++++++
 3 files changed, 157 insertions(+), 42 deletions(-)

diff --git 
a/test/simulator/main/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulation.java
 
b/test/simulator/main/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulation.java
index ca6988ed59..c0fec09c43 100644
--- 
a/test/simulator/main/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulation.java
+++ 
b/test/simulator/main/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulation.java
@@ -29,6 +29,7 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import java.util.stream.Stream;
 
+import com.google.common.annotations.VisibleForTesting;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -130,53 +131,60 @@ abstract class AbstractPairOfSequencesPaxosSimulation 
extends PaxosSimulation
             {
                 LogAction logs = inst.logs();
 
-                LogResult<List<String>> errors = logs.grepForErrors();
-                if (!errors.getResult().isEmpty())
-                {
-                    List<Pair<String, String>> errorsSeen = new ArrayList<>();
-                    for (String error : errors.getResult())
-                    {
-                        for (String line : error.split("\\n"))
-                        {
-                            line = line.trim();
-                            if (line.startsWith("ERROR")) continue;
-                            if (line.startsWith("at ")) continue;
-                            errorsSeen.add(Pair.create(line.split(":")[0], 
error));
-                            break;
-                        }
-                    }
-                    Class<? extends Throwable>[] expected = 
expectedExceptions();
-                    StringBuilder sb = new StringBuilder();
-                    for (Pair<String, String> pair : errorsSeen)
-                    {
-                        String name = pair.left;
-                        String exception = pair.right;
-                        Class<?> klass;
-                        try
-                        {
-                            klass = Class.forName(name);
-                        }
-                        catch (ClassNotFoundException e)
-                        {
-                            throw new RuntimeException(e);
-                        }
-
-                        if (!Stream.of(expected).anyMatch(e -> 
e.isAssignableFrom(klass)))
-                            sb.append("Unexpected 
exception:\n").append(exception).append('\n');
-                    }
-                    if (sb.length() > 0)
-                    {
-                        AssertionError error = new AssertionError("Saw errors 
in node" + inst.config().num() + ": " + sb);
-                        // this stacktrace isn't helpful, can be more confusing
-                        error.setStackTrace(new StackTraceElement[0]);
-                        throw error;
-                    }
-                }
+                checkErrorLogs(inst.config().num(), logs.grepForErrors());
                 return ActionList.empty();
             }
         };
     }
 
+    @VisibleForTesting
+    protected void checkErrorLogs(int node, LogResult<List<String>> errors)
+    {
+        if (!errors.getResult().isEmpty())
+        {
+            List<Pair<String, String>> errorsSeen = new ArrayList<>();
+            for (String error : errors.getResult())
+            {
+                for (String line : error.split("\\n"))
+                {
+                    line = line.trim();
+                    if (line.startsWith("ERROR")) continue;
+                    if (line.startsWith("WARN")) continue;
+                    if (line.startsWith("at ")) continue;
+                    errorsSeen.add(Pair.create(line.split(":")[0], error));
+                    break;
+                }
+            }
+            Class<? extends Throwable>[] expected = expectedExceptions();
+            StringBuilder sb = new StringBuilder();
+            for (Pair<String, String> pair : errorsSeen)
+            {
+                String name = pair.left;
+                String exception = pair.right;
+                Class<?> klass;
+                try
+                {
+                    klass = Class.forName(name);
+                }
+                catch (ClassNotFoundException e)
+                {
+                    sb.append("Unexpected exception (could not parse 
line):\n").append(exception).append('\n');
+                    continue;
+                }
+
+                if (!Stream.of(expected).anyMatch(e -> 
e.isAssignableFrom(klass)))
+                    sb.append("Unexpected 
exception:\n").append(exception).append('\n');
+            }
+            if (sb.length() > 0)
+            {
+                AssertionError error = new AssertionError("Saw errors in node" 
+ node + ": " + sb);
+                // this stacktrace isn't helpful, can be more confusing
+                error.setStackTrace(new StackTraceElement[0]);
+                throw error;
+            }
+        }
+    }
+
     protected Metrics getMetrics(int coordinatorIndex)
     {
         return cluster.get(coordinatorIndex).metrics();
diff --git 
a/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
 
b/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
index 7965c29fc1..4702f1c67e 100644
--- 
a/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
+++ 
b/test/simulator/main/org/apache/cassandra/simulator/paxos/PairOfSequencesAccordSimulation.java
@@ -33,6 +33,7 @@ import javax.annotation.Nullable;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import accord.coordinate.CoordinationFailed;
 import com.carrotsearch.hppc.IntArrayList;
 import com.carrotsearch.hppc.IntHashSet;
 import com.carrotsearch.hppc.cursors.IntCursor;
@@ -48,6 +49,7 @@ 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.impl.Query;
+import org.apache.cassandra.exceptions.RequestExecutionException;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.service.consensus.TransactionalMode;
@@ -150,6 +152,13 @@ public class PairOfSequencesAccordSimulation extends 
AbstractPairOfSequencesPaxo
         this.validator = validator;
     }
 
+    @Override
+    protected Class<? extends Throwable>[] expectedExceptions()
+    {
+        return (Class<? extends Throwable>[]) new Class<?>[] { 
RequestExecutionException.class,
+                                                               
CoordinationFailed.class };
+    }
+
     @Override
     protected String createTableStmt()
     {
diff --git 
a/test/simulator/test/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulationTest.java
 
b/test/simulator/test/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulationTest.java
new file mode 100644
index 0000000000..32af571748
--- /dev/null
+++ 
b/test/simulator/test/org/apache/cassandra/simulator/paxos/AbstractPairOfSequencesPaxosSimulationTest.java
@@ -0,0 +1,98 @@
+/*
+ * 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.simulator.paxos;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.junit.Test;
+
+import org.apache.cassandra.distributed.api.LogAction;
+import org.apache.cassandra.distributed.api.LogResult;
+import org.assertj.core.api.Assertions;
+import org.mockito.Mockito;
+
+public class AbstractPairOfSequencesPaxosSimulationTest
+{
+    @Test
+    public void parseSuccess()
+    {
+        String log = "WARN  [AccordExecutor[1,0]:1] node1 2024-12-03 
17:45:24,574 [10,1577851211987004,9(RX),1]: Exception coordinating 
ExclusiveSyncPoint for 
[1b255f4d-ef25-40a6-0000-000000000009:[(-2978380553567688022,-2930342157542402732]]]
 durability. Increased numberOfSplits to 256\n" +
+                     "accord.coordinate.Invalidated: null\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.lambda$proposeAndCommitInvalidate$2(Propose.java:193)\n"
 +
+                     "\tat accord.local.Node.withEpoch(Node.java:391)\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.lambda$proposeAndCommitInvalidate$3(Propose.java:186)\n"
 +
+                     "\tat 
accord.coordinate.Propose$Invalidate.onSuccess(Propose.java:217)\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.onSuccess(Propose.java:146)\n" +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe$RegisteredCallback.unsafeOnSuccess(RequestCallbacks.java:119)\n"
 +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe.lambda$onSuccess$0(RequestCallbacks.java:189)\n"
 +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe$RegisteredCallback.lambda$safeInvoke$0(RequestCallbacks.java:140)\n"
 +
+                     "\tat 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)\n"
 +
+                     "\tat 
accord.utils.async.AsyncChains.lambda$encapsulate$0(AsyncChains.java:498)\n" +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutor$PlainRunnable.run(AccordExecutor.java:989)\n"
 +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutor$CommandStoreQueue.run(AccordExecutor.java:729)\n"
 +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutorSimple.run(AccordExecutorSimple.java:95)\n"
 +
+                     "\tat 
org.apache.cassandra.concurrent.FutureTask$2.call(FutureTask.java:124)\n" +
+                     "\tat 
org.apache.cassandra.concurrent.SyncFutureTask.run(SyncFutureTask.java:68)\n" +
+                     "\tat 
org.apache.cassandra.simulator.systems.InterceptingExecutor$AbstractSingleThreadedExecutorPlus.lambda$new$0(InterceptingExecutor.java:585)\n"
 +
+                     "\tat 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n"
 +
+                     "\tat java.base/java.lang.Thread.run(Thread.java:829)";
+        LogResult<List<String>> errors = new LogAction.BasicLogResult<>(42, 
Collections.singletonList(log));
+
+        AbstractPairOfSequencesPaxosSimulation simulation = 
Mockito.mock(AbstractPairOfSequencesPaxosSimulation.class);
+        
Mockito.doCallRealMethod().when(simulation).checkErrorLogs(Mockito.eq(0), 
Mockito.eq(errors));
+        Mockito.when(simulation.expectedExceptions()).thenReturn((Class<? 
extends Throwable>[]) new Class<?>[] { accord.coordinate.Invalidated.class });
+
+        simulation.checkErrorLogs(0, errors);
+    }
+
+    @Test
+    public void parseFailure()
+    {
+        String log = "FAKE  [AccordExecutor[1,0]:1] node1 2024-12-03 
17:45:24,574 [10,1577851211987004,9(RX),1]: Exception coordinating 
ExclusiveSyncPoint for 
[1b255f4d-ef25-40a6-0000-000000000009:[(-2978380553567688022,-2930342157542402732]]]
 durability. Increased numberOfSplits to 256\n" +
+                     "accord.coordinate.Invalidated: null\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.lambda$proposeAndCommitInvalidate$2(Propose.java:193)\n"
 +
+                     "\tat accord.local.Node.withEpoch(Node.java:391)\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.lambda$proposeAndCommitInvalidate$3(Propose.java:186)\n"
 +
+                     "\tat 
accord.coordinate.Propose$Invalidate.onSuccess(Propose.java:217)\n" +
+                     "\tat 
accord.coordinate.Propose$Invalidate.onSuccess(Propose.java:146)\n" +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe$RegisteredCallback.unsafeOnSuccess(RequestCallbacks.java:119)\n"
 +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe.lambda$onSuccess$0(RequestCallbacks.java:189)\n"
 +
+                     "\tat 
accord.impl.RequestCallbacks$CallbackStripe$RegisteredCallback.lambda$safeInvoke$0(RequestCallbacks.java:140)\n"
 +
+                     "\tat 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)\n"
 +
+                     "\tat 
accord.utils.async.AsyncChains.lambda$encapsulate$0(AsyncChains.java:498)\n" +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutor$PlainRunnable.run(AccordExecutor.java:989)\n"
 +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutor$CommandStoreQueue.run(AccordExecutor.java:729)\n"
 +
+                     "\tat 
org.apache.cassandra.service.accord.AccordExecutorSimple.run(AccordExecutorSimple.java:95)\n"
 +
+                     "\tat 
org.apache.cassandra.concurrent.FutureTask$2.call(FutureTask.java:124)\n" +
+                     "\tat 
org.apache.cassandra.concurrent.SyncFutureTask.run(SyncFutureTask.java:68)\n" +
+                     "\tat 
org.apache.cassandra.simulator.systems.InterceptingExecutor$AbstractSingleThreadedExecutorPlus.lambda$new$0(InterceptingExecutor.java:585)\n"
 +
+                     "\tat 
io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)\n"
 +
+                     "\tat java.base/java.lang.Thread.run(Thread.java:829)";
+        LogResult<List<String>> errors = new LogAction.BasicLogResult<>(42, 
Collections.singletonList(log));
+
+        AbstractPairOfSequencesPaxosSimulation simulation = 
Mockito.mock(AbstractPairOfSequencesPaxosSimulation.class);
+        
Mockito.doCallRealMethod().when(simulation).checkErrorLogs(Mockito.eq(0), 
Mockito.eq(errors));
+        Mockito.when(simulation.expectedExceptions()).thenReturn((Class<? 
extends Throwable>[]) new Class<?>[] { accord.coordinate.Invalidated.class });
+
+        Assertions.assertThatThrownBy(() -> simulation.checkErrorLogs(0, 
errors))
+                  .isInstanceOf(AssertionError.class)
+                  .hasMessageStartingWith("Saw errors in node0: Unexpected 
exception (could not parse line):");
+    }
+}
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to