dcapwell commented on code in PR #3785:
URL: https://github.com/apache/cassandra/pull/3785#discussion_r1929398703


##########
test/distributed/org/apache/cassandra/distributed/test/cql3/SingleNodeTableWalkTest.java:
##########
@@ -0,0 +1,540 @@
+/*
+ * 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.distributed.test.cql3;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.EnumSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.NavigableSet;
+import java.util.Optional;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Sets;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import accord.utils.Gen;
+import accord.utils.Gens;
+import accord.utils.Property;
+import accord.utils.RandomSource;
+import org.apache.cassandra.cql3.ast.Bind;
+import org.apache.cassandra.cql3.ast.Conditional;
+import org.apache.cassandra.cql3.ast.CreateIndexDDL;
+import org.apache.cassandra.cql3.ast.FunctionCall;
+import org.apache.cassandra.cql3.ast.Mutation;
+import org.apache.cassandra.cql3.ast.ReferenceExpression;
+import org.apache.cassandra.cql3.ast.Select;
+import org.apache.cassandra.cql3.ast.Symbol;
+import org.apache.cassandra.cql3.ast.TableReference;
+import org.apache.cassandra.cql3.ast.Value;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.InetAddressType;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.dht.Murmur3Partitioner;
+import org.apache.cassandra.distributed.Cluster;
+import org.apache.cassandra.distributed.shared.ClusterUtils;
+import org.apache.cassandra.harry.model.BytesPartitionState;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.service.consensus.TransactionalMode;
+import org.apache.cassandra.utils.ASTGenerators;
+import org.apache.cassandra.utils.AbstractTypeGenerators;
+import org.apache.cassandra.utils.AbstractTypeGenerators.TypeGenBuilder;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.CassandraGenerators.TableMetadataBuilder;
+import org.apache.cassandra.utils.Generators;
+import org.apache.cassandra.utils.ImmutableUniqueList;
+import org.quicktheories.generators.SourceDSL;
+
+import static accord.utils.Property.commands;
+import static accord.utils.Property.stateful;
+import static org.apache.cassandra.utils.ASTGenerators.safeColumns;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getTypeSupport;
+import static org.apache.cassandra.utils.Generators.toGen;
+
+public class SingleNodeTableWalkTest extends StatefulASTBase
+{
+    private static final Logger logger = 
LoggerFactory.getLogger(SingleNodeTableWalkTest.class);
+
+    @Nullable
+    private final TransactionalMode transactionalMode;
+
+    public SingleNodeTableWalkTest()
+    {
+        this(null);
+    }
+
+    protected SingleNodeTableWalkTest(@Nullable TransactionalMode 
transactionalMode)
+    {
+        this.transactionalMode = transactionalMode;
+    }
+
+    protected void preCheck(Property.StatefulBuilder builder)
+    {
+        // if a failing seed is detected, populate here
+        // Example: builder.withSeed(42L);
+        // To show string/blobs
+        // SHOW_REAL_VALUES = true;
+    }
+
+    protected TypeGenBuilder supportedTypes()
+    {
+        return 
AbstractTypeGenerators.withoutUnsafeEquality(AbstractTypeGenerators.builder()
+                                                                               
   .withTypeKinds(AbstractTypeGenerators.TypeKind.PRIMITIVE));
+    }
+
+    protected List<CreateIndexDDL.Indexer> supportedIndexers()
+    {
+        // since legacy is async it's not clear how the test can wait for the 
background write to complete...
+        return Arrays.asList(CreateIndexDDL.SAI);
+    }
+
+    public Property.Command<State, Void, ?> insert(RandomSource rs, State 
state)
+    {
+        //TODO (maintaince): can this become reusable?  its the same as token 
conflict test...
+        Mutation mutation = state.mutationGen.next(rs);
+        return state.command(rs, mutation);
+    }
+
+    public Property.Command<State, Void, ?> selectExisting(RandomSource rs, 
State state)
+    {
+        NavigableSet<BytesPartitionState.Ref> keys = 
state.model.partitionKeys();
+        BytesPartitionState.Ref ref = rs.pickOrderedSet(keys);
+        Clustering<ByteBuffer> key = ref.key;
+
+        Select.Builder builder = Select.builder().table(state.metadata);
+        ImmutableUniqueList<Symbol> pks = state.model.factory.pkPositions;
+        ImmutableUniqueList<Symbol> cks = state.model.factory.ckPositions;
+        for (Symbol pk : pks)
+            builder.value(pk, key.bufferAt(pks.indexOf(pk)));
+
+        boolean wholePartition = cks.isEmpty() ? true : rs.nextBoolean();
+        if (!wholePartition)
+        {
+            // find a row to select
+            BytesPartitionState partition = state.model.get(ref);
+            if (partition.isEmpty())
+            {
+                wholePartition = true;
+            }
+            else
+            {
+                NavigableSet<Clustering<ByteBuffer>> clusteringKeys = 
partition.clusteringKeys();
+                Clustering<ByteBuffer> clusteringKey = 
rs.pickOrderedSet(clusteringKeys);
+                for (Symbol ck : cks)
+                    builder.value(ck, clusteringKey.bufferAt(cks.indexOf(ck)));
+            }
+        }
+        Select select = builder.build();
+        return state.command(rs, select, (wholePartition ? "Whole Partition" : 
"Single Row"));
+    }
+
+    public Property.Command<State, Void, ?> selectToken(RandomSource rs, State 
state)
+    {
+        NavigableSet<BytesPartitionState.Ref> keys = 
state.model.partitionKeys();
+        BytesPartitionState.Ref ref = rs.pickOrderedSet(keys);
+
+        Select.Builder builder = Select.builder().table(state.metadata);
+        builder.where(FunctionCall.tokenByColumns(new 
ArrayList<>(state.model.factory.pkPositions)),
+                      Conditional.Where.Inequality.EQUAL,
+                      token(state, ref));
+
+        Select select = builder.build();
+        return state.command(rs, select, "by token");
+    }
+
+    public Property.Command<State, Void, ?> selectTokenRange(RandomSource rs, 
State state)
+    {
+        NavigableSet<BytesPartitionState.Ref> keys = 
state.model.partitionKeys();
+        BytesPartitionState.Ref start, end;
+        switch (keys.size())
+        {
+            case 1:
+                start = end = Iterables.get(keys, 0);
+                break;
+            case 2:
+                start = Iterables.get(keys, 0);
+                end = Iterables.get(keys, 1);
+                break;
+            case 0:
+                throw new IllegalArgumentException("Unable to select token 
ranges when no partitions exist");
+            default:
+            {
+                int si = rs.nextInt(0, keys.size() - 1);
+                int ei = rs.nextInt(si + 1, keys.size());
+                start = Iterables.get(keys, si);
+                end = Iterables.get(keys, ei);
+            }
+            break;
+        }
+        Select.Builder builder = Select.builder().table(state.metadata);
+        FunctionCall pkToken = FunctionCall.tokenByColumns(new 
ArrayList<>(state.model.factory.pkPositions));
+        boolean startInclusive = rs.nextBoolean();
+        boolean endInclusive = rs.nextBoolean();
+        if (startInclusive && endInclusive && rs.nextBoolean())
+        {
+            // between
+            builder.between(pkToken, token(state, start), token(state, end));
+        }
+        else
+        {
+            builder.where(pkToken,
+                          startInclusive ? 
Conditional.Where.Inequality.GREATER_THAN_EQ : 
Conditional.Where.Inequality.GREATER_THAN,
+                          token(state, start));
+            builder.where(pkToken,
+                          endInclusive ? 
Conditional.Where.Inequality.LESS_THAN_EQ : 
Conditional.Where.Inequality.LESS_THAN,
+                          token(state, end));
+        }
+        Select select = builder.build();
+        return state.command(rs, select, "by token range");
+    }
+
+    public Property.Command<State, Void, ?> nonPartitionQuery(RandomSource rs, 
State state)
+    {
+        Symbol symbol = selectColumn(rs, state);
+        TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe = 
state.model.index(symbol);
+        // we need to index 'null' so LT works, but we can not directly query 
it... so filter out when selecting values
+        NavigableSet<ByteBuffer> allowed = 
Sets.filter(universe.navigableKeySet(), b -> 
!ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b));
+        if (allowed.isEmpty())
+            return Property.ignoreCommand();
+        ByteBuffer value = rs.pickOrderedSet(allowed);
+        Select.Builder builder = Select.builder().table(state.metadata);
+
+        EnumSet<CreateIndexDDL.QueryType> supported = 
!state.indexes.containsKey(symbol) ? 
EnumSet.noneOf(CreateIndexDDL.QueryType.class) : 
state.indexes.get(symbol).supportedQueries();
+        if (supported.isEmpty() || 
!supported.contains(CreateIndexDDL.QueryType.Range))
+            builder.allowFiltering();
+
+        // there are known SAI bugs, so need to avoid them to stay stable...
+        if (state.indexes.containsKey(symbol) && 
state.indexes.get(symbol).indexDDL.indexer == CreateIndexDDL.SAI)
+        {
+            if (symbol.type() == InetAddressType.instance
+                && IGNORED_ISSUES.contains(KnownIssue.SAI_INET_MIXED))
+                return eqSearch(rs, state, symbol, value, builder);
+
+            if (symbol.reversed
+                && SAI_REVERSE_OF_FIXED_LENGTH_TYPES.contains(symbol.type())
+                && 
IGNORED_ISSUES.contains(KnownIssue.SAI_REVERSE_OF_FIXED_LENGTH))
+                return eqSearch(rs, state, symbol, value, builder);
+        }
+
+        if (rs.nextBoolean())
+            return simpleRangeSearch(rs, state, symbol, value, builder);
+        //TODO (coverage): define search that has a upper and lower bound: a > 
and a < | a beteeen ? and ?
+        return eqSearch(rs, state, symbol, value, builder);
+    }
+
+    public Property.Command<State, Void, ?> multiColumnQuery(RandomSource rs, 
State state)
+    {
+        if (state.metadata.columns().size() == 1)
+            throw new IllegalArgumentException("Unable to do multiple column 
query when there is only a single column");
+        int numColumns = rs.nextInt(1, state.metadata.columns().size()) + 1;
+        List<Symbol> cols = 
Gens.lists(Gens.pick(state.model.factory.selectOrder)).unique().ofSize(numColumns).next(rs);
+
+        Select.Builder builder = 
Select.builder().table(state.metadata).allowFiltering();
+
+        for (Symbol symbol : cols)
+        {
+            TreeMap<ByteBuffer, List<BytesPartitionState.PrimaryKey>> universe 
= state.model.index(symbol);
+            NavigableSet<ByteBuffer> allowed = 
Sets.filter(universe.navigableKeySet(), b -> 
!ByteBufferUtil.EMPTY_BYTE_BUFFER.equals(b));
+            //TODO (now): support
+            if (allowed.isEmpty())
+                return Property.ignoreCommand();
+            ByteBuffer value = rs.pickOrderedSet(allowed);
+            builder.value(symbol, value);
+        }
+
+        Select select = builder.build();
+        String annotate = cols.stream().map(symbol -> {
+            var indexed = state.indexes.get(symbol);
+            return symbol.detailedName() + (indexed == null ? "" : " (indexed 
with " + indexed.indexDDL.indexer.name() + ")");
+        }).collect(Collectors.joining(", "));
+        return state.command(rs, select, annotate);
+    }
+
+    private Property.Command<State, Void, ?> simpleRangeSearch(RandomSource 
rs, State state, Symbol symbol, ByteBuffer value, Select.Builder builder)
+    {
+        // do a simple search, like > or <
+        Conditional.Where.Inequality kind = state.rangeInequalityGen.next(rs);
+        builder.where(symbol, kind, value);
+        Select select = builder.build();
+        var indexed = state.indexes.get(symbol);
+        return state.command(rs, select, symbol.detailedName() + (indexed == 
null ? "" : ", indexed with " + indexed.indexDDL.indexer.name()));
+    }
+
+    private Property.Command<State, Void, ?> eqSearch(RandomSource rs, State 
state, Symbol symbol, ByteBuffer value, Select.Builder builder)
+    {
+        builder.value(symbol, value);
+
+        Select select = builder.build();
+        var indexed = state.indexes.get(symbol);
+        return state.command(rs, select, symbol.detailedName() + (indexed == 
null ? "" : ", indexed with " + indexed.indexDDL.indexer.name()));
+    }
+
+    protected State createState(RandomSource rs, Cluster cluster)
+    {
+        return new State(rs, cluster);
+    }
+
+    protected Cluster createCluster() throws IOException
+    {
+        return createCluster(1, i -> {});
+    }
+
+    @Test
+    public void test() throws IOException
+    {
+        try (Cluster cluster = createCluster())
+        {
+            Property.StatefulBuilder statefulBuilder = 
stateful().withExamples(10).withSteps(400);
+            preCheck(statefulBuilder);
+            statefulBuilder.check(commands(() -> rs -> createState(rs, 
cluster))
+                                                             .add(this::insert)
+                                                             
.addIf(State::hasPartitions, this::selectExisting)
+                                                             
.addAllIf(State::supportTokens, b -> b.add(this::selectToken)
+                                                                               
                    .add(this::selectTokenRange))
+                                                             
.addIf(State::allowNonPartitionQuery, this::nonPartitionQuery)
+                                                             
.addIf(State::hasEnoughMemtable, StatefulASTBase::flushTable)
+                                                             
.addIf(State::hasEnoughSSTables, StatefulASTBase::compactTable)
+                                                             .addIf(s -> 
s.metadata.columns().size() > 1, this::multiColumnQuery)
+                                                             
.destroyState(State::close)
+                                                             
.onSuccess(onSuccess(logger))
+                                                             .build());
+        }
+    }
+
+    private TableMetadata defineTable(RandomSource rs, String ks)
+    {
+        //TODO (correctness): the id isn't correct... this is what we use to 
create the table, so would miss the actual ID
+        // Defaults may also be incorrect, but given this is the same version 
it "shouldn't"
+        //TODO (coverage): partition is defined at the cluster level, so have 
to hard code in this model as the table is changed rather than cluster being 
recreated... this limits coverage
+        var metadata = Generators.toGen(new TableMetadataBuilder()
+                                        
.withTableKinds(TableMetadata.Kind.REGULAR)
+                                        .withKnownMemtables()
+                                        
.withKeyspaceName(ks).withTableName("tbl")
+                                        .withSimpleColumnNames()
+                                        .withDefaultTypeGen(supportedTypes())
+                                        
.withPartitioner(Murmur3Partitioner.instance)
+                                        .build())
+                                 .next(rs);
+        if (transactionalMode != null)
+            metadata = 
metadata.withSwapped(metadata.params.unbuild().transactionalMode(transactionalMode).build());
+        return metadata;
+    }
+
+    private List<CreateIndexDDL.Indexer> allowed(TableMetadata metadata, 
ColumnMetadata col)
+    {
+        return supportedIndexers().stream()
+                                  .filter(i -> i.supported(metadata, col))
+                                  .collect(Collectors.toList());
+    }
+
+    private static FunctionCall token(State state, BytesPartitionState.Ref ref)
+    {
+        List<Value> values = new ArrayList<>(ref.key.size());
+        for (int i = 0; i < ref.key.size(); i++)
+        {
+            ByteBuffer bb = ref.key.bufferAt(i);
+            Symbol type = state.model.factory.pkPositions.get(i);
+            values.add(new Bind(bb, type.type()));
+        }
+        return FunctionCall.tokenByValue(values);
+    }
+
+    private Symbol selectColumn(RandomSource rs, State state)
+    {
+        ColumnKind kind = rs.pickOrderedSet(state.allowedToQuery);
+        Symbol symbol;
+        switch (kind)
+        {
+            case Partition:
+                symbol = rs.pick(state.model.factory.pkPositions);
+                break;
+            case Clustering:
+                symbol = rs.pick(state.model.factory.ckPositions);
+                break;
+            case Static:
+                symbol = rs.pick(state.model.factory.staticPositions);
+                break;
+            case Regular:
+                symbol = rs.pick(state.model.factory.regularPositions);
+                break;
+            default:
+                throw new UnsupportedOperationException(kind.name());
+        }
+        return symbol;
+    }
+
+    private enum ColumnKind { Partition, Clustering, Static, Regular }
+
+    public class State extends BaseState
+    {
+        private final LinkedHashMap<Symbol, IndexedColumn> indexes;
+        private final Gen<Mutation> mutationGen;
+        private final EnumSet<ColumnKind> allowedToQuery = 
EnumSet.noneOf(ColumnKind.class);
+
+        public State(RandomSource rs, Cluster cluster)
+        {
+            super(rs, cluster, defineTable(rs, nextKeyspace()));
+
+            this.indexes = createIndexes(rs, metadata);
+
+            cluster.forEach(i -> i.nodetoolResult("disableautocompaction", 
metadata.keyspace, this.metadata.name).asserts().success());
+
+            List<LinkedHashMap<Symbol, Object>> uniquePartitions;
+            {
+                int unique = rs.nextInt(1, 10);
+                if (metadata.partitionKeyColumns().size() == 1)
+                {
+                    Symbol col = 
Symbol.from(metadata.partitionKeyColumns().get(0));
+                    List<Object> bbs = 
Gens.lists(toGen(getTypeSupport(col.type()).valueGen).map(v -> (Object) v))
+                                           .uniqueBestEffort()
+                                           .ofSize(unique)
+                                           .next(rs);
+                    uniquePartitions = bbs.stream().map(b -> {
+                        LinkedHashMap<Symbol, Object> map = new 
LinkedHashMap<>(1);
+                        map.put(col, b);
+                        return map;
+                    }).collect(Collectors.toList());
+                }
+                else
+                {
+                    List<Symbol> columns = 
metadata.partitionKeyColumns().stream().map(Symbol::from).collect(Collectors.toList());
+                    uniquePartitions = Gens.lists(r2 -> {
+                        LinkedHashMap<Symbol, Object> vs = new 
LinkedHashMap<>();
+                        for (Symbol column : columns)
+                        {
+                            Object value = 
toGen(getTypeSupport(column.type()).valueGen).next(r2);
+                            vs.put(column, value);
+                        }
+                        return vs;
+                    }).uniqueBestEffort().ofSize(unique).next(rs);
+                }
+            }
+
+            this.mutationGen = toGen(new 
ASTGenerators.MutationGenBuilder(metadata)
+                                     .withoutTransaction()
+                                     .withoutTtl()
+                                     .withoutTimestamp()
+                                     
.withPartitions(SourceDSL.arbitrary().pick(uniquePartitions))
+                                     .build());
+
+            if (metadata.partitionKeyColumns().size() > 1)
+                allowedToQuery.add(ColumnKind.Partition);
+            if (!metadata.clusteringColumns().isEmpty())
+                allowedToQuery.add(ColumnKind.Clustering);
+            if (!metadata.staticColumns().isEmpty())
+                allowedToQuery.add(ColumnKind.Static);
+            if (!metadata.regularColumns().isEmpty())
+                allowedToQuery.add(ColumnKind.Regular);
+        }
+
+        private LinkedHashMap<Symbol, IndexedColumn> 
createIndexes(RandomSource rs, TableMetadata metadata)
+        {
+            LinkedHashMap<Symbol, IndexedColumn> indexed = new 
LinkedHashMap<>();
+            // for some test runs, avoid using indexes
+            if (rs.nextBoolean())
+                return indexed;
+            for (ColumnMetadata col : safeColumns(metadata))

Review Comment:
   renamed to `allColumnsInFixedOrder` and added a doc saying why this exists.



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

Reply via email to