adelapena commented on code in PR #2310:
URL: https://github.com/apache/cassandra/pull/2310#discussion_r1232111013


##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -64,6 +99,30 @@
     private static final Gen<Integer> VERY_SMALL_POSITIVE_SIZE_GEN = 
SourceDSL.integers().between(1, 3);
     private static final Gen<Boolean> BOOLEAN_GEN = SourceDSL.booleans().all();
 
+    public static final Map<Class<? extends AbstractType<?>>, String> 
UNSUPPORTED = ImmutableMap.<Class<? extends AbstractType<?>>, String>builder()

Review Comment:
   Nit: we might want to annotate the entire class with 
`@SuppressWarnings({"unchecked", "rawtypes"})`.



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -249,86 +702,481 @@ public static <T> TypeSupport<T> 
getTypeSupport(AbstractType<T> type)
         return getTypeSupport(type, VERY_SMALL_POSITIVE_SIZE_GEN);
     }
 
+    public enum ValueDomain { NULL, EMPTY_BYTES, NORMAL }
+
+    public static <T> TypeSupport<T> getTypeSupportWithNulls(AbstractType<T> 
type, @Nullable Gen<ValueDomain> valueDomainGen)
+    {
+        return getTypeSupport(type, VERY_SMALL_POSITIVE_SIZE_GEN, 
valueDomainGen);
+    }
+
+    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen)
+    {
+        return getTypeSupport(type, sizeGen, null);
+    }
+
     /**
      * For a type, create generators for data that matches that type
      */
