You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by "aratno (via GitHub)" <gi...@apache.org> on 2023/03/13 20:03:53 UTC

[GitHub] [cassandra-harry] aratno commented on a diff in pull request #17: Cassandra 17603

aratno commented on code in PR #17:
URL: https://github.com/apache/cassandra-harry/pull/17#discussion_r1134543954


##########
harry-core/src/harry/core/Configuration.java:
##########
@@ -719,18 +720,30 @@ public static class DefaultPDSelectorConfiguration implements PDSelectorConfigur
     {
         public final int window_size;
         public final int slide_after_repeats;
+        public final long position_offset;
+
+        @Deprecated
+        public DefaultPDSelectorConfiguration(int window_size,
+                                              int slide_after_repeats)
+        {
+            this.window_size = window_size;
+            this.slide_after_repeats = slide_after_repeats;
+            this.position_offset = 0L;
+        }
 
         @JsonCreator
         public DefaultPDSelectorConfiguration(@JsonProperty(value = "window_size", defaultValue = "10") int window_size,
-                                              @JsonProperty(value = "slide_after_repeats", defaultValue = "100") int slide_after_repeats)
+                                              @JsonProperty(value = "slide_after_repeats", defaultValue = "100") int slide_after_repeats,
+                                              @JsonProperty(value = "position_offset", defaultValue = "0") long position_offset)

Review Comment:
   Would be useful to have more documentation around how to use this, especially for parallel runners on the same seed



##########
harry-core/src/harry/runner/Runner.java:
##########
@@ -275,9 +276,13 @@ public void runInternal() throws Throwable
                 {
                     Visitor visitor = poolConfiguration.visitor.make(run);
                     String name = String.format("%s-%d", poolConfiguration.prefix, i + 1);
+                    AtomicLong counter = new AtomicLong();
                     Interruptible thread = ExecutorFactory.Global.executorFactory().infiniteLoop(name, wrapInterrupt((state) -> {
                         if (state == Interruptible.State.NORMAL)
                             visitor.visit();
+                        long cnt = counter.incrementAndGet();
+                        if (cnt % 1000 == 0)
+                            logger.info("Visitor {} has cycled {} times", name, cnt);

Review Comment:
   I interpret "cycle" to mean that it has returned back to its initial LTS - which isn't what's happening here.



##########
harry-core/src/harry/visitors/RandomValidator.java:
##########
@@ -0,0 +1,126 @@
+/*
+ *  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 harry.visitors;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.MetricReporter;
+import harry.core.Run;
+import harry.generators.RngUtils;
+import harry.generators.Surjections;
+import harry.model.Model;
+import harry.model.OpSelectors;
+import harry.operations.Query;
+import harry.operations.QueryGenerator;
+
+public class RandomValidator implements Visitor
+{
+    private final BufferedWriter validationLog;
+    private static final Logger logger = LoggerFactory.getLogger(RandomValidator.class);
+    private final Model model;
+
+    private final OpSelectors.PdSelector pdSelector;
+    private final QueryGenerator.TypedQueryGenerator querySelector;
+    private final MetricReporter metricReporter;
+    private final OpSelectors.MonotonicClock clock;
+
+    private final int partitionCount;
+    private final int queries;
+
+    public RandomValidator(int partitionCount,
+                           int queries,
+                           Run run,
+                           Model.ModelFactory modelFactory)
+    {
+        this.partitionCount = partitionCount;
+        this.queries = Math.max(queries, 1);
+        this.metricReporter = run.metricReporter;
+        this.pdSelector = run.pdSelector;
+        this.clock = run.clock;
+        this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng,
+                                                                    Surjections.pick(Query.QueryKind.SINGLE_PARTITION),
+                                                                    run.rangeSelector);
+        this.model = modelFactory.make(run);
+        File f = new File("validation.log");
+        try
+        {
+            validationLog = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(f)));
+        }
+        catch (FileNotFoundException e)
+        {
+            throw new RuntimeException(e);
+        }
+    }
+
+    // TODO: expose metric, how many times validated recent partitions
+    private int validateRandomPartitions()
+    {
+        Random rng = new Random();
+        long maxPos = pdSelector.maxPosition(clock.peek());

Review Comment:
   Is this a source of non-determinism? Runs with the same seed will validate different random partitions, right? I was thinking it could be possible to have another stream ID for validation partitions, so runs with the same seed are deterministic in which partitions they validate.



##########
harry-core/src/harry/visitors/RandomValidator.java:
##########
@@ -0,0 +1,126 @@
+/*
+ *  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 harry.visitors;
+
+import java.io.BufferedWriter;
+import java.io.File;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import harry.core.MetricReporter;
+import harry.core.Run;
+import harry.generators.RngUtils;
+import harry.generators.Surjections;
+import harry.model.Model;
+import harry.model.OpSelectors;
+import harry.operations.Query;
+import harry.operations.QueryGenerator;
+
+public class RandomValidator implements Visitor
+{
+    private final BufferedWriter validationLog;
+    private static final Logger logger = LoggerFactory.getLogger(RandomValidator.class);
+    private final Model model;
+
+    private final OpSelectors.PdSelector pdSelector;
+    private final QueryGenerator.TypedQueryGenerator querySelector;
+    private final MetricReporter metricReporter;
+    private final OpSelectors.MonotonicClock clock;
+
+    private final int partitionCount;
+    private final int queries;
+
+    public RandomValidator(int partitionCount,
+                           int queries,
+                           Run run,
+                           Model.ModelFactory modelFactory)
+    {
+        this.partitionCount = partitionCount;
+        this.queries = Math.max(queries, 1);
+        this.metricReporter = run.metricReporter;
+        this.pdSelector = run.pdSelector;
+        this.clock = run.clock;
+        this.querySelector = new QueryGenerator.TypedQueryGenerator(run.rng,
+                                                                    Surjections.pick(Query.QueryKind.SINGLE_PARTITION),
+                                                                    run.rangeSelector);
+        this.model = modelFactory.make(run);
+        File f = new File("validation.log");

Review Comment:
   It should be possible to run this visitor without the validation log. For longer runs (soak tests), this file will fill the runner's disk. Can you add an option to disable logging?



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