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:39 UTC

[cassandra-harry] 02/10: Allow for YAML-based configuration for execution against an external cluster; Build an uber jar for harry-integration-external.

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 4de2debdcab63443738d8da936db54e3195ed45f
Author: Gianluca Righetto <gi...@menttis.com>
AuthorDate: Mon Dec 7 13:14:40 2020 -0300

    Allow for YAML-based configuration for execution against an external cluster; Build an uber jar for harry-integration-external.
---
 conf/external.yaml                                 | 91 ++++++++++++++++++++++
 harry-core/src/harry/core/Configuration.java       |  7 +-
 harry-integration-external/pom.xml                 | 39 +++++++++-
 .../model/sut/external/ExternalClusterSut.java     | 59 ++++++++++----
 .../harry/runner/external/HarryRunnerExternal.java | 33 ++++++++
 harry-integration/pom.xml                          |  8 +-
 .../src/harry/runner/HarryRunnerJvm.java           | 18 +++++
 .../src/harry/runner/Reproduce.java                |  0
 harry-runner/pom.xml                               | 11 ---
 harry-runner/src/harry/runner/HarryRunner.java     | 35 ++-------
 .../src/harry/runner/ThrowingRunnable.java         | 15 ++++
 pom.xml                                            |  7 ++
 12 files changed, 268 insertions(+), 55 deletions(-)