-    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen)
+    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen, @Nullable Gen<ValueDomain> valueDomainGen)
     {
+        Objects.requireNonNull(sizeGen, "sizeGen");
         // this doesn't affect the data, only sort order, so drop it
-        if (type.isReversed())
-            type = ((ReversedType<T>) type).baseType;
+        type = type.unwrap();
         // cast is safe since type is a constant and was type cast while 
inserting into the map
         @SuppressWarnings("unchecked")
         TypeSupport<T> gen = (TypeSupport<T>) 
PRIMITIVE_TYPE_DATA_GENS.get(type);
+        TypeSupport<T> support;
         if (gen != null)
-            return gen;
+        {
+            support = gen;
+        }
         // might be... complex...
-        if (type instanceof SetType)
+        else if (type instanceof SetType)
         {
             // T = Set<A> so can not use T here
             SetType<Object> setType = (SetType<Object>) type;
-            TypeSupport<?> elementSupport = 
getTypeSupport(setType.getElementsType(), sizeGen);
-            @SuppressWarnings("unchecked")
-            TypeSupport<T> support = (TypeSupport<T>) TypeSupport.of(setType, 
rnd -> {
+            TypeSupport<Object> elementSupport = 
getTypeSupport(setType.getElementsType(), sizeGen, valueDomainGen);
+            Comparator<Object> elComparator = elementSupport.valueComparator;
+            Comparator<List<Object>> setComparator = 
listComparator(elComparator);
+            Comparator<Set<Object>> comparator = (Set<Object> a, Set<Object> 
b) -> {
+                List<Object> as = new ArrayList<>(a);
+                Collections.sort(as, elComparator);
+                List<Object> bs = new ArrayList<>(b);
+                Collections.sort(bs, elComparator);

Review Comment:
   ```suggestion
                   as.sort(elComparator);
                   List<Object> bs = new ArrayList<>(b);
                   bs.sort(elComparator);
   ```



##########
test/unit/org/apache/cassandra/db/marshal/AbstractTypeTest.java:
##########
@@ -0,0 +1,678 @@
+/*
+ * 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.db.marshal;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.lang.reflect.Modifier;
+import java.nio.ByteBuffer;
+import java.security.CodeSource;
+import java.security.ProtectionDomain;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Sets;
+import org.junit.Test;
+
+import org.apache.cassandra.config.CassandraRelevantProperties;
+import org.apache.cassandra.cql3.AssignmentTestable;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.ColumnSpecification;
+import org.apache.cassandra.cql3.Constants;
+import org.apache.cassandra.cql3.Json;
+import org.apache.cassandra.cql3.QueryOptions;
+import org.apache.cassandra.cql3.Term;
+import org.apache.cassandra.cql3.VariableSpecifications;
+import org.apache.cassandra.io.util.DataInputBuffer;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputBuffer;
+import org.apache.cassandra.io.util.File;
+import org.apache.cassandra.schema.CQLTypeParser;
+import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.Types;
+import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.serializers.TypeSerializer;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.AbstractTypeGenerators;
+import org.apache.cassandra.utils.AbstractTypeGenerators.Releaser;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FastByteOperations;
+import org.apache.cassandra.utils.bytecomparable.ByteComparable;
+import org.apache.cassandra.utils.bytecomparable.ByteSource;
+import org.apache.cassandra.utils.bytecomparable.ByteSourceInverse;
+import org.quicktheories.core.Gen;
+import org.reflections.Reflections;
+import org.reflections.scanners.Scanners;
+import org.reflections.util.ConfigurationBuilder;
+
+import static org.apache.cassandra.utils.AbstractTypeGenerators.TypeKind.*;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.TypeSupport.of;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.extractUDTs;
+import static 
org.apache.cassandra.utils.AbstractTypeGenerators.overridePrimitiveTypeSupport;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.typeTree;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+import static org.quicktheories.QuickTheory.qt;
+import static org.quicktheories.generators.SourceDSL.doubles;
+
+public class AbstractTypeTest
+{
+    static
+    {
+        // make sure blob is always the same
+        CassandraRelevantProperties.TEST_BLOB_SHARED_SEED.setInt(42);
+    }
+
+    private static final Reflections reflections = new Reflections(new 
ConfigurationBuilder()
+                                                                   
.forPackage("org.apache.cassandra")
+                                                                   
.setScanners(Scanners.SubTypes)
+                                                                   
.setExpandSuperTypes(true)
+                                                                   
.setParallel(true));
+
+    // TODO
+    // isCompatibleWith/isValueCompatibleWith/isSerializationCompatibleWith,
+    // withUpdatedUserType/expandUserTypes/referencesDuration - types that 
recursive check types
+    // getMaskedValue
+
+    @Test
+    public void empty()
+    {
+        qt().forAll(genBuilder().build()).checkAssert(type -> {
+            if (type.allowsEmpty())
+            {
+                type.validate(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+                // empty container or null is valid; only checks that this 
method doesn't fail
+                type.compose(ByteBufferUtil.EMPTY_BYTE_BUFFER);
+            }
+            else
+            {
+                assertThatThrownBy(() -> 
type.validate(ByteBufferUtil.EMPTY_BYTE_BUFFER)).isInstanceOf(MarshalException.class);
+                assertThatThrownBy(() -> 
type.getSerializer().validate(ByteBufferUtil.EMPTY_BYTE_BUFFER)).isInstanceOf(MarshalException.class);
+                // ByteSerializer returns null
+//                assertThatThrownBy(() -> 
type.compose(ByteBufferUtil.EMPTY_BYTE_BUFFER)).isInstanceOf(MarshalException.class);
+            }
+        });
+    }
+
+    @Test
+    @SuppressWarnings("rawtypes")
+    public void allTypesCovered()
+    {
+        // this test just makes sure that all types are covered and no new 
type is left out
+        Set<Class<? extends AbstractType>> subTypes = 
reflections.getSubTypesOf(AbstractType.class);
+        Set<Class<? extends AbstractType>> coverage = 
AbstractTypeGenerators.knownTypes();
+        StringBuilder sb = new StringBuilder();
+        for (Class<? extends AbstractType> klass : Sets.difference(subTypes, 
coverage))
+        {
+            if (Modifier.isAbstract(klass.getModifiers()))
+                continue;
+            if (isTestType(klass))
+                continue;
+            String name = klass.getCanonicalName();
+            if (name == null)
+                name = klass.getName();
+            sb.append(name).append('\n');
+        }
+        if (sb.length() > 0)
+            throw new AssertionError("Uncovered types:\n" + sb);
+    }
+
+    @SuppressWarnings("rawtypes")
+    private boolean isTestType(Class<? extends AbstractType> klass)
+    {
+        String name = klass.getCanonicalName();
+        if (name == null)
+            name = klass.getName();
+        if (name == null)
+            name = klass.toString();
+        if (name.contains("Test"))
+            return true;
+        ProtectionDomain domain = klass.getProtectionDomain();
+        if (domain == null) return false;
+        CodeSource src = domain.getCodeSource();
+        if (src == null) return false;
+        return "test".equals(new File(src.getLocation().getPath()).name());
+    }
+
+    @Test
+    public void unsafeSharedSerializer()
+    {
+        // For all types, make sure the serializer returned is unique to that 
type,
+        // this is required as some places, such as SetSerializer, cache at 
this level!
+        Map<TypeSerializer<?>, AbstractType<?>> lookup = new HashMap<>();
+        qt().forAll(genBuilder().withMaxDepth(0).build()).checkAssert(t -> {
+            AbstractType<?> old = lookup.put(t.getSerializer(), t);
+            // for frozen types, ignore the fact that the mapping breaks...  
The reason this test exists is that
+            // org.apache.cassandra.db.marshal.AbstractType.comparatorSet 
needs to match the serializer, but when serialziers
+            // break this mapping they may cause the wrong comparator 
(happened in cases like uuid and lexecal uuid; which have different orderings!).
+            // Frozen types (as of this writing) do not change the sort 
ordering, so this simplification is fine...
+            if (old != null && !old.unfreeze().equals(t.unfreeze()))
+                throw new AssertionError(String.format("Different types 
detected that shared the same serializer: %s != %s", old.asCQL3Type(), 
t.asCQL3Type()));
+        });
+    }
+
+    @Test
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void eqHashSafe()
+    {
+        StringBuilder sb = new StringBuilder();
+        outter: for (Class<? extends AbstractType> type : 
reflections.getSubTypesOf(AbstractType.class))
+        {
+            if (Modifier.isAbstract(type.getModifiers()) || isTestType(type) 
|| AbstractTypeGenerators.UNSUPPORTED.containsKey(type))
+                continue;
+            boolean hasEq = false;
+            boolean hasHashCode = false;
+            for (Class<? extends AbstractType> t = type; 
!t.equals(AbstractType.class); t = (Class<? extends AbstractType>) 
t.getSuperclass())
+            {
+                try
+                {
+                    t.getDeclaredMethod("getInstance");
+                    continue outter;
+                }
+                catch (NoSuchMethodException e)
+                {
+                    // ignore
+                }
+                try
+                {
+                    t.getDeclaredField("instance");
+                    continue outter;
+                }
+                catch (NoSuchFieldException e)
+                {
+                    // ignore
+                }
+                try
+                {
+                    t.getDeclaredMethod("equals", Object.class);
+                    hasEq = true;
+                }
+                catch (NoSuchMethodException e)
+                {
+                    // ignore
+                }
+                try
+                {
+                    t.getDeclaredMethod("hashCode");
+                    hasHashCode = true;
+                }
+                catch (NoSuchMethodException e)
+                {
+                    // ignore
+                }
+                if (hasEq && hasHashCode)
+                    continue outter;
+            }
+            sb.append("AbstractType must be safe for map keys, so must either 
be a singleton or define ");
+            if (!hasEq)
+                sb.append("equals");
+            if (!hasHashCode)
+            {
+                if (!hasEq)
+                    sb.append('/');
+                sb.append("hashCode");
+            }
+            sb.append("; ").append(type).append('\n');
+        }
+        if (sb.length() != 0)
+        {
+            sb.setLength(sb.length() - 1);
+            throw new AssertionError(sb.toString());
+        }
+    }
+
+    @Test
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void comparableBytes()
+    {
+        Gen<AbstractType<?>> gen = genBuilder()
+                                   // decimal "normalizes" the data to 
compare, so primary columns "may" mutate the data, causing missmatches
+                                   // see CASSANDRA-18530
+                                   .withoutPrimitive(DecimalType.instance)
+                                   .withoutTypeKinds(COUNTER)
+                                   .build();
+        qt().withShrinkCycles(0).forAll(examples(1, gen)).checkAssert(example 
-> {
+            AbstractType type = example.type;
+            for (Object value : example.samples)
+            {
+                ByteBuffer bb = type.decompose(value);
+                for (ByteComparable.Version bcv : 
ByteComparable.Version.values())
+                {
+                    // LEGACY, // Encoding used in legacy sstable format; 
forward (value to byte-comparable) translation only
+                    // Legacy is a one-way conversion, so for this test ignore
+                    if (bcv == ByteComparable.Version.LEGACY)
+                        continue;
+                    // Test normal type APIs
+                    ByteSource.Peekable comparable = 
ByteSource.peekable(type.asComparableBytes(bb, bcv));
+                    if (comparable == null)
+                        throw new NullPointerException();
+                    ByteBuffer read;
+                    try
+                    {
+                        read = type.fromComparableBytes(comparable, bcv);
+                    }
+                    catch (Exception | Error e)
+                    {
+                        throw new AssertionError(String.format("Unable to 
parse comparable bytes for type %s and version %s; value %s", 
type.asCQL3Type(), bcv, type.toCQLString(bb)), e);
+                    }
+                    assertBytesEquals(read, bb, 
"fromComparableBytes(asComparableBytes(bb)) != bb; version %s", bcv);
+
+                    // test byte[] api
+                    byte[] bytes = 
ByteSourceInverse.readBytes(type.asComparableBytes(bb, bcv));
+                    
assertBytesEquals(type.fromComparableBytes(ByteSource.peekable(ByteSource.fixedLength(bytes)),
 bcv), bb, "fromOrderedBytes(toOrderedBytes(bb)) != bb");
+                }
+            }
+        });
+    }
+
+    @Test
+    public void knowThySelf()
+    {
+        
qt().withShrinkCycles(0).forAll(AbstractTypeGenerators.typeGen()).checkAssert(type
 -> {
+            assertThat(type.testAssignment(null, new ColumnSpecification(null, 
null, null, type))).isEqualTo(AssignmentTestable.TestResult.EXACT_MATCH);
+            
assertThat(type.testAssignment(type)).isEqualTo(AssignmentTestable.TestResult.EXACT_MATCH);
+        });
+    }
+
+    @Test
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public void json()
+    {
+        // Double type is special as NaN and Infinite are treated differently 
than other code paths as they are convered to null!
+        // This is fine in most cases, but when found in a collection, this is 
not allowed and can cause flakeyness
+        try (Releaser ignore = 
overridePrimitiveTypeSupport(DoubleType.instance,
+                                                            
of(DoubleType.instance, doubles().between(Double.MIN_VALUE, Double.MAX_VALUE))))
+        {
+            Gen<AbstractType<?>> typeGen = genBuilder()
+                                           
.withDefaultSetKey(AbstractTypeGenerators.withoutUnsafeEquality().withoutTypeKinds(COMPOSITE,
 DYNAMIC_COMPOSITE, COUNTER))
+                                           // toCQLLiteral is lossy, which 
causes deserialization to produce different bytes
+                                           
.withoutPrimitive(DecimalType.instance)
+                                           // does not support toJSONString
+                                           .withoutTypeKinds(COMPOSITE, 
DYNAMIC_COMPOSITE, COUNTER)
+                                           .build();
+            qt().withShrinkCycles(0).forAll(examples(1, 
typeGen)).checkAssert(es -> {
+                AbstractType type = es.type;
+                for (Object example : es.samples)
+                {
+                    ByteBuffer bb = type.decompose(example);
+                    String json = type.toJSONString(bb, 
ProtocolVersion.CURRENT);
+                    ColumnMetadata column = fake(type);
+                    String cqlJson = "{\"" + column.name + "\": " + json + "}";
+                    try
+                    {
+                        Json.Prepared prepared = new 
Json.Literal(cqlJson).prepareAndCollectMarkers(null, 
Collections.singletonList(column), VariableSpecifications.empty());
+                        Term.Raw literal = 
prepared.getRawTermForColumn(column, false);
+                        
assertThat(literal).isNotEqualTo(Constants.NULL_LITERAL);
+                        Term term = literal.prepare(column.ksName, column);
+                        ByteBuffer read = 
term.bindAndGet(QueryOptions.DEFAULT);
+                        assertBytesEquals(read, bb, 
"fromJSONString(toJSONString(bb)) != bb");
+                    }
+                    catch (Exception e)
+                    {
+                        throw new AssertionError("Unable to parse JSON for " + 
json + "; type " + type.asCQL3Type(), e);
+                    }
+                }
+            });
+        }
+    }
+
+    @Test
+    @SuppressWarnings({"rawtypes", "unchecked"})

Review Comment:
   Nit: `@SuppressWarnings("rawtypes")` is enough



##########
test/unit/org/apache/cassandra/tools/ToolRunner.java:
##########
@@ -389,6 +389,19 @@ public Exception getException()
         {
             return e;
         }
+
+        /**
+         * Checks if the stdErr is empty after removing any potential JVM env 
info output and other noise
+         *
+         * Some JVM configs may output env info on stdErr. We need to remove 
those to see what was the tool's actual
+         * stdErr
+         *
+         * @return The ToolRunner instance

Review Comment:
   Nit: it doesn't return anything. Same for the other overload of 
`assertCleanStdErr`, which was already wrong before the changes.



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -72,42 +131,344 @@
               TypeSupport.of(LongType.instance, SourceDSL.longs().all()),
               TypeSupport.of(FloatType.instance, SourceDSL.floats().any()),
               TypeSupport.of(DoubleType.instance, SourceDSL.doubles().any()),
-              TypeSupport.of(BytesType.instance, Generators.bytes(1, 1024)),
+              TypeSupport.of(BytesType.instance, Generators.bytes(0, 1024), 
FastByteOperations::compareUnsigned), // use the faster version...
               TypeSupport.of(UUIDType.instance, Generators.UUID_RANDOM_GEN),
-              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN), // serialization strips the hostname, 
only keeps the address
-              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(1, 1024)),
-              TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 1024)),
+              TypeSupport.of(TimeUUIDType.instance, 
Generators.UUID_TIME_GEN.map(TimeUUID::fromUuid)),
+              TypeSupport.of(LexicalUUIDType.instance, 
Generators.UUID_RANDOM_GEN.mix(Generators.UUID_TIME_GEN)),
+              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN, (a, b) -> 
FastByteOperations.compareUnsigned(a.getAddress(), b.getAddress())), // 
serialization strips the hostname, only keeps the address
+              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(0, 1024), 
stringComparator(AsciiType.instance)),
+              TypeSupport.of(UTF8Type.instance, Generators.utf8(0, 1024), 
stringComparator(UTF8Type.instance)),
               TypeSupport.of(TimestampType.instance, Generators.DATE_GEN),
+              TypeSupport.of(SimpleDateType.instance, 
SourceDSL.integers().between(0, Integer.MAX_VALUE)), // can't use time gen as 
this is an int, and in Milliseconds... so overflows...
+              TypeSupport.of(TimeType.instance, SourceDSL.longs().between(0, 
24L * 60L * 60L * 1_000_000_000L - 1L)),
               // null is desired here as #decompose will call 
org.apache.cassandra.serializers.EmptySerializer.serialize which ignores the 
input and returns empty bytes
-              TypeSupport.of(EmptyType.instance, rnd -> null)
-              //TODO add the following
-              // IntegerType.instance,
-              // DecimalType.instance,
-              // TimeUUIDType.instance,
-              // LexicalUUIDType.instance,
-              // SimpleDateType.instance,
-              // TimeType.instance,
-              // DurationType.instance,
+              TypeSupport.of(EmptyType.instance, rnd -> null, (a, b) -> 0),
+              TypeSupport.of(DurationType.instance, 
CassandraGenerators.duration(), Comparator.comparingInt(Duration::getMonths)
+                                                                               
               .thenComparingInt(Duration::getDays)
+                                                                               
               .thenComparingLong(Duration::getNanoseconds)),
+              TypeSupport.of(IntegerType.instance, Generators.bigInt()),
+              TypeSupport.of(DecimalType.instance, Generators.bigDecimal())
     ).collect(Collectors.toMap(t -> t.type, t -> t));
     // NOTE not supporting reversed as CQL doesn't allow nested reversed types
     // when generating part of the clustering key, it would be good to allow 
reversed types as the top level
-    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = 
SourceDSL.arbitrary().pick(new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()));
+    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN;
+    static
+    {
+        ArrayList<AbstractType<?>> types = new 
ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet());
+        Collections.sort(types, Comparator.comparing(a -> 
a.getClass().getName()));
+        PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(types);
+    }
+
+    private static final Set<Class<? extends AbstractType>> 
NON_PRIMITIVE_TYPES = ImmutableSet.<Class<? extends AbstractType>>builder()
+                                                                               
               .add(SetType.class)
+                                                                               
               .add(ListType.class)
+                                                                               
               .add(MapType.class)
+                                                                               
               .add(TupleType.class)
+                                                                               
               .add(UserType.class)
+                                                                               
               .add(VectorType.class)
+                                                                               
               .add(CompositeType.class)
+                                                                               
               .add(DynamicCompositeType.class)
+                                                                               
               .add(CounterColumnType.class)
+                                                                               
               .build();
 
     private AbstractTypeGenerators()
     {
 
     }
 
     public enum TypeKind
-    {PRIMITIVE, SET, LIST, MAP, TUPLE, UDT}
+    {
+        PRIMITIVE,
+        SET, LIST, MAP,
+        TUPLE, UDT,
+        VECTOR,
+        COMPOSITE, DYNAMIC_COMPOSITE,
+        COUNTER
+    }
 
     private static final Gen<TypeKind> TYPE_KIND_GEN = 
SourceDSL.arbitrary().enumValuesWithNoOrder(TypeKind.class);
 
+    public static Set<Class<? extends AbstractType>> knownTypes()
+    {
+        Set<Class<? extends AbstractType>> types = 
PRIMITIVE_TYPE_DATA_GENS.keySet().stream().map(a -> 
a.getClass()).collect(Collectors.toSet());
+        types.addAll(NON_PRIMITIVE_TYPES);
+        types.addAll(UNSUPPORTED.keySet());
+        return types;
+    }
+
     public static Gen<AbstractType<?>> primitiveTypeGen()
     {
         return PRIMITIVE_TYPE_GEN;
     }
 
+    public static Set<Class<? extends AbstractType<?>>> UNSAFE_EQUALITY = 
ImmutableSet.of(EmptyType.class,
+                                                                               
           DurationType.class,
+                                                                               
           DecimalType.class,
+                                                                               
           CounterColumnType.class);
+
+    public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> 
type, TypeSupport<T> support)
+    {
+        if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(type))
+            throw new IllegalArgumentException("Type " + type.asCQL3Type() + " 
is not a primitive");
+        TypeSupport<?> original = PRIMITIVE_TYPE_DATA_GENS.get(type);
+        PRIMITIVE_TYPE_DATA_GENS.put(type, support);
+        return () -> PRIMITIVE_TYPE_DATA_GENS.put(type, original);
+    }
+
+    public static TypeGenBuilder withoutUnsafeEquality()
+    {
+        // make sure to keep UNSAFE_EQUALITY in-sync
+        return AbstractTypeGenerators.builder()
+                                     .withoutEmpty()
+                                     .withoutPrimitive(DurationType.instance)
+                                     // decimal "normalizes" the data to 
compare, so primary columns "may" mutate the data, causing missmatches
+                                     // see CASSANDRA-18530
+                                     .withoutPrimitive(DecimalType.instance)
+                                     // counters are only for top level
+                                     .withoutTypeKinds(TypeKind.COUNTER);
+    }
+
+    public interface Releaser extends AutoCloseable
+    {
+        @Override
+        void close();
+    }
+
+    public static class TypeGenBuilder
+    {
+        private int maxDepth = 3;
+        private EnumSet<TypeKind> kinds;
+        private Gen<TypeKind> typeKindGen;
+        private Gen<Integer> defaultSizeGen = VERY_SMALL_POSITIVE_SIZE_GEN;
+        private Gen<Integer> vectorSizeGen, tupleSizeGen, udtSizeGen, 
compositeSizeGen;
+        private Gen<AbstractType<?>> primitiveGen = PRIMITIVE_TYPE_GEN, 
compositeElementGen;
+        private Gen<String> userTypeKeyspaceGen = IDENTIFIER_GEN;
+        private Function<Integer, Gen<AbstractType<?>>> defaultSetKeyFunc;
+
+        public TypeGenBuilder()
+        {
+        }
+
+        public TypeGenBuilder(TypeGenBuilder other)
+        {
+            maxDepth = other.maxDepth;
+            kinds = other.kinds == null ? null : EnumSet.copyOf(other.kinds);
+            typeKindGen = other.typeKindGen;
+            defaultSizeGen = other.defaultSizeGen;
+            vectorSizeGen = other.vectorSizeGen;
+            tupleSizeGen = other.tupleSizeGen;
+            udtSizeGen = other.udtSizeGen;
+            primitiveGen = other.primitiveGen;
+            userTypeKeyspaceGen = other.userTypeKeyspaceGen;
+            defaultSetKeyFunc = other.defaultSetKeyFunc;
+        }
+
+        public TypeGenBuilder withCompositeElementGen(Gen<AbstractType<?>> gen)
+        {
+            this.compositeElementGen = gen;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(Function<Integer, 
Gen<AbstractType<?>>> mapKeyFunc)
+        {
+            this.defaultSetKeyFunc = mapKeyFunc;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(TypeGenBuilder builder)
+        {
+            this.defaultSetKeyFunc = maxDepth -> 
builder.buildRecursive(maxDepth);

Review Comment:
   ```suggestion
               this.defaultSetKeyFunc = builder::buildRecursive;
   ```



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -72,42 +131,344 @@
               TypeSupport.of(LongType.instance, SourceDSL.longs().all()),
               TypeSupport.of(FloatType.instance, SourceDSL.floats().any()),
               TypeSupport.of(DoubleType.instance, SourceDSL.doubles().any()),
-              TypeSupport.of(BytesType.instance, Generators.bytes(1, 1024)),
+              TypeSupport.of(BytesType.instance, Generators.bytes(0, 1024), 
FastByteOperations::compareUnsigned), // use the faster version...
               TypeSupport.of(UUIDType.instance, Generators.UUID_RANDOM_GEN),
-              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN), // serialization strips the hostname, 
only keeps the address
-              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(1, 1024)),
-              TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 1024)),
+              TypeSupport.of(TimeUUIDType.instance, 
Generators.UUID_TIME_GEN.map(TimeUUID::fromUuid)),
+              TypeSupport.of(LexicalUUIDType.instance, 
Generators.UUID_RANDOM_GEN.mix(Generators.UUID_TIME_GEN)),
+              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN, (a, b) -> 
FastByteOperations.compareUnsigned(a.getAddress(), b.getAddress())), // 
serialization strips the hostname, only keeps the address
+              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(0, 1024), 
stringComparator(AsciiType.instance)),
+              TypeSupport.of(UTF8Type.instance, Generators.utf8(0, 1024), 
stringComparator(UTF8Type.instance)),
               TypeSupport.of(TimestampType.instance, Generators.DATE_GEN),
+              TypeSupport.of(SimpleDateType.instance, 
SourceDSL.integers().between(0, Integer.MAX_VALUE)), // can't use time gen as 
this is an int, and in Milliseconds... so overflows...
+              TypeSupport.of(TimeType.instance, SourceDSL.longs().between(0, 
24L * 60L * 60L * 1_000_000_000L - 1L)),
               // null is desired here as #decompose will call 
org.apache.cassandra.serializers.EmptySerializer.serialize which ignores the 
input and returns empty bytes
-              TypeSupport.of(EmptyType.instance, rnd -> null)
-              //TODO add the following
-              // IntegerType.instance,
-              // DecimalType.instance,
-              // TimeUUIDType.instance,
-              // LexicalUUIDType.instance,
-              // SimpleDateType.instance,
-              // TimeType.instance,
-              // DurationType.instance,
+              TypeSupport.of(EmptyType.instance, rnd -> null, (a, b) -> 0),
+              TypeSupport.of(DurationType.instance, 
CassandraGenerators.duration(), Comparator.comparingInt(Duration::getMonths)
+                                                                               
               .thenComparingInt(Duration::getDays)
+                                                                               
               .thenComparingLong(Duration::getNanoseconds)),
+              TypeSupport.of(IntegerType.instance, Generators.bigInt()),
+              TypeSupport.of(DecimalType.instance, Generators.bigDecimal())
     ).collect(Collectors.toMap(t -> t.type, t -> t));
     // NOTE not supporting reversed as CQL doesn't allow nested reversed types
     // when generating part of the clustering key, it would be good to allow 
reversed types as the top level
-    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = 
SourceDSL.arbitrary().pick(new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()));
+    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN;
+    static
+    {
+        ArrayList<AbstractType<?>> types = new 
ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet());
+        Collections.sort(types, Comparator.comparing(a -> 
a.getClass().getName()));
+        PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(types);
+    }
+
+    private static final Set<Class<? extends AbstractType>> 
NON_PRIMITIVE_TYPES = ImmutableSet.<Class<? extends AbstractType>>builder()
+                                                                               
               .add(SetType.class)
+                                                                               
               .add(ListType.class)
+                                                                               
               .add(MapType.class)
+                                                                               
               .add(TupleType.class)
+                                                                               
               .add(UserType.class)
+                                                                               
               .add(VectorType.class)
+                                                                               
               .add(CompositeType.class)
+                                                                               
               .add(DynamicCompositeType.class)
+                                                                               
               .add(CounterColumnType.class)
+                                                                               
               .build();
 
     private AbstractTypeGenerators()
     {
 
     }
 
     public enum TypeKind
-    {PRIMITIVE, SET, LIST, MAP, TUPLE, UDT}
+    {
+        PRIMITIVE,
+        SET, LIST, MAP,
+        TUPLE, UDT,
+        VECTOR,
+        COMPOSITE, DYNAMIC_COMPOSITE,
+        COUNTER
+    }
 
     private static final Gen<TypeKind> TYPE_KIND_GEN = 
SourceDSL.arbitrary().enumValuesWithNoOrder(TypeKind.class);
 
+    public static Set<Class<? extends AbstractType>> knownTypes()
+    {
+        Set<Class<? extends AbstractType>> types = 
PRIMITIVE_TYPE_DATA_GENS.keySet().stream().map(a -> 
a.getClass()).collect(Collectors.toSet());
+        types.addAll(NON_PRIMITIVE_TYPES);
+        types.addAll(UNSUPPORTED.keySet());
+        return types;
+    }
+
     public static Gen<AbstractType<?>> primitiveTypeGen()
     {
         return PRIMITIVE_TYPE_GEN;
     }
 
+    public static Set<Class<? extends AbstractType<?>>> UNSAFE_EQUALITY = 
ImmutableSet.of(EmptyType.class,
+                                                                               
           DurationType.class,
+                                                                               
           DecimalType.class,
+                                                                               
           CounterColumnType.class);
+
+    public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> 
type, TypeSupport<T> support)
+    {
+        if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(type))

Review Comment:
   ```suggestion
           if (!PRIMITIVE_TYPE_DATA_GENS.containsKey(type))
   ```



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -72,42 +131,344 @@
               TypeSupport.of(LongType.instance, SourceDSL.longs().all()),
               TypeSupport.of(FloatType.instance, SourceDSL.floats().any()),
               TypeSupport.of(DoubleType.instance, SourceDSL.doubles().any()),
-              TypeSupport.of(BytesType.instance, Generators.bytes(1, 1024)),
+              TypeSupport.of(BytesType.instance, Generators.bytes(0, 1024), 
FastByteOperations::compareUnsigned), // use the faster version...
               TypeSupport.of(UUIDType.instance, Generators.UUID_RANDOM_GEN),
-              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN), // serialization strips the hostname, 
only keeps the address
-              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(1, 1024)),
-              TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 1024)),
+              TypeSupport.of(TimeUUIDType.instance, 
Generators.UUID_TIME_GEN.map(TimeUUID::fromUuid)),
+              TypeSupport.of(LexicalUUIDType.instance, 
Generators.UUID_RANDOM_GEN.mix(Generators.UUID_TIME_GEN)),
+              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN, (a, b) -> 
FastByteOperations.compareUnsigned(a.getAddress(), b.getAddress())), // 
serialization strips the hostname, only keeps the address
+              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(0, 1024), 
stringComparator(AsciiType.instance)),
+              TypeSupport.of(UTF8Type.instance, Generators.utf8(0, 1024), 
stringComparator(UTF8Type.instance)),
               TypeSupport.of(TimestampType.instance, Generators.DATE_GEN),
+              TypeSupport.of(SimpleDateType.instance, 
SourceDSL.integers().between(0, Integer.MAX_VALUE)), // can't use time gen as 
this is an int, and in Milliseconds... so overflows...
+              TypeSupport.of(TimeType.instance, SourceDSL.longs().between(0, 
24L * 60L * 60L * 1_000_000_000L - 1L)),
               // null is desired here as #decompose will call 
org.apache.cassandra.serializers.EmptySerializer.serialize which ignores the 
input and returns empty bytes
-              TypeSupport.of(EmptyType.instance, rnd -> null)
-              //TODO add the following
-              // IntegerType.instance,
-              // DecimalType.instance,
-              // TimeUUIDType.instance,
-              // LexicalUUIDType.instance,
-              // SimpleDateType.instance,
-              // TimeType.instance,
-              // DurationType.instance,
+              TypeSupport.of(EmptyType.instance, rnd -> null, (a, b) -> 0),
+              TypeSupport.of(DurationType.instance, 
CassandraGenerators.duration(), Comparator.comparingInt(Duration::getMonths)
+                                                                               
               .thenComparingInt(Duration::getDays)
+                                                                               
               .thenComparingLong(Duration::getNanoseconds)),
+              TypeSupport.of(IntegerType.instance, Generators.bigInt()),
+              TypeSupport.of(DecimalType.instance, Generators.bigDecimal())
     ).collect(Collectors.toMap(t -> t.type, t -> t));
     // NOTE not supporting reversed as CQL doesn't allow nested reversed types
     // when generating part of the clustering key, it would be good to allow 
reversed types as the top level
-    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = 
SourceDSL.arbitrary().pick(new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()));
+    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN;
+    static
+    {
+        ArrayList<AbstractType<?>> types = new 
ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet());
+        Collections.sort(types, Comparator.comparing(a -> 
a.getClass().getName()));

Review Comment:
   ```suggestion
           types.sort(Comparator.comparing(a -> a.getClass().getName()));
   ```



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -249,86 +702,481 @@ public static <T> TypeSupport<T> 
getTypeSupport(AbstractType<T> type)
         return getTypeSupport(type, VERY_SMALL_POSITIVE_SIZE_GEN);
     }
 
+    public enum ValueDomain { NULL, EMPTY_BYTES, NORMAL }
+
+    public static <T> TypeSupport<T> getTypeSupportWithNulls(AbstractType<T> 
type, @Nullable Gen<ValueDomain> valueDomainGen)
+    {
+        return getTypeSupport(type, VERY_SMALL_POSITIVE_SIZE_GEN, 
valueDomainGen);
+    }
+
+    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen)
+    {
+        return getTypeSupport(type, sizeGen, null);
+    }
+
     /**
      * For a type, create generators for data that matches that type
      */
-    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen)
+    public static <T> TypeSupport<T> getTypeSupport(AbstractType<T> type, 
Gen<Integer> sizeGen, @Nullable Gen<ValueDomain> valueDomainGen)
     {
+        Objects.requireNonNull(sizeGen, "sizeGen");
         // this doesn't affect the data, only sort order, so drop it
-        if (type.isReversed())
-            type = ((ReversedType<T>) type).baseType;
+        type = type.unwrap();
         // cast is safe since type is a constant and was type cast while 
inserting into the map
         @SuppressWarnings("unchecked")
         TypeSupport<T> gen = (TypeSupport<T>) 
PRIMITIVE_TYPE_DATA_GENS.get(type);
+        TypeSupport<T> support;
         if (gen != null)
-            return gen;
+        {
+            support = gen;
+        }
         // might be... complex...
-        if (type instanceof SetType)
+        else if (type instanceof SetType)
         {
             // T = Set<A> so can not use T here
             SetType<Object> setType = (SetType<Object>) type;
-            TypeSupport<?> elementSupport = 
getTypeSupport(setType.getElementsType(), sizeGen);
-            @SuppressWarnings("unchecked")
-            TypeSupport<T> support = (TypeSupport<T>) TypeSupport.of(setType, 
rnd -> {
+            TypeSupport<Object> elementSupport = 
getTypeSupport(setType.getElementsType(), sizeGen, valueDomainGen);
+            Comparator<Object> elComparator = elementSupport.valueComparator;
+            Comparator<List<Object>> setComparator = 
listComparator(elComparator);
+            Comparator<Set<Object>> comparator = (Set<Object> a, Set<Object> 
b) -> {
+                List<Object> as = new ArrayList<>(a);
+                Collections.sort(as, elComparator);
+                List<Object> bs = new ArrayList<>(b);
+                Collections.sort(bs, elComparator);
+                return setComparator.compare(as, bs);
+            };
+            support = (TypeSupport<T>) TypeSupport.of(setType, rnd -> {
                 int size = sizeGen.generate(rnd);
+                size = normalizeSizeFromType(elementSupport, size);
                 HashSet<Object> set = Sets.newHashSetWithExpectedSize(size);
                 for (int i = 0; i < size; i++)
-                    set.add(elementSupport.valueGen.generate(rnd));
+                {
+                    Object generate = elementSupport.valueGen.generate(rnd);
+                    for (int attempts = 0; set.contains(generate); attempts++)
+                    {
+                        if (attempts == 42)
+                            throw new AssertionError(String.format("Unable to 
get unique element for type %s with the size %d", 
typeTree(elementSupport.type), size));
+                        rnd = JavaRandom.wrap(rnd);
+                        generate = elementSupport.valueGen.generate(rnd);
+                    }
+
+                    set.add(generate);
+                }
                 return set;
-            });
-            return support;
+            }, comparator);
         }
         else if (type instanceof ListType)
         {
             // T = List<A> so can not use T here
             ListType<Object> listType = (ListType<Object>) type;
-            TypeSupport<?> elementSupport = 
getTypeSupport(listType.getElementsType(), sizeGen);
-            @SuppressWarnings("unchecked")
-            TypeSupport<T> support = (TypeSupport<T>) TypeSupport.of(listType, 
rnd -> {
+            TypeSupport<Object> elementSupport = 
getTypeSupport(listType.getElementsType(), sizeGen, valueDomainGen);
+            support = (TypeSupport<T>) TypeSupport.of(listType, rnd -> {
                 int size = sizeGen.generate(rnd);
                 List<Object> list = new ArrayList<>(size);
                 for (int i = 0; i < size; i++)
                     list.add(elementSupport.valueGen.generate(rnd));
                 return list;
-            });
-            return support;
+            }, listComparator(elementSupport.valueComparator));
         }
         else if (type instanceof MapType)
         {
             // T = Map<A, B> so can not use T here
             MapType<Object, Object> mapType = (MapType<Object, Object>) type;
-            TypeSupport<?> keySupport = getTypeSupport(mapType.getKeysType(), 
sizeGen);
-            TypeSupport<?> valueSupport = 
getTypeSupport(mapType.getValuesType(), sizeGen);
-            @SuppressWarnings("unchecked")
-            TypeSupport<T> support = (TypeSupport<T>) TypeSupport.of(mapType, 
rnd -> {
+            // do not use valueDomainGen as map doesn't allow null/empty
+            TypeSupport<Object> keySupport = 
getTypeSupport(mapType.getKeysType(), sizeGen, null);
+            Comparator<Object> keyType = keySupport.valueComparator;
+            TypeSupport<Object> valueSupport = 
getTypeSupport(mapType.getValuesType(), sizeGen, null);
+            Comparator<Object> valueType = valueSupport.valueComparator;
+            Comparator<Map<Object, Object>> comparator = (Map<Object, Object> 
a, Map<Object, Object> b) -> {
+                List<Object> ak = new ArrayList<>(a.keySet());
+                Collections.sort(ak, keyType);
+                List<Object> bk = new ArrayList<>(b.keySet());
+                Collections.sort(bk, keyType);

Review Comment:
   ```suggestion
                   ak.sort(keyType);
                   List<Object> bk = new ArrayList<>(b.keySet());
                   bk.sort(keyType);
   ```



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -72,42 +131,344 @@
               TypeSupport.of(LongType.instance, SourceDSL.longs().all()),
               TypeSupport.of(FloatType.instance, SourceDSL.floats().any()),
               TypeSupport.of(DoubleType.instance, SourceDSL.doubles().any()),
-              TypeSupport.of(BytesType.instance, Generators.bytes(1, 1024)),
+              TypeSupport.of(BytesType.instance, Generators.bytes(0, 1024), 
FastByteOperations::compareUnsigned), // use the faster version...
               TypeSupport.of(UUIDType.instance, Generators.UUID_RANDOM_GEN),
-              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN), // serialization strips the hostname, 
only keeps the address
-              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(1, 1024)),
-              TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 1024)),
+              TypeSupport.of(TimeUUIDType.instance, 
Generators.UUID_TIME_GEN.map(TimeUUID::fromUuid)),
+              TypeSupport.of(LexicalUUIDType.instance, 
Generators.UUID_RANDOM_GEN.mix(Generators.UUID_TIME_GEN)),
+              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN, (a, b) -> 
FastByteOperations.compareUnsigned(a.getAddress(), b.getAddress())), // 
serialization strips the hostname, only keeps the address
+              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(0, 1024), 
stringComparator(AsciiType.instance)),
+              TypeSupport.of(UTF8Type.instance, Generators.utf8(0, 1024), 
stringComparator(UTF8Type.instance)),
               TypeSupport.of(TimestampType.instance, Generators.DATE_GEN),
+              TypeSupport.of(SimpleDateType.instance, 
SourceDSL.integers().between(0, Integer.MAX_VALUE)), // can't use time gen as 
this is an int, and in Milliseconds... so overflows...
+              TypeSupport.of(TimeType.instance, SourceDSL.longs().between(0, 
24L * 60L * 60L * 1_000_000_000L - 1L)),
               // null is desired here as #decompose will call 
org.apache.cassandra.serializers.EmptySerializer.serialize which ignores the 
input and returns empty bytes
-              TypeSupport.of(EmptyType.instance, rnd -> null)
-              //TODO add the following
-              // IntegerType.instance,
-              // DecimalType.instance,
-              // TimeUUIDType.instance,
-              // LexicalUUIDType.instance,
-              // SimpleDateType.instance,
-              // TimeType.instance,
-              // DurationType.instance,
+              TypeSupport.of(EmptyType.instance, rnd -> null, (a, b) -> 0),
+              TypeSupport.of(DurationType.instance, 
CassandraGenerators.duration(), Comparator.comparingInt(Duration::getMonths)
+                                                                               
               .thenComparingInt(Duration::getDays)
+                                                                               
               .thenComparingLong(Duration::getNanoseconds)),
+              TypeSupport.of(IntegerType.instance, Generators.bigInt()),
+              TypeSupport.of(DecimalType.instance, Generators.bigDecimal())
     ).collect(Collectors.toMap(t -> t.type, t -> t));
     // NOTE not supporting reversed as CQL doesn't allow nested reversed types
     // when generating part of the clustering key, it would be good to allow 
reversed types as the top level
-    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = 
SourceDSL.arbitrary().pick(new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()));
+    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN;
+    static
+    {
+        ArrayList<AbstractType<?>> types = new 
ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet());
+        Collections.sort(types, Comparator.comparing(a -> 
a.getClass().getName()));
+        PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(types);
+    }
+
+    private static final Set<Class<? extends AbstractType>> 
NON_PRIMITIVE_TYPES = ImmutableSet.<Class<? extends AbstractType>>builder()
+                                                                               
               .add(SetType.class)
