This is an automated email from the ASF dual-hosted git repository. ifesdjeen pushed a commit to branch CASSANDRA-16262-2 in repository https://gitbox.apache.org/repos/asf/cassandra-harry.git
commit 57053decc6b810d1148e1b7f680e54feece57d99 Author: Alex Petrov <[email protected]> AuthorDate: Mon Jul 12 17:54:42 2021 +0200 Integration improvements --- .../test/resources/single_partition_test.yml | 55 ++++++ harry-integration-external/pom.xml | 5 + .../model/sut/external/ExternalClusterSut.java | 14 +- harry-integration/pom.xml | 14 +- .../src/harry/model/sut/ExternalClusterSut.java | 187 --------------------- .../src/harry/model/sut/InJvmSutBase.java | 3 +- .../src/harry/runner/HarryRunnerJvm.java | 43 +++++ harry-integration/test/conf/cassandra.yaml | 44 +++++ .../test/harry/ddl/SchemaGenTest.java | 15 +- .../generators/DataGeneratorsIntegrationTest.java | 104 ++++++++++++ .../test/harry/model/IntegrationTestBase.java | 6 +- .../model/QuiescentCheckerIntegrationTest.java | 3 +- .../test/harry/model/TestEveryClustering.java | 89 ++++++++++ .../test/harry/op/RowVisitorTest.java | 8 +- .../test/resources/single_partition_test.yml | 55 ++++++ 15 files changed, 435 insertions(+), 210 deletions(-) diff --git a/harry-integration-backup/test/resources/single_partition_test.yml b/harry-integration-backup/test/resources/single_partition_test.yml new file mode 100644 index 0000000..0ebe2aa --- /dev/null +++ b/harry-integration-backup/test/resources/single_partition_test.yml @@ -0,0 +1,55 @@ +seed: 1 + +# Default schema provider generates random schema +schema_provider: + default: {} + +drop_schema: false +create_schema: true +truncate_table: false + +clock: + offset: + offset: 1000 + +run_time: 10 +run_time_unit: "MINUTES" + +system_under_test: + println: {} + +partition_descriptor_selector: + always_same: + pd: 12345 + +clustering_descriptor_selector: + default: + modifications_per_lts: + type: "constant" + constant: 2 + rows_per_modification: + type: "constant" + constant: 2 + operation_kind_weights: + DELETE_RANGE: 1 + DELETE_SLICE: 1 + DELETE_ROW: 1 + DELETE_COLUMN: 1 + DELETE_PARTITION: 1 + DELETE_COLUMN_WITH_STATICS: 1 + INSERT_WITH_STATICS: 24 + INSERT: 24 + UPDATE_WITH_STATICS: 23 + UPDATE: 23 + column_mask_bitsets: null + max_partition_size: 100 + +data_tracker: + no_op: {} + +runner: + sequential: + partition_visitors: [] + +metric_reporter: + no_op: {} \ No newline at end of file diff --git a/harry-integration-external/pom.xml b/harry-integration-external/pom.xml index 8c57e0e..0b49f6d 100755 --- a/harry-integration-external/pom.xml +++ b/harry-integration-external/pom.xml @@ -34,6 +34,11 @@ <dependencies> <dependency> + <groupId>com.datastax.cassandra</groupId> + <artifactId>cassandra-driver-core</artifactId> + </dependency> + + <dependency> <groupId>org.apache.cassandra</groupId> <artifactId>harry-core</artifactId> <version>${project.parent.version}</version> 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 e471ede..76a4e3c 100644 --- a/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java +++ b/harry-integration-external/src/harry/model/sut/external/ExternalClusterSut.java @@ -93,9 +93,7 @@ public class ExternalClusterSut implements SystemUnderTest { try { - Statement st = new SimpleStatement(statement, bindings); - st.setConsistencyLevel(toDriverCl(cl)); - return resultSetToObjectArray(session.execute(st)); + return resultSetToObjectArray(session.execute(statement, bindings)); } catch (Throwable t) { @@ -134,9 +132,7 @@ public class ExternalClusterSut implements SystemUnderTest public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) { CompletableFuture<Object[][]> future = new CompletableFuture<>(); - Statement st = new SimpleStatement(statement, bindings); - st.setConsistencyLevel(toDriverCl(cl)); - Futures.addCallback(session.executeAsync(st), + Futures.addCallback(session.executeAsync(statement, bindings), new FutureCallback<ResultSet>() { public void onSuccess(ResultSet rows) @@ -185,8 +181,10 @@ public class ExternalClusterSut implements SystemUnderTest { switch (cl) { - case ALL: return com.datastax.driver.core.ConsistencyLevel.ALL; - case QUORUM: return com.datastax.driver.core.ConsistencyLevel.QUORUM; + case ALL: + return com.datastax.driver.core.ConsistencyLevel.ALL; + case QUORUM: + return com.datastax.driver.core.ConsistencyLevel.QUORUM; } throw new IllegalArgumentException("Don't know a CL: " + cl); } diff --git a/harry-integration/pom.xml b/harry-integration/pom.xml index 4695d38..0b2d131 100755 --- a/harry-integration/pom.xml +++ b/harry-integration/pom.xml @@ -34,9 +34,14 @@ <dependencies> <dependency> + <groupId>org.reflections</groupId> + <artifactId>reflections</artifactId> + </dependency> + + <dependency> <groupId>org.apache.cassandra</groupId> <artifactId>harry-core</artifactId> - <version>${project.parent.version}</version> + <version>${project.parent.version}</version> </dependency> <dependency> @@ -62,5 +67,12 @@ <scope>test</scope> </dependency> </dependencies> + <build> + <testResources> + <testResource> + <directory>test/resources</directory> + </testResource> + </testResources> + </build> </project> diff --git a/harry-integration/src/harry/model/sut/ExternalClusterSut.java b/harry-integration/src/harry/model/sut/ExternalClusterSut.java deleted file mode 100644 index 74085e4..0000000 --- a/harry-integration/src/harry/model/sut/ExternalClusterSut.java +++ /dev/null @@ -1,187 +0,0 @@ -/* - * 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.model.sut; - -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.QueryOptions; -import com.datastax.driver.core.ResultSet; -import com.datastax.driver.core.Row; -import com.datastax.driver.core.Session; -import com.datastax.driver.core.SimpleStatement; -import com.datastax.driver.core.Statement; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.annotation.JsonTypeName; -import harry.core.Configuration; - -public class ExternalClusterSut implements SystemUnderTest -{ - public static void init() - { - Configuration.registerSubtypes(ExternalClusterSutConfiguration.class); - } - - private final Session session; - private final ExecutorService executor; - - // TODO: pass cluster, not session - public ExternalClusterSut(Session session) - { - this(session, 10); - } - - public ExternalClusterSut(Session session, int threads) - { - this.session = session; - this.executor = Executors.newFixedThreadPool(threads); - } - - public static ExternalClusterSut create() - { - return new ExternalClusterSut(Cluster.builder() - .withQueryOptions(new QueryOptions().setConsistencyLevel(toDriverCl(ConsistencyLevel.QUORUM))) - .addContactPoints("127.0.0.1") - .build() - .connect()); - } - - public boolean isShutdown() - { - return session.isClosed(); - } - - public void shutdown() - { - session.close(); - executor.shutdown(); - try - { - executor.awaitTermination(60, TimeUnit.SECONDS); - } - catch (InterruptedException e) - { - throw new RuntimeException(e); - } - } - - // TODO: this is rather simplistic - public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) - { - int repeat = 10; - while (true) - { - try - { - Statement st = new SimpleStatement(statement, bindings); - st.setConsistencyLevel(toDriverCl(cl)); - return resultSetToObjectArray(session.execute(st)); - } - catch (Throwable t) - { - if (repeat < 0) - throw t; - - t.printStackTrace(); - repeat--; - // retry unconditionally - } - } - } - - public static Object[][] resultSetToObjectArray(ResultSet rs) - { - List<Row> rows = rs.all(); - if (rows.size() == 0) - return new Object[0][]; - Object[][] results = new Object[rows.size()][]; - for (int i = 0; i < results.length; i++) - { - Row row = rows.get(i); - ColumnDefinitions cds = row.getColumnDefinitions(); - Object[] result = new Object[cds.size()]; - for (int j = 0; j < cds.size(); j++) - { - if (!row.isNull(j)) - result[j] = row.getObject(j); - } - results[i] = result; - } - return results; - } - - public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) - { - CompletableFuture<Object[][]> future = new CompletableFuture<>(); - Statement st = new SimpleStatement(statement, bindings); - st.setConsistencyLevel(toDriverCl(cl)); - Futures.addCallback(session.executeAsync(st), - new FutureCallback<ResultSet>() - { - public void onSuccess(ResultSet rows) - { - future.complete(resultSetToObjectArray(rows)); - } - - public void onFailure(Throwable throwable) - { - future.completeExceptionally(throwable); - } - }, - executor); - - return future; - } - - public static com.datastax.driver.core.ConsistencyLevel toDriverCl(SystemUnderTest.ConsistencyLevel cl) - { - switch (cl) - { - case ALL: return com.datastax.driver.core.ConsistencyLevel.ALL; - case QUORUM: return com.datastax.driver.core.ConsistencyLevel.QUORUM; - } - throw new IllegalArgumentException("Don't know a CL: " + cl); - } - - @JsonTypeName("external") - public static class ExternalClusterSutConfiguration implements Configuration.SutConfiguration - { - public final String[] hosts; - - public ExternalClusterSutConfiguration(@JsonProperty(value = "hosts") String[] hosts) - { - this.hosts = hosts; - } - - public SystemUnderTest make() - { - Cluster cluster = Cluster.builder().addContactPoints(hosts).build(); - Session session = cluster.newSession().init(); - return new ExternalClusterSut(session); - } - } -} \ No newline at end of file diff --git a/harry-integration/src/harry/model/sut/InJvmSutBase.java b/harry-integration/src/harry/model/sut/InJvmSutBase.java index e29481f..aa2608f 100644 --- a/harry-integration/src/harry/model/sut/InJvmSutBase.java +++ b/harry-integration/src/harry/model/sut/InJvmSutBase.java @@ -30,20 +30,19 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; -import com.google.common.collect.Iterators; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import harry.core.Configuration; -import org.apache.cassandra.distributed.api.ConsistencyLevel; import org.apache.cassandra.distributed.api.Feature; import org.apache.cassandra.distributed.api.ICluster; import org.apache.cassandra.distributed.api.IInstance; import org.apache.cassandra.distributed.api.IInstanceConfig; import org.apache.cassandra.distributed.api.IMessage; import org.apache.cassandra.distributed.api.IMessageFilters; +import relocated.shaded.com.google.common.collect.Iterators; public class InJvmSutBase<NODE extends IInstance, CLUSTER extends ICluster<NODE>> implements SystemUnderTest.FaultInjectingSut { diff --git a/harry-integration/src/harry/runner/HarryRunnerJvm.java b/harry-integration/src/harry/runner/HarryRunnerJvm.java new file mode 100644 index 0000000..fbb30b9 --- /dev/null +++ b/harry-integration/src/harry/runner/HarryRunnerJvm.java @@ -0,0 +1,43 @@ +/* + * 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.runner; + +import harry.core.Configuration; +import harry.model.sut.InJvmSut; + +import java.io.File; + +public class HarryRunnerJvm extends HarryRunner { + + public static void main(String[] args) throws Throwable { + InJvmSut.init(); + + HarryRunnerJvm runner = new HarryRunnerJvm(); + File configFile = runner.loadConfig(args); + + Configuration configuration = Configuration.fromFile(configFile); + runner.run(configuration); + } + + + @Override + public void beforeRun(Runner runner) { + + } +} diff --git a/harry-integration/test/conf/cassandra.yaml b/harry-integration/test/conf/cassandra.yaml new file mode 100644 index 0000000..2536aa8 --- /dev/null +++ b/harry-integration/test/conf/cassandra.yaml @@ -0,0 +1,44 @@ +# +# Warning! +# Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing schemas in this file. +# +cluster_name: Test Cluster +memtable_allocation_type: heap_buffers +commitlog_sync: batch +commitlog_sync_batch_window_in_ms: 1.0 +commitlog_segment_size_in_mb: 5 +commitlog_directory: build/test/cassandra/commitlog +hints_directory: build/test/cassandra/hints +partitioner: org.apache.cassandra.dht.ByteOrderedPartitioner +listen_address: 127.0.0.1 +storage_port: 7010 +rpc_port: 9170 +start_native_transport: true +native_transport_port: 9042 +column_index_size_in_kb: 4 +saved_caches_directory: build/test/cassandra/saved_caches +data_file_directories: + - build/test/cassandra/data +disk_access_mode: mmap +seed_provider: + - class_name: org.apache.cassandra.locator.SimpleSeedProvider + parameters: + - seeds: "127.0.0.1" +endpoint_snitch: org.apache.cassandra.locator.SimpleSnitch +dynamic_snitch: true +request_scheduler: org.apache.cassandra.scheduler.RoundRobinScheduler +request_scheduler_id: keyspace +server_encryption_options: + internode_encryption: none + keystore: conf/.keystore + keystore_password: cassandra + truststore: conf/.truststore + truststore_password: cassandra +incremental_backups: true +concurrent_compactors: 4 +compaction_throughput_mb_per_sec: 0 +row_cache_class_name: org.apache.cassandra.cache.OHCProvider +row_cache_size_in_mb: 16 +enable_user_defined_functions: true +enable_scripted_user_defined_functions: true +enable_drop_compact_storage: true diff --git a/harry-integration/test/harry/ddl/SchemaGenTest.java b/harry-integration/test/harry/ddl/SchemaGenTest.java index 74a9c53..2f7e898 100644 --- a/harry-integration/test/harry/ddl/SchemaGenTest.java +++ b/harry-integration/test/harry/ddl/SchemaGenTest.java @@ -50,11 +50,12 @@ public class SchemaGenTest extends CQLTester // TODO: compact storage tests @Test - public void testSelectForwardAndReverseIteration() + public void testSelectForwardAndReverseIteration() throws Throwable { Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 4) .clusteringColumnCount(1, 10) .regularColumnCount(0, 10) + .staticColumnCount(0, 10) .generator(); @@ -78,11 +79,12 @@ public class SchemaGenTest extends CQLTester } @Test - public void createTableRoundTrip() + public void createTableRoundTrip() throws Throwable { Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(1, 10) - .clusteringColumnCount(0, 10) + .clusteringColumnCount(1, 10) .regularColumnCount(0, 10) + .staticColumnCount(0, 10) .generator(); TestRunner.test(gen, @@ -93,6 +95,7 @@ public class SchemaGenTest extends CQLTester compareColumns(schemaDefinition.partitionKeys, tableMetadata.partitionKeyColumns()); compareColumns(schemaDefinition.clusteringKeys, tableMetadata.clusteringColumns()); compareColumns(schemaDefinition.regularColumns, tableMetadata.regularColumns()); + compareColumns(schemaDefinition.staticColumns, tableMetadata.staticColumns()); }); } @@ -108,8 +111,8 @@ public class SchemaGenTest extends CQLTester ColumnSpec.regularColumn("v2", ColumnSpec.asciiType), ColumnSpec.regularColumn("v3", ColumnSpec.int64Type), ColumnSpec.regularColumn("v4", ColumnSpec.int64Type)), - Arrays.asList(ColumnSpec.staticColumn("regular1", ColumnSpec.asciiType), - ColumnSpec.staticColumn("regular2", ColumnSpec.int64Type))); + Arrays.asList(ColumnSpec.staticColumn("static1", ColumnSpec.asciiType), + ColumnSpec.staticColumn("static2", ColumnSpec.int64Type))); String tableDef = spec.compile().cql(); @@ -118,6 +121,7 @@ public class SchemaGenTest extends CQLTester compareColumns(spec.partitionKeys, tableMetadata.partitionKeyColumns()); compareColumns(spec.clusteringKeys, tableMetadata.clusteringColumns()); compareColumns(spec.regularColumns, tableMetadata.regularColumns()); + compareColumns(spec.staticColumns, tableMetadata.staticColumns()); } @@ -126,6 +130,7 @@ public class SchemaGenTest extends CQLTester { Gen<Pair<Integer, Integer>> ckCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); Gen<Pair<Integer, Integer>> regCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); +// Gen<Pair<Integer, Integer>> staticCounts = integers().between(0, 4).zip(integers().between(0, 6), Pair::create); Gen<Pair<Integer, Integer>> pkCounts = integers().between(1, 4).zip(integers().between(0, 6), Pair::create); Gen<SchemaGenerationInputs> inputs = pkCounts.zip(ckCounts, regCounts, diff --git a/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java index affd954..cadee0c 100644 --- a/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java +++ b/harry-integration/test/harry/generators/DataGeneratorsIntegrationTest.java @@ -19,11 +19,28 @@ package harry.generators; import java.util.Random; +import java.util.concurrent.CompletableFuture; import org.junit.Test; +import harry.core.Configuration; +import harry.core.Run; import harry.ddl.ColumnSpec; +import harry.ddl.SchemaGenerators; +import harry.ddl.SchemaSpec; +import harry.generators.distribution.Distribution; +import harry.model.NoOpChecker; +import harry.model.OpSelectors; +import harry.model.sut.SystemUnderTest; +import harry.runner.MutatingPartitionVisitor; +import harry.runner.MutatingRowVisitor; +import harry.runner.PartitionVisitor; +import harry.runner.SinglePartitionValidator; +import harry.util.TestRunner; import org.apache.cassandra.cql3.CQLTester; +import org.apache.cassandra.cql3.UntypedResultSet; +import org.apache.cassandra.distributed.impl.RowUtil; +import relocated.shaded.com.google.common.collect.Iterators; public class DataGeneratorsIntegrationTest extends CQLTester { @@ -60,4 +77,91 @@ public class DataGeneratorsIntegrationTest extends CQLTester } } } + + @Test + public void queryParseabilityTest() throws Throwable + { + Generator<SchemaSpec> gen = new SchemaGenerators.Builder(KEYSPACE).partitionKeyColumnCount(2, 4) + .clusteringColumnCount(1, 4) + .regularColumnCount(1, 4) + .staticColumnCount(1, 4) + .generator(); + + TestRunner.test(gen, + (schema) -> { + createTable(schema.compile().cql()); + + Configuration.ConfigurationBuilder builder = Configuration.fromFile(getClass().getClassLoader().getResource("single_partition_test.yml").getFile()) + .unbuild() + .setSchemaProvider(new Configuration.FixedSchemaProviderConfiguration(schema, null, null, null, null)) + .setSUT(CqlTesterSut::new); + + for (OpSelectors.OperationKind opKind : OpSelectors.OperationKind.values()) + { + Run run = builder + .setClusteringDescriptorSelector((rng, schema_) -> { + return new OpSelectors.DefaultDescriptorSelector(rng, + OpSelectors.columnSelectorBuilder().forAll(schema_).build(), + OpSelectors.OperationSelector.weighted(Surjections.weights(100), opKind), + new Distribution.ConstantDistribution(2), + new Distribution.ConstantDistribution(2), + 100); + }) + .build() + .createRun(); + + PartitionVisitor visitor = new MutatingPartitionVisitor(run, MutatingRowVisitor::new); + for (int lts = 0; lts < 100; lts++) + visitor.visitPartition(lts); + } + + Run run = builder.build() + .createRun(); + PartitionVisitor visitor = new SinglePartitionValidator(100, run, NoOpChecker::new); + for (int lts = 0; lts < 100; lts++) + visitor.visitPartition(lts); + + }); + + } + + public class CqlTesterSut implements SystemUnderTest + { + public boolean isShutdown() + { + return false; + } + + public void shutdown() + { + cleanup(); + } + + public void schemaChange(String statement) + { + createTable(statement); + } + + public Object[][] execute(String statement, ConsistencyLevel cl, Object... bindings) + { + try + { + UntypedResultSet res = DataGeneratorsIntegrationTest.this.execute(statement, bindings); + if (res == null) + return new Object[][] {}; + + return Iterators.toArray(RowUtil.toIter(res), Object[].class); + } + catch (Throwable throwable) + { + throw new RuntimeException(throwable); + } + } + + public CompletableFuture<Object[][]> executeAsync(String statement, ConsistencyLevel cl, Object... bindings) + { + return CompletableFuture.completedFuture(execute(statement, cl, bindings)); + } + } } + diff --git a/harry-integration/test/harry/model/IntegrationTestBase.java b/harry-integration/test/harry/model/IntegrationTestBase.java index 551124f..a6a9697 100644 --- a/harry-integration/test/harry/model/IntegrationTestBase.java +++ b/harry-integration/test/harry/model/IntegrationTestBase.java @@ -81,8 +81,10 @@ public class IntegrationTestBase extends TestBaseImpl .addWeight(OpSelectors.OperationKind.DELETE_SLICE, 1) .addWeight(OpSelectors.OperationKind.DELETE_PARTITION, 1) .addWeight(OpSelectors.OperationKind.DELETE_COLUMN_WITH_STATICS, 5) - .addWeight(OpSelectors.OperationKind.WRITE_WITH_STATICS, 45) - .addWeight(OpSelectors.OperationKind.WRITE, 45) + .addWeight(OpSelectors.OperationKind.INSERT_WITH_STATICS, 20) + .addWeight(OpSelectors.OperationKind.INSERT, 20) + .addWeight(OpSelectors.OperationKind.UPDATE_WITH_STATICS, 25) + .addWeight(OpSelectors.OperationKind.UPDATE, 25) .build()); } diff --git a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java index 4c83276..7d5956f 100644 --- a/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java +++ b/harry-integration/test/harry/model/QuiescentCheckerIntegrationTest.java @@ -128,7 +128,8 @@ public class QuiescentCheckerIntegrationTest extends ModelTestBase (t, run) -> { String expected = "doesn't match the one predicted by the model"; String expected2 = "don't match ones predicted by the model"; - if (t.getMessage().contains(expected) || t.getMessage().contains(expected2)) + String expected3 = "Found a row in the model that is not present in the resultset"; + if (t.getMessage().contains(expected) || t.getMessage().contains(expected2) || t.getMessage().contains(expected3)) return; throw new AssertionError(String.format("Exception string mismatch.\nExpected error: %s.\nActual error: %s", expected, t.getMessage()), diff --git a/harry-integration/test/harry/model/TestEveryClustering.java b/harry-integration/test/harry/model/TestEveryClustering.java new file mode 100644 index 0000000..f844b41 --- /dev/null +++ b/harry-integration/test/harry/model/TestEveryClustering.java @@ -0,0 +1,89 @@ +package harry.model; + +import java.util.HashSet; +import java.util.Set; +import java.util.function.Supplier; + +import org.junit.Assert; +import org.junit.Test; + +import harry.core.Configuration; +import harry.core.Run; +import harry.ddl.SchemaGenerators; +import harry.ddl.SchemaSpec; +import harry.generators.distribution.Distribution; +import harry.model.sut.SystemUnderTest; +import harry.operations.CompiledStatement; +import harry.operations.Relation; +import harry.runner.FaultInjectingPartitionVisitor; +import harry.runner.LoggingPartitionVisitor; +import harry.runner.MutatingPartitionVisitor; +import harry.runner.MutatingRowVisitor; +import harry.runner.PartitionVisitor; +import harry.runner.Query; +import harry.runner.QueryGenerator; +import org.apache.cassandra.distributed.api.IInvokableInstance; + +public class TestEveryClustering extends IntegrationTestBase +{ + int CYCLES = 1000; + + @Test + public void basicQuerySelectorTest() + { + Supplier<SchemaSpec> schemaGen = SchemaGenerators.progression(SchemaGenerators.DEFAULT_SWITCH_AFTER); + for (int cnt = 0; cnt < Integer.MAX_VALUE; cnt++) + { + beforeEach(); + SchemaSpec schemaSpec = schemaGen.get(); + + System.out.println(schemaSpec.compile().cql()); + int partitionSize = 1000; + + Configuration config = sharedConfiguration(cnt, schemaSpec) + .setPartitionDescriptorSelector(new Configuration.DefaultPDSelectorConfiguration(1, partitionSize)) + .setClusteringDescriptorSelector(sharedCDSelectorConfiguration() + .setNumberOfModificationsDistribution(() -> new Distribution.ConstantDistribution(1L)) + .setRowsPerModificationDistribution(() -> new Distribution.ConstantDistribution(1L)) + .setMaxPartitionSize(250) + .build()) + .build(); + + Run run = config.createRun(); + run.sut.schemaChange(run.schemaSpec.compile().cql()); + OpSelectors.MonotonicClock clock = run.clock; + + Set<Long> visitedCds = new HashSet<>(); + PartitionVisitor partitionVisitor = new LoggingPartitionVisitor(run, (r) -> { + return new MutatingRowVisitor(r) { + public CompiledStatement perform(OpSelectors.OperationKind op, long lts, long pd, long cd, long opId) + { + visitedCds.add(cd); + return super.perform(op, lts, pd, cd, opId); + } + }; + }); + sut.cluster().stream().forEach((IInvokableInstance node) -> node.nodetool("disableautocompaction")); + for (int i = 0; i < CYCLES; i++) + { + long lts = clock.nextLts(); + partitionVisitor.visitPartition(lts); + + if (i > 0 && i % 250 == 0) + sut.cluster().stream().forEach((IInvokableInstance node) -> node.nodetool("flush", schemaSpec.keyspace, schemaSpec.table)); + } + + for (Long cd : visitedCds) + { + Query query = new Query.SingleClusteringQuery(Query.QueryKind.SINGLE_CLUSTERING, + run.pdSelector.pd(0), + cd, + false, + Relation.eqRelations(run.schemaSpec.ckGenerator.slice(cd), run.schemaSpec.clusteringKeys), + run.schemaSpec); + Model model = new QuiescentChecker(run); + model.validate(query); + } + } + } +} diff --git a/harry-integration/test/harry/op/RowVisitorTest.java b/harry-integration/test/harry/op/RowVisitorTest.java index d7061f9..ae51115 100644 --- a/harry-integration/test/harry/op/RowVisitorTest.java +++ b/harry-integration/test/harry/op/RowVisitorTest.java @@ -83,10 +83,10 @@ public class RowVisitorTest extends CQLTester MutatingRowVisitor visitor = new MutatingRowVisitor(run); long[] descriptors = rand.next(4); - execute(visitor.write(Math.abs(descriptors[0]), - descriptors[1], - descriptors[2], - descriptors[3])); + execute(visitor.insert(Math.abs(descriptors[0]), + descriptors[1], + descriptors[2], + descriptors[3])); } } diff --git a/harry-integration/test/resources/single_partition_test.yml b/harry-integration/test/resources/single_partition_test.yml new file mode 100644 index 0000000..0ebe2aa --- /dev/null +++ b/harry-integration/test/resources/single_partition_test.yml @@ -0,0 +1,55 @@ +seed: 1 + +# Default schema provider generates random schema +schema_provider: + default: {} + +drop_schema: false +create_schema: true +truncate_table: false + +clock: + offset: + offset: 1000 + +run_time: 10 +run_time_unit: "MINUTES" + +system_under_test: + println: {} + +partition_descriptor_selector: + always_same: + pd: 12345 + +clustering_descriptor_selector: + default: + modifications_per_lts: + type: "constant" + constant: 2 + rows_per_modification: + type: "constant" + constant: 2 + operation_kind_weights: + DELETE_RANGE: 1 + DELETE_SLICE: 1 + DELETE_ROW: 1 + DELETE_COLUMN: 1 + DELETE_PARTITION: 1 + DELETE_COLUMN_WITH_STATICS: 1 + INSERT_WITH_STATICS: 24 + INSERT: 24 + UPDATE_WITH_STATICS: 23 + UPDATE: 23 + column_mask_bitsets: null + max_partition_size: 100 + +data_tracker: + no_op: {} + +runner: + sequential: + partition_visitors: [] + +metric_reporter: + no_op: {} \ No newline at end of file --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
