You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by if...@apache.org on 2020/12/11 13:28:42 UTC

[cassandra-harry] 05/10: Fixed broken & missing props in example YAML; Added new example config for external cluster; Added run scripts to help with getting started; Fixed object mapping for `ApproximateMonotonicClockConfiguration`; Removed hardcoded configuration for in-JVM execution;

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

ifesdjeen pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-harry.git

commit 9c5eb00b2f88dd63a40c21d4ff7b618b5d2c1457
Author: Gianluca Righetto <gi...@menttis.com>
AuthorDate: Tue Dec 8 17:05:31 2020 -0300

    Fixed broken & missing props in example YAML; Added new example config for external cluster; Added run scripts to help with getting started; Fixed object mapping for `ApproximateMonotonicClockConfiguration`; Removed hardcoded configuration for in-JVM execution;
---
 conf/example.yaml                                  | 10 +++-
 conf/external.yaml                                 | 12 ++--
 docker/Dockerfile.local                            |  1 +
 docker/run.sh                                      |  3 +-
 harry-core/src/harry/core/Configuration.java       |  1 +
 .../model/sut/external/ExternalClusterSut.java     |  2 +-
 .../harry/runner/external/HarryRunnerExternal.java | 23 ++------
 .../src/harry/model/sut/InJvmSut.java              |  2 +-
 .../src/harry/runner/HarryRunnerJvm.java           | 13 +++--
 harry-integration/src/harry/runner/Reproduce.java  |  2 +-
 harry-runner/src/harry/runner/HarryRunner.java     | 55 ++++++++---------
 run-external.sh                                    |  3 +
 docker/run.sh => run-jvm.sh                        | 68 ++--------------------
 13 files changed, 69 insertions(+), 126 deletions(-)

diff --git a/conf/example.yaml b/conf/example.yaml
index de8c77d..3a3c5ab 100644
--- a/conf/example.yaml
+++ b/conf/example.yaml
@@ -31,9 +31,9 @@ truncate_table: false
 # to map it back to the logical timestamp of the operation that wrote this value.
 clock:
   approximate_monotonic:
-    historySize: 7300
-    epochPeriod: 1
-    epochTimeUnit: "SECONDS"
+    history_size: 7300
+    epoch_length: 1
+    epoch_time_unit: "SECONDS"
 
 # Runner is a is a component that schedules operations that change the cluster (system under test)
 # and model state.
@@ -88,3 +88,7 @@ clustering_descriptor_selector:
       DELETE_COLUMN: 1
     column_mask_bitsets: null
     max_partition_size: 100
+
+# Default Row Visitor
+row_visitor:
+  default: {}
\ No newline at end of file
diff --git a/conf/external.yaml b/conf/external.yaml
index 742b624..770dda9 100644
--- a/conf/external.yaml
+++ b/conf/external.yaml
@@ -30,10 +30,10 @@ truncate_table: false
 # be taken to map a real-time timestamp from the value retrieved from the database in order
 # to map it back to the logical timestamp of the operation that wrote this value.
 clock:
-  debug_approximate_monotonic:
-    historySize: 7300
-    epochPeriod: 1
-    epochTimeUnit: "SECONDS"
+  approximate_monotonic:
+    history_size: 7300
+    epoch_length: 1
+    epoch_time_unit: "SECONDS"
 
 # Runner is a is a component that schedules operations that change the cluster (system under test)
 # and model state.
@@ -89,3 +89,7 @@ clustering_descriptor_selector:
       DELETE_COLUMN: 1
     column_mask_bitsets: null
     max_partition_size: 100