+                                                                               
               .add(ListType.class)
+                                                                               
               .add(MapType.class)
+                                                                               
               .add(TupleType.class)
+                                                                               
               .add(UserType.class)
+                                                                               
               .add(VectorType.class)
+                                                                               
               .add(CompositeType.class)
+                                                                               
               .add(DynamicCompositeType.class)
+                                                                               
               .add(CounterColumnType.class)
+                                                                               
               .build();
 
     private AbstractTypeGenerators()
     {
 
     }
 
     public enum TypeKind
-    {PRIMITIVE, SET, LIST, MAP, TUPLE, UDT}
+    {
+        PRIMITIVE,
+        SET, LIST, MAP,
+        TUPLE, UDT,
+        VECTOR,
+        COMPOSITE, DYNAMIC_COMPOSITE,
+        COUNTER
+    }
 
     private static final Gen<TypeKind> TYPE_KIND_GEN = 
SourceDSL.arbitrary().enumValuesWithNoOrder(TypeKind.class);
 
+    public static Set<Class<? extends AbstractType>> knownTypes()
+    {
+        Set<Class<? extends AbstractType>> types = 
PRIMITIVE_TYPE_DATA_GENS.keySet().stream().map(a -> 
a.getClass()).collect(Collectors.toSet());
+        types.addAll(NON_PRIMITIVE_TYPES);
+        types.addAll(UNSUPPORTED.keySet());
+        return types;
+    }
+
     public static Gen<AbstractType<?>> primitiveTypeGen()
     {
         return PRIMITIVE_TYPE_GEN;
     }
 