diff --git a/conf/external.yaml b/conf/external.yaml
new file mode 100644
index 0000000..742b624
--- /dev/null
+++ b/conf/external.yaml
@@ -0,0 +1,91 @@
+#   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.
+
+seed: 1596731732524
+
+# Default schema provider generates random schema
+schema_provider:
+  default: {}
+
+drop_schema: false
+create_schema: true
+truncate_table: false
+
+# Clock is a component responsible for mapping _logical_ timestamps to _real-time_ ones.
+#
+# When reproducing test failures, and for validation purposes, a snapshot of such clock can
+# 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"
+
+# Runner is a is a component that schedules operations that change the cluster (system under test)
+# and model state.
+runner:
+  concurrent:
+    writer_threads: 2
+    round_robin_validator_threads: 1
+    recent_partition_validator_threads: 1
+
+run_time: 2
+run_time_unit: "HOURS"
+
+# System under test: a Cassandra node or cluster. Default implementation is in_jvm (in-jvm DTest cluster).
+# Harry also supports external clusters.
+system_under_test:
+  external:
+    contact_points: 127.0.0.1
+    port: 9042
+    username: null
+    password: null
+
+# Model is responsible for tracking logical timestamps that
+model:
+  exhaustive_checker:
+    max_seen_lts: 19
+    max_complete_lts: 16
+
+# Partition descriptor selector controls how partitions is selected based on the current logical
+# timestamp. Default implementation is a sliding window of partition descriptors that will visit
+# one partition after the other in the window `slide_after_repeats` times. After that will
+# retire one partition descriptor, and pick one instead of it.
+partition_descriptor_selector:
+  default:
+    window_size: 10
+    slide_after_repeats: 100
+
+# Clustering descriptor selector controls how clusterings are picked within the partition:
+# how many rows there can be in a partition, how many rows will be visited for a logical timestamp,
+# how many operations there will be in batch, what kind of operations there will and how often
+# each kind of operation is going to occur.
+clustering_descriptor_selector:
+  default:
+    modifications_per_lts:
+      type: "constant"
+      constant: 10
+    rows_per_modification:
+      type: "constant"
+      constant: 10
+    operation_kind_weights:
+      WRITE: 97
+      DELETE_RANGE: 1
+      DELETE_ROW: 1
+      DELETE_COLUMN: 1
+    column_mask_bitsets: null
+    max_partition_size: 100
diff --git a/harry-core/src/harry/core/Configuration.java b/harry-core/src/harry/core/Configuration.java
index 3a68053..0c37f74 100644
--- a/harry-core/src/harry/core/Configuration.java
+++ b/harry-core/src/harry/core/Configuration.java
@@ -160,9 +160,14 @@ public class Configuration
 
     public static Configuration fromFile(String path)
     {
+        return fromFile(new File(path));
+    }
+
+    public static Configuration fromFile(File file)
+    {
         try
         {
-            return mapper.readValue(new File(path), Configuration.class);
+            return mapper.readValue(file, Configuration.class);
         }
         catch (Throwable t)
         {
diff --git a/harry-integration-external/pom.xml b/harry-integration-external/pom.xml
index 3e4e72e..7af3301 100755
--- a/harry-integration-external/pom.xml
+++ b/harry-integration-external/pom.xml
@@ -36,7 +36,13 @@
         <dependency>
             <groupId>org.apache.cassandra</groupId>
             <artifactId>harry-core</artifactId>
-	    <version>${project.parent.version}</version>
+	        <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.cassandra</groupId>
+            <artifactId>harry-runner</artifactId>
+            <version>${project.parent.version}</version>
         </dependency>
 
         <dependency>
@@ -46,11 +52,42 @@
         </dependency>
 
         <dependency>
+            <groupId>ch.qos.logback</groupId>
+            <artifactId>logback-classic</artifactId>
+        </dependency>
+
+        <dependency>
             <groupId>org.quicktheories</groupId>
             <artifactId>quicktheories</artifactId>
             <version>0.25</version>
             <scope>test</scope>
         </dependency>
     </dependencies>
+
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-shade-plugin</artifactId>
+                <version>3.2.4</version>
+                <executions>
+                    <execution>
+                        <phase>package</phase>
+                        <goals>
+                            <goal>shade</goal>
+                        </goals>
+                        <configuration>
+                            <transformers>
+                                <transformer implementation="org.apache.maven.plugins.shade.resource.ManifestResourceTransformer">
+                                    <mainClass>harry.runner.external.HarryRunnerExternal</mainClass>
+                                </transformer>
+                            </transformers>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
+        </plugins>
+    </build>
+
 </project>
 
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 146a162..969e1d1 100644
--- a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
+++ b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java
@@ -18,26 +18,28 @@
 
 package harry.model.sut.external;
 
+import com.datastax.driver.core.*;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.google.common.util.concurrent.FutureCallback;
+import com.google.common.util.concurrent.Futures;
+import harry.core.Configuration;
+import harry.model.sut.SystemUnderTest;
+
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 
-import com.google.common.util.concurrent.FutureCallback;
-import com.google.common.util.concurrent.Futures;
-
-import com.datastax.driver.core.Cluster;
-import com.datastax.driver.core.ColumnDefinitions;
-import com.datastax.driver.core.ConsistencyLevel;
-import com.datastax.driver.core.QueryOptions;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.Session;
-import harry.model.sut.SystemUnderTest;
-
 public class ExternalClusterSut implements SystemUnderTest
 {
+    public static void registerSubtype()
+    {
+        Configuration.registerSubtypes(ExternalSutConfiguration.class);
+    }
+
     private final Session session;
     private final ExecutorService executor;
 
@@ -52,12 +54,14 @@ public class ExternalClusterSut implements SystemUnderTest
         this.executor = Executors.newFixedThreadPool(threads);
     }
 
-    public static ExternalClusterSut create()
+    public static ExternalClusterSut create(ExternalSutConfiguration config)
     {
         // TODO: close Cluster and Session!
         return new ExternalClusterSut(Cluster.builder()
                                              .withQueryOptions(new QueryOptions().setConsistencyLevel(ConsistencyLevel.QUORUM))
-                                             .addContactPoints("127.0.0.1")
+                                             .addContactPoints(config.contactPoints)
+                                             .withPort(config.port)
+                                             .withCredentials(config.username, config.password)
                                              .build()
                                              .connect());
     }
@@ -145,4 +149,31 @@ public class ExternalClusterSut implements SystemUnderTest
 
         return future;
     }
+
+    @JsonTypeName("external")
+    public static class ExternalSutConfiguration implements Configuration.SutConfiguration
+    {
+
+        private final String contactPoints;
+        private final int port;
+        private final String username;
+        private final String password;
+
+        @JsonCreator
+        public ExternalSutConfiguration(@JsonProperty(value = "contact_points") String contactPoints,
+                                        @JsonProperty(value = "port") int port,
+                                        @JsonProperty(value = "username") String username,
+                                        @JsonProperty(value = "password") String password)
+        {
+            this.contactPoints = contactPoints;
+            this.port = port;
+            this.username = username;
+            this.password = password;
+        }
+
+        public SystemUnderTest make()
+        {
+            return ExternalClusterSut.create(this);
+        }
+    }
 }
