dcapwell commented on code in PR #3416: URL: https://github.com/apache/cassandra/pull/3416#discussion_r1777762435
########## test/distributed/org/apache/cassandra/fuzz/topology/AccordTopologyMixupTest.java: ########## @@ -0,0 +1,397 @@ +/* + * 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.fuzz.topology; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.function.BiFunction; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import javax.annotation.Nullable; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import accord.coordinate.Exhausted; +import accord.coordinate.Preempted; +import accord.coordinate.Timeout; +import accord.local.Node; +import accord.local.cfk.CommandsForKey; +import accord.primitives.Ranges; +import accord.primitives.Seekables; +import accord.primitives.Timestamp; +import accord.primitives.TxnId; +import accord.utils.Gen; +import accord.utils.Gens; +import accord.utils.Invariants; +import accord.utils.Property; +import accord.utils.Property.Command; +import accord.utils.RandomSource; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.JsonSerializer; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.SerializationFeature; +import com.fasterxml.jackson.databind.SerializerProvider; +import com.fasterxml.jackson.databind.module.SimpleModule; +import org.apache.cassandra.config.CassandraRelevantProperties; +import org.apache.cassandra.config.Config; +import org.apache.cassandra.cql3.ast.Mutation; +import org.apache.cassandra.cql3.ast.Statement; +import org.apache.cassandra.distributed.Cluster; +import org.apache.cassandra.distributed.api.ConsistencyLevel; +import org.apache.cassandra.distributed.api.IInstanceConfig; +import org.apache.cassandra.distributed.api.IInvokableInstance; +import org.apache.cassandra.distributed.api.SimpleQueryResult; +import org.apache.cassandra.distributed.shared.ClusterUtils; +import org.apache.cassandra.distributed.test.accord.AccordTestBase; +import org.apache.cassandra.schema.TableMetadata; +import org.apache.cassandra.service.accord.api.AccordAgent; +import org.apache.cassandra.service.accord.api.PartitionKey; +import org.apache.cassandra.service.consensus.TransactionalMode; +import org.apache.cassandra.utils.ASTGenerators; +import org.apache.cassandra.utils.CassandraGenerators; +import org.apache.cassandra.utils.Isolated; +import org.apache.cassandra.utils.JsonUtils; +import org.apache.cassandra.utils.Shared; + +import static org.apache.cassandra.utils.AccordGenerators.fromQT; + +public class AccordTopologyMixupTest extends TopologyMixupTestBase<AccordTopologyMixupTest.Spec> +{ + private static final Logger logger = LoggerFactory.getLogger(AccordTopologyMixupTest.class); + + static + { + CassandraRelevantProperties.ACCORD_AGENT_CLASS.setString(InterceptAgent.class.getName()); + // enable most expensive debugging checks + CassandraRelevantProperties.ACCORD_KEY_PARANOIA_CPU.setString(Invariants.Paranoia.QUADRATIC.name()); + CassandraRelevantProperties.ACCORD_KEY_PARANOIA_MEMORY.setString(Invariants.Paranoia.QUADRATIC.name()); + CassandraRelevantProperties.ACCORD_KEY_PARANOIA_COSTFACTOR.setString(Invariants.ParanoiaCostFactor.HIGH.name()); + } + + private static final List<TransactionalMode> TRANSACTIONAL_MODES = Stream.of(TransactionalMode.values()).filter(t -> t.accordIsEnabled).collect(Collectors.toList()); + + @Override + protected Gen<State<Spec>> stateGen() + { + return AccordState::new; + } + + @Override + protected void preCheck(Property.StatefulBuilder builder) + { + // if a failing seed is detected, populate here + // Example: builder.withSeed(42L); + } + + private static Spec createSchemaSpec(RandomSource rs, Cluster cluster) + { + TransactionalMode mode = rs.pick(TRANSACTIONAL_MODES); + boolean enableMigration = allowsMigration(mode) && rs.nextBoolean(); + TableMetadata metadata = fromQT(new CassandraGenerators.TableMetadataBuilder() + .withKeyspaceName(KEYSPACE) + .withTableKinds(TableMetadata.Kind.REGULAR) + .withKnownMemtables() + //TODO (coverage): include "fast_path = 'keyspace'" override + .withTransactionalMode(enableMigration ? TransactionalMode.off : mode) + .withoutEmpty() + .build()) + .next(rs); + maybeCreateUDTs(cluster, metadata); + String schemaCQL = metadata.toCqlString(false, false, false); + logger.info("Creating test table:\n{}", schemaCQL); + cluster.schemaChange(schemaCQL); + if (enableMigration) + { + cluster.schemaChange("ALTER TABLE " + metadata + " WITH " + mode.asCqlParam()); + cluster.get(1).nodetoolResult("consensus_admin", "begin-migration", "--target-protocol", "accord", metadata.keyspace, metadata.name).asserts().success(); + } + return new Spec(mode, enableMigration, metadata); + } + + private static BiFunction<RandomSource, State<Spec>, Command<State<Spec>, Void, ?>> cqlOperations(Spec spec) + { + Gen<Statement> select = (Gen<Statement>) (Gen<?>) fromQT(new ASTGenerators.SelectGenBuilder(spec.metadata).withLimit1().build()); + Gen<Statement> mutation = (Gen<Statement>) (Gen<?>) fromQT(new ASTGenerators.MutationGenBuilder(spec.metadata).withoutTimestamp().build()); + Gen<Statement> txn = (Gen<Statement>) (Gen<?>) fromQT(new ASTGenerators.TxnGenBuilder(spec.metadata).build()); + Map<Gen<Statement>, Integer> operations = new LinkedHashMap<>(); + operations.put(select, 1); + operations.put(mutation, 1); + operations.put(txn, 1); + Gen<Statement> statementGen = Gens.oneOf(operations); + return (rs, state) -> cqlOperation(rs, state, statementGen); + } + + private static Command<State<Spec>, Void, ?> cqlOperation(RandomSource rs, State<Spec> state, Gen<Statement> statementGen) + { + Statement stmt = statementGen.next(rs); + String cql; + //TODO (usability): are there any transaction_modes that actually need simple mutations/select to be wrapped in a BEGIN TRANSACTION? If not then this logica can be simplified Review Comment: writes are now supposed to be transparent but reads are not... I do think this is also `transaction_mode` explicit... since we are single threaded it would be good to delegate but still need the read side -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]

