You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2020/08/24 23:46:51 UTC

[GitHub] [cassandra] jonmeredith commented on a change in pull request #716: Cassandra 16064

jonmeredith commented on a change in pull request #716:
URL: https://github.com/apache/cassandra/pull/716#discussion_r475949374



##########
File path: src/java/org/apache/cassandra/serializers/UserTypeSerializer.java
##########
@@ -36,9 +36,10 @@ public UserTypeSerializer(LinkedHashMap<String, TypeSerializer<?>> fields)
     public void validate(ByteBuffer bytes) throws MarshalException
     {
         ByteBuffer input = bytes.duplicate();
-        int i = 0;
+        int i = -1; // first thing in the loop is to increment, so when starting this will get set to 0 and match the fields

Review comment:
       This change doesn't seem necessary for the `MarshalException` check below, did you plan to assert something like at least one field exists?

##########
File path: test/unit/org/apache/cassandra/db/marshal/TimestampTypeTest.java
##########
@@ -0,0 +1,28 @@
+package org.apache.cassandra.db.marshal;

Review comment:
       Missing header.

##########
File path: test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
##########
@@ -233,5 +256,149 @@ public void testReversedTypeTuple() throws Throwable
         DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
         assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
     }
+
+    @Test
+    public void tuplePkReadWrite()
+    {
+        int numRows = 10;
+        class Case {
+            TupleType type;
+            List<ByteBuffer> uniqueRows;
+        }
+        Gen<TupleType> baseGen = getTupleType();
+        Gen<Case> gen = rnd -> {
+            Case c = new Case();
+            c.type = baseGen.generate(rnd);
+            TypeSupport<ByteBuffer> support = getTypeSupport(c.type);
+            Gen<ByteBuffer> valueGen = Generators.filter(support.valueGen, b -> b.remaining() <= Short.MAX_VALUE);
+            Set<ByteBuffer> distinctRows = new HashSet<>(numRows);
+            boolean done = false;
+            for (int i = 0; !done && i < numRows; i++)

Review comment:
       nit - could just write as a simpler loop - you bound the total number of generation attempts to get the list you want that way. I don't think `HashSet.size()` is expensive
   ```
   final int OVER_GEN_FACTOR = 10;
   for (int i = 0; distinctRows.size() < numRows && i < OVER_GEN_FACTOR * numRows; i++)
   {
       distinctRows.add(valueGen.generate(rnd)
   }
   if (distinctRows.size() != numRows)
   {
       System.err.println("Not enough uniq row for type " + c.type + ", returning with " + i + "rows");
   }

##########
File path: test/unit/org/apache/cassandra/db/marshal/TupleTypeTest.java
##########
@@ -0,0 +1,82 @@
+package org.apache.cassandra.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.Pair;
+import org.assertj.core.api.Assertions;
+import org.quicktheories.core.Gen;
+import org.quicktheories.generators.SourceDSL;
+
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getTypeSupport;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getTupleType;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getPrimitiveType;
+import static org.quicktheories.QuickTheory.qt;
+
+public class TupleTypeTest
+{
+    protected Gen<? extends TupleType> flatGen()
+    {
+        return getTupleType(getPrimitiveType(), SourceDSL.integers().between(0, 20));

Review comment:
       Are empty tuples permitted?

##########
File path: test/unit/org/apache/cassandra/db/marshal/TupleTypeTest.java
##########
@@ -0,0 +1,82 @@
+package org.apache.cassandra.db.marshal;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.cassandra.utils.Pair;
+import org.assertj.core.api.Assertions;
+import org.quicktheories.core.Gen;
+import org.quicktheories.generators.SourceDSL;
+
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getTypeSupport;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getTupleType;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getPrimitiveType;
+import static org.quicktheories.QuickTheory.qt;
+
+public class TupleTypeTest
+{
+    protected Gen<? extends TupleType> flatGen()
+    {
+        return getTupleType(getPrimitiveType(), SourceDSL.integers().between(0, 20));
+    }
+
+    protected Gen<? extends TupleType> nestedGen()
+    {
+        return getTupleType();
+    }
+
+    @Test
+    public void buildAndSpliteFlat()

Review comment:
       typo `buildAndSplitFlat()`

##########
File path: test/unit/org/apache/cassandra/db/marshal/UserTypeTest.java
##########
@@ -0,0 +1,22 @@
+package org.apache.cassandra.db.marshal;
+
+import org.quicktheories.core.Gen;
+import org.quicktheories.generators.SourceDSL;
+
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getPrimitiveType;
+import static org.apache.cassandra.utils.AbstractTypeGenerators.getUserType;
+
+public class UserTypeTest extends TupleTypeTest
+{
+    @Override
+    protected Gen<? extends TupleType> flatGen()
+    {
+        return getUserType(getPrimitiveType(), SourceDSL.integers().between(0, 20));

Review comment:
       Are zero length user types permitted?

##########
File path: src/java/org/apache/cassandra/serializers/TimestampSerializer.java
##########
@@ -188,7 +180,7 @@ public void validate(ByteBuffer bytes) throws MarshalException
 
     public String toString(Date value)
     {
-        return value == null ? "" : FORMATTER.get().format(value);
+        return toStringUTC(value);

Review comment:
       I like the extra resolution in the output, but was wondering why the change was necessary. Does anything rely on this being a particular format?

##########
File path: test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
##########
@@ -233,5 +256,149 @@ public void testReversedTypeTuple() throws Throwable
         DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
         assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
     }
+
+    @Test
+    public void tuplePkReadWrite()
+    {
+        int numRows = 10;
+        class Case {
+            TupleType type;
+            List<ByteBuffer> uniqueRows;
+        }
+        Gen<TupleType> baseGen = getTupleType();
+        Gen<Case> gen = rnd -> {
+            Case c = new Case();
+            c.type = baseGen.generate(rnd);
+            TypeSupport<ByteBuffer> support = getTypeSupport(c.type);
+            Gen<ByteBuffer> valueGen = Generators.filter(support.valueGen, b -> b.remaining() <= Short.MAX_VALUE);
+            Set<ByteBuffer> distinctRows = new HashSet<>(numRows);
+            boolean done = false;
+            for (int i = 0; !done && i < numRows; i++)
+            {
+                for (int retry = 0; retry < 10 && !distinctRows.add(valueGen.generate(rnd)); retry++)
+                {
+                    if (retry == 9)
+                    {
+                        System.err.println("Not enough uniq row for type " + c.type + ", returning with " + i + "rows");
+                        done = true;
+                        break;
+                    }
+                }
+            }
+            c.uniqueRows = new ArrayList<>(distinctRows);
+            return c;
+        };
+        gen = gen.describedAs(c -> c.type.asCQL3Type().toString());
+        qt().withExamples(10).withShrinkCycles(0).forAll(gen).checkAssert(orFail(testcase -> {
+            TupleType tupleType = testcase.type;
+            createTable("CREATE TABLE %s (id " + toCqlType(tupleType) + ", value int, PRIMARY KEY(id))");
+            SortedMap<ByteBuffer, Integer> map = new TreeMap<>(Comparator.comparing(currentTableMetadata().partitioner::decorateKey));
+            int count = 0;
+            for (ByteBuffer value : testcase.uniqueRows)
+            {
+                map.put(value, count);
+                ByteBuffer[] tupleBuffers = tupleType.split(value);
+
+                execute("INSERT INTO %s (id, value) VALUES (?, ?)", tuple(tupleBuffers), count);
+
+                assertRows(execute("SELECT * FROM %s WHERE id = ?", tuple(tupleBuffers)),
+                           row(tuple(tupleBuffers), count));
+                count++;
+            }
+            assertRows(execute("SELECT * FROM %s LIMIT 100"),
+                       map.entrySet().stream().map(e -> row(e.getKey(), e.getValue())).toArray(Object[][]::new));
+        }));
+    }
+
+    @Test
+    public void tupleCkReadWriteAsc()
+    {
+        tupleCkReadWrite(Order.ASC);
+    }
+
+    @Test
+    public void tupleCkReadWriteDesc()
+    {
+        tupleCkReadWrite(Order.Desc);
+    }
+
+    private void tupleCkReadWrite(Order order)
+    {
+        int numRows = 10;
+        class Case {
+            TupleType type;
+            List<ByteBuffer> uniqueRows;
+        }
+        Gen<TupleType> baseGen = getTupleType(getPrimitiveType(), SourceDSL.integers().between(1, 10));
+        Set<ByteBuffer> distinctRows = new HashSet<>(); // reuse the memory
+        Gen<Case> gen = rnd -> {
+            Case c = new Case();
+            c.type = baseGen.generate(rnd);
+            TypeSupport<ByteBuffer> support = getTypeSupport(c.type);
+            Gen<ByteBuffer> valueGen = Generators.filter(support.valueGen, b -> b.remaining() <= Short.MAX_VALUE);
+
+            distinctRows.clear();
+            for (int i = 0; i < numRows; i++)
+            {
+                while (!distinctRows.add(valueGen.generate(rnd))) { }

Review comment:
       Could this loop forever with any type with cardinality lower than `numRows=10` - for example`{boolean, boolean, boolean}`. I liked the bounded approach in the previous method.

##########
File path: src/java/org/apache/cassandra/db/marshal/UserType.java
##########
@@ -67,7 +68,10 @@ public UserType(String keyspace, ByteBuffer name, List<FieldIdentifier> fieldNam
         {
             String stringFieldName = fieldNames.get(i).toString();
             stringFieldNames.add(stringFieldName);
-            fieldSerializers.put(stringFieldName, fieldTypes.get(i).getSerializer());
+            TypeSerializer<?> previous = fieldSerializers.put(stringFieldName, fieldTypes.get(i).getSerializer());
+            assert previous == null : String.format("Duplicate names found for UDT(%s.%s) and column %s",

Review comment:
       If a user happened to have created an invalid schema in the field, would this prevent startup?

##########
File path: test/unit/org/apache/cassandra/db/marshal/TupleTypeTest.java
##########
@@ -0,0 +1,82 @@
+package org.apache.cassandra.db.marshal;

Review comment:
       Missing header.

##########
File path: test/unit/org/apache/cassandra/cql3/validation/entities/TupleTypeTest.java
##########
@@ -233,5 +256,149 @@ public void testReversedTypeTuple() throws Throwable
         DateFormat df = new SimpleDateFormat("yyyy-MM-dd HH:mmX", Locale.ENGLISH);
         assertRows(execute("SELECT tdemo FROM %s"), row(tuple( df.parse("2017-02-03 03:05+0000"), "Europe")));
     }
+
+    @Test
+    public void tuplePkReadWrite()
+    {
+        int numRows = 10;
+        class Case {
+            TupleType type;
+            List<ByteBuffer> uniqueRows;
+        }
+        Gen<TupleType> baseGen = getTupleType();
+        Gen<Case> gen = rnd -> {
+            Case c = new Case();
+            c.type = baseGen.generate(rnd);
+            TypeSupport<ByteBuffer> support = getTypeSupport(c.type);
+            Gen<ByteBuffer> valueGen = Generators.filter(support.valueGen, b -> b.remaining() <= Short.MAX_VALUE);
+            Set<ByteBuffer> distinctRows = new HashSet<>(numRows);
+            boolean done = false;
+            for (int i = 0; !done && i < numRows; i++)
+            {
+                for (int retry = 0; retry < 10 && !distinctRows.add(valueGen.generate(rnd)); retry++)
+                {
+                    if (retry == 9)
+                    {
+                        System.err.println("Not enough uniq row for type " + c.type + ", returning with " + i + "rows");
+                        done = true;
+                        break;
+                    }
+                }
+            }
+            c.uniqueRows = new ArrayList<>(distinctRows);
+            return c;
+        };
+        gen = gen.describedAs(c -> c.type.asCQL3Type().toString());
+        qt().withExamples(10).withShrinkCycles(0).forAll(gen).checkAssert(orFail(testcase -> {
+            TupleType tupleType = testcase.type;
+            createTable("CREATE TABLE %s (id " + toCqlType(tupleType) + ", value int, PRIMARY KEY(id))");
+            SortedMap<ByteBuffer, Integer> map = new TreeMap<>(Comparator.comparing(currentTableMetadata().partitioner::decorateKey));
+            int count = 0;
+            for (ByteBuffer value : testcase.uniqueRows)
+            {
+                map.put(value, count);
+                ByteBuffer[] tupleBuffers = tupleType.split(value);
+
+                execute("INSERT INTO %s (id, value) VALUES (?, ?)", tuple(tupleBuffers), count);
+
+                assertRows(execute("SELECT * FROM %s WHERE id = ?", tuple(tupleBuffers)),
+                           row(tuple(tupleBuffers), count));
+                count++;
+            }
+            assertRows(execute("SELECT * FROM %s LIMIT 100"),
+                       map.entrySet().stream().map(e -> row(e.getKey(), e.getValue())).toArray(Object[][]::new));
+        }));
+    }
+
+    @Test
+    public void tupleCkReadWriteAsc()
+    {
+        tupleCkReadWrite(Order.ASC);
+    }
+
+    @Test
+    public void tupleCkReadWriteDesc()
+    {
+        tupleCkReadWrite(Order.Desc);
+    }
+
+    private void tupleCkReadWrite(Order order)
+    {
+        int numRows = 10;
+        class Case {
+            TupleType type;
+            List<ByteBuffer> uniqueRows;
+        }
+        Gen<TupleType> baseGen = getTupleType(getPrimitiveType(), SourceDSL.integers().between(1, 10));
+        Set<ByteBuffer> distinctRows = new HashSet<>(); // reuse the memory
+        Gen<Case> gen = rnd -> {
+            Case c = new Case();
+            c.type = baseGen.generate(rnd);
+            TypeSupport<ByteBuffer> support = getTypeSupport(c.type);
+            Gen<ByteBuffer> valueGen = Generators.filter(support.valueGen, b -> b.remaining() <= Short.MAX_VALUE);
+
+            distinctRows.clear();
+            for (int i = 0; i < numRows; i++)
+            {
+                while (!distinctRows.add(valueGen.generate(rnd))) { }
+            }
+            c.uniqueRows = new ArrayList<>(distinctRows);
+            return c;
+        };
+        gen = gen.describedAs(c -> c.type.asCQL3Type().toString());

Review comment:
       On failure you wouldn't get to see the `uniqueRows`.




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

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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