\ No newline at end of file
diff --git a/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java b/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java
new file mode 100644
index 0000000..7dd8082
--- /dev/null
+++ b/harry-integration-external/src/harry/runner/external/HarryRunnerExternal.java
@@ -0,0 +1,33 @@
+package harry.runner.external;
+
+import harry.core.Configuration;
+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);
+
+        HarryRunner runner = new HarryRunnerExternal();
+        runner.run(config.system_under_test);
+    }
+}
diff --git a/harry-integration/pom.xml b/harry-integration/pom.xml
index 3bc873a..c27b55d 100755
--- a/harry-integration/pom.xml
+++ b/harry-integration/pom.xml
@@ -36,7 +36,13 @@
         <dependency>
             <groupId>org.apache.cassandra</groupId>
             <artifactId>harry-core</artifactId>
-	    <version>${project.parent.version}</version>
+	        <version>${project.parent.version}</version>
+        </dependency>
+
+        <dependency>
+            <groupId>org.apache.cassandra</groupId>
+            <artifactId>harry-runner</artifactId>
+            <version>${project.parent.version}</version>
         </dependency>
 
         <dependency>
diff --git a/harry-integration/src/harry/runner/HarryRunnerJvm.java b/harry-integration/src/harry/runner/HarryRunnerJvm.java
new file mode 100644
index 0000000..9b4093c
--- /dev/null
+++ b/harry-integration/src/harry/runner/HarryRunnerJvm.java
@@ -0,0 +1,18 @@
+package harry.runner;
+
+import harry.model.sut.InJvmSut;
+
+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.InJvmSutConfiguration config = new InJvmSut.InJvmSutConfiguration(3,
+                10,
+                System.getProperty("harry.root", "/tmp/cassandra/harry/") + System.currentTimeMillis());
+
+        runner.run(config);
+    }
+
+
+}
diff --git a/harry-runner/src/harry/runner/Reproduce.java b/harry-integration/src/harry/runner/Reproduce.java
similarity index 100%
rename from harry-runner/src/harry/runner/Reproduce.java
rename to harry-integration/src/harry/runner/Reproduce.java
diff --git a/harry-runner/pom.xml b/harry-runner/pom.xml
index a73436b..c5c401c 100755
--- a/harry-runner/pom.xml
+++ b/harry-runner/pom.xml
@@ -39,17 +39,6 @@
             <version>${project.parent.version}</version>
         </dependency>
 
-        <dependency>
-            <groupId>org.apache.cassandra</groupId>
-            <artifactId>harry-integration</artifactId>
-            <version>${project.parent.version}</version>
-        </dependency>
-
-        <dependency>
-            <groupId>org.apache.cassandra</groupId>
-            <artifactId>cassandra-dtest-shaded</artifactId>
-        </dependency>
-
     </dependencies>
 </project>
 
diff --git a/harry-runner/src/harry/runner/HarryRunner.java b/harry-runner/src/harry/runner/HarryRunner.java
index 9c9c86b..a2ee043 100644
--- a/harry-runner/src/harry/runner/HarryRunner.java
+++ b/harry-runner/src/harry/runner/HarryRunner.java
@@ -18,6 +18,7 @@
 
 package harry.runner;
 
+import java.io.File;
 import java.util.Random;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ScheduledExecutorService;