+    public static Set<Class<? extends AbstractType<?>>> UNSAFE_EQUALITY = 
ImmutableSet.of(EmptyType.class,
+                                                                               
           DurationType.class,
+                                                                               
           DecimalType.class,
+                                                                               
           CounterColumnType.class);
+
+    public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> 
type, TypeSupport<T> support)
+    {
+        if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(type))
+            throw new IllegalArgumentException("Type " + type.asCQL3Type() + " 
is not a primitive");
+        TypeSupport<?> original = PRIMITIVE_TYPE_DATA_GENS.get(type);
+        PRIMITIVE_TYPE_DATA_GENS.put(type, support);
+        return () -> PRIMITIVE_TYPE_DATA_GENS.put(type, original);
+    }
+
+    public static TypeGenBuilder withoutUnsafeEquality()
+    {
+        // make sure to keep UNSAFE_EQUALITY in-sync
+        return AbstractTypeGenerators.builder()
+                                     .withoutEmpty()
+                                     .withoutPrimitive(DurationType.instance)
+                                     // decimal "normalizes" the data to 
compare, so primary columns "may" mutate the data, causing missmatches
+                                     // see CASSANDRA-18530
+                                     .withoutPrimitive(DecimalType.instance)
+                                     // counters are only for top level
+                                     .withoutTypeKinds(TypeKind.COUNTER);
+    }
+
+    public interface Releaser extends AutoCloseable
+    {
+        @Override
+        void close();
+    }
+
+    public static class TypeGenBuilder
+    {
+        private int maxDepth = 3;
+        private EnumSet<TypeKind> kinds;
+        private Gen<TypeKind> typeKindGen;
+        private Gen<Integer> defaultSizeGen = VERY_SMALL_POSITIVE_SIZE_GEN;
+        private Gen<Integer> vectorSizeGen, tupleSizeGen, udtSizeGen, 
compositeSizeGen;
+        private Gen<AbstractType<?>> primitiveGen = PRIMITIVE_TYPE_GEN, 
compositeElementGen;
+        private Gen<String> userTypeKeyspaceGen = IDENTIFIER_GEN;
+        private Function<Integer, Gen<AbstractType<?>>> defaultSetKeyFunc;
+
+        public TypeGenBuilder()
+        {
+        }
+
+        public TypeGenBuilder(TypeGenBuilder other)
+        {
+            maxDepth = other.maxDepth;
+            kinds = other.kinds == null ? null : EnumSet.copyOf(other.kinds);
+            typeKindGen = other.typeKindGen;
+            defaultSizeGen = other.defaultSizeGen;
+            vectorSizeGen = other.vectorSizeGen;
+            tupleSizeGen = other.tupleSizeGen;
+            udtSizeGen = other.udtSizeGen;
+            primitiveGen = other.primitiveGen;
+            userTypeKeyspaceGen = other.userTypeKeyspaceGen;
+            defaultSetKeyFunc = other.defaultSetKeyFunc;
+        }
+
+        public TypeGenBuilder withCompositeElementGen(Gen<AbstractType<?>> gen)
+        {
+            this.compositeElementGen = gen;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(Function<Integer, 
Gen<AbstractType<?>>> mapKeyFunc)
+        {
+            this.defaultSetKeyFunc = mapKeyFunc;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(TypeGenBuilder builder)
+        {
+            this.defaultSetKeyFunc = maxDepth -> 
builder.buildRecursive(maxDepth);
+            return this;
+        }
+
+        public TypeGenBuilder withUserTypeKeyspace(String keyspace)
+        {
+            userTypeKeyspaceGen = SourceDSL.arbitrary().constant(keyspace);
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSizeGen(int size)
+        {
+            return withDefaultSizeGen(SourceDSL.arbitrary().constant(size));
+        }
+
+        public TypeGenBuilder withDefaultSizeGen(Gen<Integer> sizeGen)
+        {
+            this.defaultSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withVectorSizeGen(Gen<Integer> sizeGen)
+        {
+            this.vectorSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withTupleSizeGen(Gen<Integer> sizeGen)
+        {
+            this.tupleSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withUDTSizeGen(Gen<Integer> sizeGen)
+        {
+            this.udtSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withCompositeSizeGen(Gen<Integer> sizeGen)
+        {
+            this.compositeSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withoutEmpty()
+        {
+            return withoutPrimitive(EmptyType.instance);
+        }
+
+        public TypeGenBuilder withoutPrimitive(AbstractType<?> instance)
+        {
+            if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(instance))
+                throw new IllegalArgumentException("Type " + instance + " is 
not a primitive type, or PRIMITIVE_TYPE_DATA_GENS needs to add support");
+            primitiveGen = Generators.filter(primitiveGen, t -> t != instance);
+            return this;
+        }
+
+        public TypeGenBuilder withPrimitives(AbstractType<?> first, 
AbstractType<?>... remaining)
+        {
+            // any previous filters will be ignored...
+            primitiveGen = 
SourceDSL.arbitrary().pick(ArrayUtils.add(remaining, first));
+            return this;
+        }
+
+        public TypeGenBuilder withMaxDepth(int value)
+        {
+            this.maxDepth = value;
+            return this;
+        }
+
+        public TypeGenBuilder withoutTypeKinds(TypeKind... values)
+        {
+            checkTypeKindValues();
+            for (TypeKind kind : values)
+                kinds.remove(kind);
+            return this;
+        }
+
+        public TypeGenBuilder withTypeKinds(TypeKind... values)
+        {
+            checkTypeKindValues();
+            kinds.clear();
+            for (TypeKind k : values)
+                kinds.add(k);
+            return this;
+        }
+
+        private void checkTypeKindValues()
+        {
+            if (typeKindGen != null)
+                throw new IllegalArgumentException("Mixed both generator and 
individaul values for type kind");
+            if (kinds == null)
+                kinds = EnumSet.allOf(TypeKind.class);
+        }
+
+        public TypeGenBuilder withTypeKinds(Gen<TypeKind> typeKindGen)
+        {
+            if (kinds != null)
+                throw new IllegalArgumentException("Mixed both generator and 
individaul values for type kind");
+            this.typeKindGen = Objects.requireNonNull(typeKindGen);
+            return this;
+        }
+
+        // used during iteration, not something pluggable for users
+        private Gen<String> udtName = null;
+
+        public Gen<AbstractType<?>> build()
+        {
+            udtName = Generators.unique(IDENTIFIER_GEN);
+            // strip out the package to make it easier to read
+            // type parser assumes this package when one isn't provided, so 
this does not corrupt the type conversion
+            return buildRecursive(maxDepth).describedAs(t -> 
t.asCQL3Type().toString().replaceAll("org.apache.cassandra.db.marshal.", ""));
+        }
+
+        private Gen<AbstractType<?>> buildRecursive(int maxDepth)
+        {
+            if (udtName == null)
+                udtName = Generators.unique(IDENTIFIER_GEN);
+            Gen<TypeKind> kindGen;
+            if (typeKindGen != null)
+                kindGen = typeKindGen;
+            else if (kinds != null)
+            {
+                ArrayList<TypeKind> ts = new ArrayList<>(kinds);
+                Collections.sort(ts);
+                kindGen = SourceDSL.arbitrary().pick(ts);
+            }
+            else
+                kindGen = SourceDSL.arbitrary().enumValues(TypeKind.class);
+            return buildRecursive(maxDepth, maxDepth, kindGen, BOOLEAN_GEN);
+        }
+
+        private Gen<AbstractType<?>> buildRecursive(int maxDepth, int level, 
Gen<TypeKind> typeKindGen, Gen<Boolean> multiCellGen)
+        {
+            if (level == -1)
+                return primitiveGen;
+            assert level >= 0 : "max depth must be positive or zero; given " + 
level;
+            boolean atBottom = level == 0;
+            boolean atTop = maxDepth == level;
+            Gen<Boolean> multiCell = multiCellGen;

Review Comment:
   Nit: I think we can omit this var and directly use `multiCellGen`



##########
test/unit/org/apache/cassandra/utils/AbstractTypeGenerators.java:
##########
@@ -72,42 +131,344 @@
               TypeSupport.of(LongType.instance, SourceDSL.longs().all()),
               TypeSupport.of(FloatType.instance, SourceDSL.floats().any()),
               TypeSupport.of(DoubleType.instance, SourceDSL.doubles().any()),
-              TypeSupport.of(BytesType.instance, Generators.bytes(1, 1024)),
+              TypeSupport.of(BytesType.instance, Generators.bytes(0, 1024), 
FastByteOperations::compareUnsigned), // use the faster version...
               TypeSupport.of(UUIDType.instance, Generators.UUID_RANDOM_GEN),
-              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN), // serialization strips the hostname, 
only keeps the address
-              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(1, 1024)),
-              TypeSupport.of(UTF8Type.instance, Generators.utf8(1, 1024)),
+              TypeSupport.of(TimeUUIDType.instance, 
Generators.UUID_TIME_GEN.map(TimeUUID::fromUuid)),
+              TypeSupport.of(LexicalUUIDType.instance, 
Generators.UUID_RANDOM_GEN.mix(Generators.UUID_TIME_GEN)),
+              TypeSupport.of(InetAddressType.instance, 
Generators.INET_ADDRESS_UNRESOLVED_GEN, (a, b) -> 
FastByteOperations.compareUnsigned(a.getAddress(), b.getAddress())), // 
serialization strips the hostname, only keeps the address
+              TypeSupport.of(AsciiType.instance, 
SourceDSL.strings().ascii().ofLengthBetween(0, 1024), 
stringComparator(AsciiType.instance)),
+              TypeSupport.of(UTF8Type.instance, Generators.utf8(0, 1024), 
stringComparator(UTF8Type.instance)),
               TypeSupport.of(TimestampType.instance, Generators.DATE_GEN),
+              TypeSupport.of(SimpleDateType.instance, 
SourceDSL.integers().between(0, Integer.MAX_VALUE)), // can't use time gen as 
this is an int, and in Milliseconds... so overflows...
+              TypeSupport.of(TimeType.instance, SourceDSL.longs().between(0, 
24L * 60L * 60L * 1_000_000_000L - 1L)),
               // null is desired here as #decompose will call 
org.apache.cassandra.serializers.EmptySerializer.serialize which ignores the 
input and returns empty bytes
-              TypeSupport.of(EmptyType.instance, rnd -> null)
-              //TODO add the following
-              // IntegerType.instance,
-              // DecimalType.instance,
-              // TimeUUIDType.instance,
-              // LexicalUUIDType.instance,
-              // SimpleDateType.instance,
-              // TimeType.instance,
-              // DurationType.instance,
+              TypeSupport.of(EmptyType.instance, rnd -> null, (a, b) -> 0),
+              TypeSupport.of(DurationType.instance, 
CassandraGenerators.duration(), Comparator.comparingInt(Duration::getMonths)
+                                                                               
               .thenComparingInt(Duration::getDays)
+                                                                               
               .thenComparingLong(Duration::getNanoseconds)),
+              TypeSupport.of(IntegerType.instance, Generators.bigInt()),
+              TypeSupport.of(DecimalType.instance, Generators.bigDecimal())
     ).collect(Collectors.toMap(t -> t.type, t -> t));
     // NOTE not supporting reversed as CQL doesn't allow nested reversed types
     // when generating part of the clustering key, it would be good to allow 
reversed types as the top level
-    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN = 
SourceDSL.arbitrary().pick(new ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet()));
+    private static final Gen<AbstractType<?>> PRIMITIVE_TYPE_GEN;
+    static
+    {
+        ArrayList<AbstractType<?>> types = new 
ArrayList<>(PRIMITIVE_TYPE_DATA_GENS.keySet());
+        Collections.sort(types, Comparator.comparing(a -> 
a.getClass().getName()));
+        PRIMITIVE_TYPE_GEN = SourceDSL.arbitrary().pick(types);
+    }
+
+    private static final Set<Class<? extends AbstractType>> 
NON_PRIMITIVE_TYPES = ImmutableSet.<Class<? extends AbstractType>>builder()
+                                                                               
               .add(SetType.class)
+                                                                               
               .add(ListType.class)
+                                                                               
               .add(MapType.class)
+                                                                               
               .add(TupleType.class)
+                                                                               
               .add(UserType.class)
+                                                                               
               .add(VectorType.class)
+                                                                               
               .add(CompositeType.class)
+                                                                               
               .add(DynamicCompositeType.class)
+                                                                               
               .add(CounterColumnType.class)
+                                                                               
               .build();
 
     private AbstractTypeGenerators()
     {
 
     }
 
     public enum TypeKind
-    {PRIMITIVE, SET, LIST, MAP, TUPLE, UDT}
+    {
+        PRIMITIVE,
+        SET, LIST, MAP,
+        TUPLE, UDT,
+        VECTOR,
+        COMPOSITE, DYNAMIC_COMPOSITE,
+        COUNTER
+    }
 
     private static final Gen<TypeKind> TYPE_KIND_GEN = 
SourceDSL.arbitrary().enumValuesWithNoOrder(TypeKind.class);
 
+    public static Set<Class<? extends AbstractType>> knownTypes()
+    {
+        Set<Class<? extends AbstractType>> types = 
PRIMITIVE_TYPE_DATA_GENS.keySet().stream().map(a -> 
a.getClass()).collect(Collectors.toSet());
+        types.addAll(NON_PRIMITIVE_TYPES);
+        types.addAll(UNSUPPORTED.keySet());
+        return types;
+    }
+
     public static Gen<AbstractType<?>> primitiveTypeGen()
     {
         return PRIMITIVE_TYPE_GEN;
     }
 
+    public static Set<Class<? extends AbstractType<?>>> UNSAFE_EQUALITY = 
ImmutableSet.of(EmptyType.class,
+                                                                               
           DurationType.class,
+                                                                               
           DecimalType.class,
+                                                                               
           CounterColumnType.class);
+
+    public static <T> Releaser overridePrimitiveTypeSupport(AbstractType<T> 
type, TypeSupport<T> support)
+    {
+        if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(type))
+            throw new IllegalArgumentException("Type " + type.asCQL3Type() + " 
is not a primitive");
+        TypeSupport<?> original = PRIMITIVE_TYPE_DATA_GENS.get(type);
+        PRIMITIVE_TYPE_DATA_GENS.put(type, support);
+        return () -> PRIMITIVE_TYPE_DATA_GENS.put(type, original);
+    }
+
+    public static TypeGenBuilder withoutUnsafeEquality()
+    {
+        // make sure to keep UNSAFE_EQUALITY in-sync
+        return AbstractTypeGenerators.builder()
+                                     .withoutEmpty()
+                                     .withoutPrimitive(DurationType.instance)
+                                     // decimal "normalizes" the data to 
compare, so primary columns "may" mutate the data, causing missmatches
+                                     // see CASSANDRA-18530
+                                     .withoutPrimitive(DecimalType.instance)
+                                     // counters are only for top level
+                                     .withoutTypeKinds(TypeKind.COUNTER);
+    }
+
+    public interface Releaser extends AutoCloseable
+    {
+        @Override
+        void close();
+    }
+
+    public static class TypeGenBuilder
+    {
+        private int maxDepth = 3;
+        private EnumSet<TypeKind> kinds;
+        private Gen<TypeKind> typeKindGen;
+        private Gen<Integer> defaultSizeGen = VERY_SMALL_POSITIVE_SIZE_GEN;
+        private Gen<Integer> vectorSizeGen, tupleSizeGen, udtSizeGen, 
compositeSizeGen;
+        private Gen<AbstractType<?>> primitiveGen = PRIMITIVE_TYPE_GEN, 
compositeElementGen;
+        private Gen<String> userTypeKeyspaceGen = IDENTIFIER_GEN;
+        private Function<Integer, Gen<AbstractType<?>>> defaultSetKeyFunc;
+
+        public TypeGenBuilder()
+        {
+        }
+
+        public TypeGenBuilder(TypeGenBuilder other)
+        {
+            maxDepth = other.maxDepth;
+            kinds = other.kinds == null ? null : EnumSet.copyOf(other.kinds);
+            typeKindGen = other.typeKindGen;
+            defaultSizeGen = other.defaultSizeGen;
+            vectorSizeGen = other.vectorSizeGen;
+            tupleSizeGen = other.tupleSizeGen;
+            udtSizeGen = other.udtSizeGen;
+            primitiveGen = other.primitiveGen;
+            userTypeKeyspaceGen = other.userTypeKeyspaceGen;
+            defaultSetKeyFunc = other.defaultSetKeyFunc;
+        }
+
+        public TypeGenBuilder withCompositeElementGen(Gen<AbstractType<?>> gen)
+        {
+            this.compositeElementGen = gen;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(Function<Integer, 
Gen<AbstractType<?>>> mapKeyFunc)
+        {
+            this.defaultSetKeyFunc = mapKeyFunc;
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSetKey(TypeGenBuilder builder)
+        {
+            this.defaultSetKeyFunc = maxDepth -> 
builder.buildRecursive(maxDepth);
+            return this;
+        }
+
+        public TypeGenBuilder withUserTypeKeyspace(String keyspace)
+        {
+            userTypeKeyspaceGen = SourceDSL.arbitrary().constant(keyspace);
+            return this;
+        }
+
+        public TypeGenBuilder withDefaultSizeGen(int size)
+        {
+            return withDefaultSizeGen(SourceDSL.arbitrary().constant(size));
+        }
+
+        public TypeGenBuilder withDefaultSizeGen(Gen<Integer> sizeGen)
+        {
+            this.defaultSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withVectorSizeGen(Gen<Integer> sizeGen)
+        {
+            this.vectorSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withTupleSizeGen(Gen<Integer> sizeGen)
+        {
+            this.tupleSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withUDTSizeGen(Gen<Integer> sizeGen)
+        {
+            this.udtSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withCompositeSizeGen(Gen<Integer> sizeGen)
+        {
+            this.compositeSizeGen = sizeGen;
+            return this;
+        }
+
+        public TypeGenBuilder withoutEmpty()
+        {
+            return withoutPrimitive(EmptyType.instance);
+        }
+
+        public TypeGenBuilder withoutPrimitive(AbstractType<?> instance)
+        {
+            if (!PRIMITIVE_TYPE_DATA_GENS.keySet().contains(instance))

Review Comment:
   ```suggestion
               if (!PRIMITIVE_TYPE_DATA_GENS.containsKey(instance))
   ```



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