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 2022/04/25 20:50:26 UTC

[GitHub] [cassandra] frankgh commented on a diff in pull request #1584: CASSANDRA-17425: Add new CQL function maxWritetime

frankgh commented on code in PR #1584:
URL: https://github.com/apache/cassandra/pull/1584#discussion_r858011361


##########
test/unit/org/apache/cassandra/cql3/selection/SelectorSerializationTest.java:
##########
@@ -60,8 +60,8 @@ public void testSerDes() throws IOException
         checkSerialization(table.getColumn(new ColumnIdentifier("c1", false)), table);
 
         // Test WritetimeOrTTLSelector serialization
-        checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), true), table);
-        checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), false), table);
+        checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), Selectable.WritetimeOrTTL.Kind.WRITE_TIME), table);
+        checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), Selectable.WritetimeOrTTL.Kind.TTL), table);

Review Comment:
   Can we add the `MAX_WRITE_TIME` kind here as well?
   ```suggestion
           checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), Selectable.WritetimeOrTTL.Kind.TTL), table);
           checkSerialization(new Selectable.WritetimeOrTTL(table.getColumn(new ColumnIdentifier("v", false)), Selectable.WritetimeOrTTL.Kind.MAX_WRITE_TIME), table);
   ```



##########
src/java/org/apache/cassandra/cql3/selection/Selectable.java:
##########
@@ -245,18 +266,20 @@ public Selector.Factory newSelectorFactory(TableMetadata table,
             if (column.isPrimaryKeyColumn())
                 throw new InvalidRequestException(
                         String.format("Cannot use selection function %s on PRIMARY KEY part %s",
-                                      isWritetime ? "writeTime" : "ttl",
+                                      kind.name,
                                       column.name));
-            if (column.type.isCollection())
+
+            // maxwritetime is allowed for collection
+            if (kind != Kind.MAX_WRITE_TIME && column.type.isCollection())

Review Comment:
   we could potentially move this to the enum? `if (column.type.isCollection() && !kind.allowedForCollection()) then throw...`. If we ever add additional `Kind`s, we wouldn't need to touch this logic, so it becomes a little more maintainable



##########
src/java/org/apache/cassandra/cql3/selection/ResultSetBuilder.java:
##########
@@ -98,6 +102,27 @@ public void add(ByteBuffer v)
         inputRow.add(v);
     }
 
+    public void add(ComplexColumnData complexColumnData, Function<Iterator<Cell<?>>, ByteBuffer> serializer)
+    {
+        if (complexColumnData == null)
+        {
+            inputRow.add(null);
+            return;
+        }
+
+        long timestamp = -1L;
+        if (selectors.collectMaxTimestamps())
+        {
+            Iterator<Cell<?>> cells = complexColumnData.iterator();
+            while (cells.hasNext())
+            {
+                timestamp = Math.max(timestamp, cells.next().timestamp());
+            }
+        }

Review Comment:
   We could use streams here, although we might need to worry about performance 
   ```suggestion
           long timestamp = -1L;
           if (selectors.collectMaxTimestamps())
           {
               timestamp = StreamSupport.stream(complexColumnData.spliterator(), false)
                                        .mapToLong(Cell::timestamp)
                                        .max()
                                        .orElse(-1L);
           }
   ```



##########
src/java/org/apache/cassandra/cql3/selection/WritetimeOrTTLSelector.java:
##########
@@ -169,28 +176,28 @@ public boolean equals(Object o)
 
         return Objects.equal(column, s.column)
             && Objects.equal(idx, s.idx)
-            && Objects.equal(isWritetime, s.isWritetime);
+            && kind == s.kind;
     }
 
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(column, idx, isWritetime);
+        return Objects.hashCode(column, idx, kind);
     }
 
     @Override
     protected int serializedSize(int version)
     {
         return ByteBufferUtil.serializedSizeWithVIntLength(column.name.bytes)
                 + TypeSizes.sizeof(idx)
-                + TypeSizes.sizeof(isWritetime);
+                + TypeSizes.sizeofUnsignedVInt(kind.ordinal());
     }
 
     @Override
     protected void serialize(DataOutputPlus out, int version) throws IOException
     {
         ByteBufferUtil.writeWithVIntLength(column.name.bytes, out);
         out.writeInt(idx);
-        out.writeBoolean(isWritetime);
+        out.writeUnsignedVInt(kind.ordinal());

Review Comment:
   we could make this serialization more compact. I don't think we'll have more that 256 enum types 
   ```suggestion
           out.writeByte(kind.ordinal());
   ```



-- 
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: pr-unsubscribe@cassandra.apache.org

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