@@ -36,21 +37,13 @@ import harry.corruptor.HideValueCorruptor;
 import harry.corruptor.QueryResponseCorruptor;
 import harry.model.ExhaustiveChecker;
 import harry.model.OpSelectors;
-import harry.model.sut.InJvmSut;
-import org.apache.cassandra.distributed.impl.IsolatedExecutor;
-import org.apache.cassandra.distributed.test.TestBaseImpl;
 
-public class HarryRunner extends TestBaseImpl
+public interface HarryRunner
 {
 
-    private static final Logger logger = LoggerFactory.getLogger(HarryRunner.class);
+    Logger logger = LoggerFactory.getLogger(HarryRunner.class);
 
-    static
-    {
-        KEYSPACE = "harry";
-    }
-
-    public void runWithInJvmDtest() throws Throwable
+    default void run(Configuration.SutConfiguration sutConfig) throws Throwable
     {
         System.setProperty("cassandra.disable_tcactive_openssl", "true");
         System.setProperty("relocated.shaded.io.netty.transport.noNative", "true");
@@ -63,9 +56,7 @@ public class HarryRunner extends TestBaseImpl
 
         long seed = System.currentTimeMillis();
         configuration.setSeed(seed)
-                     .setSUT(new InJvmSut.InJvmSutConfiguration(3,
-                                                                10,
-                                                                System.getProperty("harry.root", "/tmp/cassandra/harry/") + System.currentTimeMillis()))
+                     .setSUT(sutConfig)
                      .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(10, 100))
                      .setClusteringDescriptorSelector((builder) -> {
                          builder.setNumberOfModificationsDistribution(new Configuration.ConstantDistributionConfig(10))
@@ -107,7 +98,8 @@ public class HarryRunner extends TestBaseImpl
                 return a;
             }).get(run.snapshot.run_time_unit.toSeconds(run.snapshot.run_time) + 30,
                    TimeUnit.SECONDS);
-            ((Throwable) result).printStackTrace();
+            if (result instanceof Throwable)
+                ((Throwable) result).printStackTrace();
 
         }
         catch (Throwable e)
@@ -140,7 +132,7 @@ public class HarryRunner extends TestBaseImpl
         }
     }
 
-    public void tryRun(IsolatedExecutor.ThrowingRunnable runnable)
+    default void tryRun(ThrowingRunnable runnable)
     {
         try
         {
@@ -190,15 +182,4 @@ public class HarryRunner extends TestBaseImpl
         }, 30, 1, TimeUnit.SECONDS);
     }
 
-    public static void main(String[] args)
-    {
-        try
-        {
-            new HarryRunner().runWithInJvmDtest();
-        }
-        catch (Throwable t)
-        {
-            t.printStackTrace();
-        }
-    }
 }
\ No newline at end of file
diff --git a/harry-runner/src/harry/runner/ThrowingRunnable.java b/harry-runner/src/harry/runner/ThrowingRunnable.java
new file mode 100644
index 0000000..399997b
--- /dev/null
+++ b/harry-runner/src/harry/runner/ThrowingRunnable.java
@@ -0,0 +1,15 @@
+package harry.runner;
+
+public interface ThrowingRunnable {
+    void run() throws Throwable;
+
+    static Runnable toRunnable(ThrowingRunnable runnable) {
+        return () -> {
+            try {
+                runnable.run();
+            } catch (Throwable var2) {
+                throw new RuntimeException(var2);
+            }
+        };
+    }
+}
\ No newline at end of file
diff --git a/pom.xml b/pom.xml
index f582caa..35e1712 100755
--- a/pom.xml
+++ b/pom.xml
@@ -180,6 +180,13 @@
                 <artifactId>hamcrest-all</artifactId>
                 <version>1.3</version>
             </dependency>
+
+            <!-- this should only be included in the uber jar for standalone execution or as a test dep -->
+            <dependency>
+                <groupId>ch.qos.logback</groupId>
+                <artifactId>logback-classic</artifactId>
+                <version>1.2.3</version>
+            </dependency>
         </dependencies>
     </dependencyManagement>
 


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