+
+# Default Row Visitor
+row_visitor:
+  default: {}
\ No newline at end of file
diff --git a/docker/Dockerfile.local b/docker/Dockerfile.local
index caaa551..5bc3943 100644
--- a/docker/Dockerfile.local
+++ b/docker/Dockerfile.local
@@ -30,6 +30,7 @@ COPY ./harry-runner/target/*.jar /opt/harry/lib/
 COPY ./harry-integration/target/lib/ /opt/harry/lib/
 COPY ./harry-integration/target/*.jar /opt/harry/
 COPY ./test/conf/logback-dtest.xml /opt/harry/test/conf/logback-dtest.xml
+COPY ./conf/example.yaml /opt/harry/example.yaml
 COPY ./docker/run.sh /opt/harry/
 
 WORKDIR /opt/harry
diff --git a/docker/run.sh b/docker/run.sh
index cde13f4..490b21f 100755
--- a/docker/run.sh
+++ b/docker/run.sh
@@ -74,7 +74,8 @@ while true; do
        -Dorg.apache.cassandra.test.logback.configurationFile=file:///opt/harry/test/conf/logback-dtest.xml \
        -cp /opt/harry/lib/*:/opt/harry/harry-integration-0.0.1-SNAPSHOT.jar \
        -Dharry.root=${HARRY_DIR} \
-       harry.runner.HarryRunnerJvm
+       harry.runner.HarryRunnerJvm \
+       /opt/harry/example.yaml
 
    if [ $? -ne 0 ]; then
       if [ -e "failure.dump" ]; then
diff --git a/harry-core/src/harry/core/Configuration.java b/harry-core/src/harry/core/Configuration.java
index 0c37f74..e01eac4 100644
--- a/harry-core/src/harry/core/Configuration.java
+++ b/harry-core/src/harry/core/Configuration.java
@@ -68,6 +68,7 @@ public class Configuration
                                   .disable(YAMLGenerator.Feature.CANONICAL_OUTPUT)
                                   .enable(YAMLGenerator.Feature.INDENT_ARRAYS));
         mapper.registerSubtypes(Configuration.DebugApproximateMonotonicClockConfiguration.class);
+        mapper.registerSubtypes(Configuration.ApproximateMonotonicClockConfiguration.class);
         mapper.registerSubtypes(Configuration.ConcurrentRunnerConfig.class);
 
         mapper.registerSubtypes(Configuration.ExhaustiveCheckerConfig.class);
diff --git a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
index 969e1d1..2cf22e7 100644
--- a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
+++ b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
@@ -35,7 +35,7 @@ import java.util.concurrent.TimeUnit;
 
 public class ExternalClusterSut implements SystemUnderTest
 {
-    public static void registerSubtype()
+    public static void registerSubtypes()
     {
         Configuration.registerSubtypes(ExternalSutConfiguration.class);
     }
diff --git a/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java b/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java
index 7dd8082..c926693 100644
--- a/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java
+++ b/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java
@@ -5,29 +5,16 @@ import harry.model.sut.external.ExternalClusterSut;
 import harry.runner.HarryRunner;
 
 import java.io.File;
-import java.io.FileNotFoundException;
 
 public class HarryRunnerExternal implements HarryRunner {
 
     public static void main(String[] args) throws Throwable {
-
-        ExternalClusterSut.registerSubtype();
-
-        if (args.length == 0) {
-            throw new RuntimeException("Harry config YAML not provided.");
-        }
-
-        File configFile =  new File(args[0]);
-        if (!configFile.exists()) {
-            throw new FileNotFoundException(configFile.getAbsolutePath());
-        }
-        if (!configFile.canRead()) {
-            throw new RuntimeException("Cannot read config file, check your permissions on " + configFile.getAbsolutePath());
-        }
-
-        Configuration config = Configuration.fromFile(configFile);
+        ExternalClusterSut.registerSubtypes();
 
         HarryRunner runner = new HarryRunnerExternal();
-        runner.run(config.system_under_test);
+        File configFile = runner.loadConfig(args);
+
+        Configuration configuration = Configuration.fromFile(configFile);
+        runner.run(configuration);
     }
 }
diff --git a/harry-integration/src/harry/model/sut/InJvmSut.java b/harry-integration/src/harry/model/sut/InJvmSut.java
index e39c4ab..d6b21a2 100644
--- a/harry-integration/src/harry/model/sut/InJvmSut.java
+++ b/harry-integration/src/harry/model/sut/InJvmSut.java
@@ -39,7 +39,7 @@ import org.apache.cassandra.distributed.api.ConsistencyLevel;
 
 public class InJvmSut implements SystemUnderTest
 {
-    public static void init()
+    public static void registerSubtypes()
     {
         Configuration.registerSubtypes(InJvmSutConfiguration.class);
     }
diff --git a/harry-integration/src/harry/runner/HarryRunnerJvm.java b/harry-integration/src/harry/runner/HarryRunnerJvm.java
index 9b4093c..71c6620 100644
--- a/harry-integration/src/harry/runner/HarryRunnerJvm.java
+++ b/harry-integration/src/harry/runner/HarryRunnerJvm.java
@@ -1,17 +1,20 @@
 package harry.runner;
 
+import harry.core.Configuration;
 import harry.model.sut.InJvmSut;
 
+import java.io.File;
+
 public class HarryRunnerJvm extends org.apache.cassandra.distributed.test.TestBaseImpl implements HarryRunner {
 
     public static void main(String[] args) throws Throwable {
-        HarryRunnerJvm runner = new HarryRunnerJvm();
+        InJvmSut.registerSubtypes();
 
-        InJvmSut.InJvmSutConfiguration config = new InJvmSut.InJvmSutConfiguration(3,
-                10,
-                System.getProperty("harry.root", "/tmp/cassandra/harry/") + System.currentTimeMillis());
+        HarryRunnerJvm runner = new HarryRunnerJvm();
+        File configFile = runner.loadConfig(args);
 
-        runner.run(config);
+        Configuration configuration = Configuration.fromFile(configFile);
+        runner.run(configuration);
     }
 
 
diff --git a/harry-integration/src/harry/runner/Reproduce.java b/harry-integration/src/harry/runner/Reproduce.java
index ecea98e..049c864 100644
--- a/harry-integration/src/harry/runner/Reproduce.java
+++ b/harry-integration/src/harry/runner/Reproduce.java
@@ -33,7 +33,7 @@ public class Reproduce extends TestBaseImpl
 
     public void runWithInJvmDtest() throws Throwable
     {
-        InJvmSut.init();
+        InJvmSut.registerSubtypes();
 
         System.setProperty("cassandra.disable_tcactive_openssl", "true");
         System.setProperty("relocated.shaded.io.netty.transport.noNative", "true");
diff --git a/harry-runner/src/harry/runner/HarryRunner.java b/harry-runner/src/harry/runner/HarryRunner.java
index a2ee043..a18885b 100644
--- a/harry-runner/src/harry/runner/HarryRunner.java
+++ b/harry-runner/src/harry/runner/HarryRunner.java
@@ -19,6 +19,7 @@
 package harry.runner;
 
 import java.io.File;
+import java.io.FileNotFoundException;
 import java.util.Random;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ScheduledExecutorService;
@@ -43,7 +44,7 @@ public interface HarryRunner
 
     Logger logger = LoggerFactory.getLogger(HarryRunner.class);
 
-    default void run(Configuration.SutConfiguration sutConfig) throws Throwable
+    default void run(Configuration configuration) throws Throwable
     {
         System.setProperty("cassandra.disable_tcactive_openssl", "true");
         System.setProperty("relocated.shaded.io.netty.transport.noNative", "true");
@@ -52,35 +53,7 @@ public interface HarryRunner
         ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
         executor.setRemoveOnCancelPolicy(true);
 
-        Configuration.ConfigurationBuilder configuration = new Configuration.ConfigurationBuilder();
-
-        long seed = System.currentTimeMillis();
-        configuration.setSeed(seed)
-                     .setSUT(sutConfig)
-                     .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(10, 100))
-                     .setClusteringDescriptorSelector((builder) -> {
-                         builder.setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(10))
-                                .setRowsPerModificationDistribution(new Configuration.ConstantDistributionConfig(10))
-                                .setMaxPartitionSize(100)
-                                .setOperationKindWeights(new Configuration.OperationKindSelectorBuilder()
-                                                         .addWeight(OpSelectors.OperationKind.DELETE_ROW, 1)
-                                                         .addWeight(OpSelectors.OperationKind.DELETE_COLUMN, 1)
-                                                         .addWeight(OpSelectors.OperationKind.DELETE_RANGE, 1)
-                                                         .addWeight(OpSelectors.OperationKind.WRITE, 97)
-                                                         .build());
-                     })
-                     .setRowVisitor(new Configuration.DefaultRowVisitorConfiguration())
-                     .setClock(new Configuration.ApproximateMonotonicClockConfiguration((int) TimeUnit.HOURS.toSeconds(2) + 100,
-                                                                                        1,
-                                                                                        TimeUnit.SECONDS))
-                     .setRunTime(2, TimeUnit.HOURS)
-                     .setCreateSchema(true)
-                     .setTruncateTable(false)
-                     .setDropSchema(false)
-                     .setModel(ExhaustiveChecker::new)
-                     .setRunner(new Configuration.ConcurrentRunnerConfig(1, 1, 1));
-
-        Runner runner = configuration.build().createRunner();
+        Runner runner = configuration.createRunner();
         Run run = runner.getRun();
 
         CompletableFuture progress = runner.initAndStartAll();
@@ -145,6 +118,28 @@ public interface HarryRunner
     }
 
     /**
+     * Parses the command-line args and returns a File for the configuration YAML.
+     * @param args Command-line args.
+     * @return Configuration YAML file.
+     * @throws Exception If file is not found or cannot be read.
+     */
+    default File loadConfig(String[] args) throws Exception {
+        if (args == null || args.length == 0) {
+            throw new Exception("Harry config YAML not provided.");
+        }
+
+        File configFile =  new File(args[0]);
+        if (!configFile.exists()) {
+            throw new FileNotFoundException(configFile.getAbsolutePath());
+        }
+        if (!configFile.canRead()) {
+            throw new Exception("Cannot read config file, check your permissions on " + configFile.getAbsolutePath());
+        }
+
+        return configFile;
+    }
+
+    /**
      * If you want to see how Harry detects problems!
      */
     public static void scheduleCorruption(Run run, ScheduledExecutorService executor)
diff --git a/run-external.sh b/run-external.sh
new file mode 100755
index 0000000..6263110
--- /dev/null
+++ b/run-external.sh
@@ -0,0 +1,3 @@
+#!/bin/sh
+
+java -Dlogback.configurationFile=test/conf/logback-dtest.xml -jar harry-integration-external/target/harry-integration-external-0.0.1-SNAPSHOT.jar conf/external.yaml
diff --git a/docker/run.sh b/run-jvm.sh
similarity index 50%
copy from docker/run.sh
copy to run-jvm.sh
index cde13f4..6c330f8 100755
--- a/docker/run.sh
+++ b/run-jvm.sh
@@ -1,34 +1,6 @@
-#   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.
+#!/bin/sh
 
-cd /opt/harry/
-
-HARRY_DIR=/cassandra/harry/
-ls -1 /mnt/harry-failures/* | xargs rm -fr
-local_run=false
-if [ $1 = "local_run" ]; then
-   local_run=true
-fi
-
-if [ "$local_run" = true ] ; then
-    echo "Running locally..."
-fi
-
-while true; do 
-  java -ea \
+java -ea \
        -Xms4g \
        -Xmx4g \
        -XX:MaxRAM=4g \
@@ -71,35 +43,7 @@ while true; do
        --add-opens java.base/jdk.internal.module=ALL-UNNAMED \
        --add-opens java.base/jdk.internal.util.jar=ALL-UNNAMED \
        --add-opens jdk.management/com.sun.management.internal=ALL-UNNAMED \
-       -Dorg.apache.cassandra.test.logback.configurationFile=file:///opt/harry/test/conf/logback-dtest.xml \
-       -cp /opt/harry/lib/*:/opt/harry/harry-integration-0.0.1-SNAPSHOT.jar \
-       -Dharry.root=${HARRY_DIR} \
-       harry.runner.HarryRunnerJvm
-
-   if [ $? -ne 0 ]; then
-      if [ -e "failure.dump" ]; then
-	echo "Creating failure dump..."
-        FAILURES_DIR="/opt/harry/dump/"
-	RUN="run-$(date +%Y%m%d%H%M%S)-${RANDOM}"
-        mkdir ${FAILURES_DIR}
-        mkdir ${FAILURES_DIR}cluster-state
-        mv ${HARRY_DIR}* ${FAILURES_DIR}/cluster-state
-        mv operation.log ${FAILURES_DIR}/
-        mv failure.dump ${FAILURES_DIR}/
-        mv run.yaml ${FAILURES_DIR}/
-
-        if [ "$local_run" = true ] ; then
-	    mv ${FAILURES_DIR}/* /shared/
-	else
-	    echo "TODO"
-	fi
-      fi
-  fi
-
- if [ "$local_run" = true ] ; then
-    exit 0
-  else
-    rm -fr ${HARRY_DIR}*
-    sleep 1
-  fi
-done
+       -Dorg.apache.cassandra.test.logback.configurationFile=file://test/conf/logback-dtest.xml \
+       -cp harry-integration/target/harry-integration-0.0.1-SNAPSHOT.jar:$(find harry-integration/target/dependency/*.jar | tr -s '\n' ':'). \
+       harry.runner.HarryRunnerJvm \
+       conf/example.yaml
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@cassandra.apache.org
For additional commands, e-mail: commits-help@cassandra.apache.org