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/21 04:14:00 UTC

[GitHub] [cassandra] yifan-c commented on a diff in pull request #1510: CASSANDRA-11871: Allow to aggregate by time intervals

yifan-c commented on code in PR #1510:
URL: https://github.com/apache/cassandra/pull/1510#discussion_r854659262


##########
src/java/org/apache/cassandra/cql3/functions/TimeFcts.java:
##########
@@ -202,7 +231,280 @@ public ByteBuffer execute(ProtocolVersion protocolVersion, List<ByteBuffer> para
 
                 return ByteBufferUtil.bytes(type.toTimeInMillis(bb));
             }
+
+            @Override
+            public boolean isMonotonic()
+            {
+                return true;
+            }
         };
     }
-}
+
+    /**
+     * Function that rounds a timestamp down to the closest multiple of a duration.
+     */
+     private static abstract class FloorFunction extends NativeScalarFunction
+     {
+         private static final Long ZERO = Long.valueOf(0);
+
+         protected FloorFunction(AbstractType<?> returnType,
+                                 AbstractType<?>... argsType)
+         {
+             super("floor", returnType, argsType);
+         }
+
+         @Override
+         protected boolean isPartialApplicationMonotonic(List<ByteBuffer> partialParameters)
+         {
+             return partialParameters.get(0) == UNRESOLVED
+                     && partialParameters.get(1) != UNRESOLVED
+                     && (partialParameters.size() == 2 || partialParameters.get(2) != UNRESOLVED);

Review Comment:
   It relies on the fact that there are at most 3 arguments, which is true with the current implementations.
   Maybe add an assertion for the number of arguments and a short comment in the constructor? so authors that add new functions with over 3 arguments in the future are informed. 



##########
test/unit/org/apache/cassandra/cql3/DurationTest.java:
##########
@@ -109,72 +108,278 @@ public void testInvalidDurations()
     @Test
     public void testAddTo()
     {
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("0m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("10us").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:10:00"), Duration.from("10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T01:30:00"), Duration.from("90m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T02:10:00"), Duration.from("2h10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-23T00:10:00"), Duration.from("2d10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-24T01:00:00"), Duration.from("2d25h").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-10-21T00:00:00"), Duration.from("1mo").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2017-11-21T00:00:00"), Duration.from("14mo").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2017-02-28T00:00:00"), Duration.from("12mo").addTo(toMillis("2016-02-29T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("0m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("10us").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:10:00.000", Duration.from("10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T01:30:00.000", Duration.from("90m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T02:10:00.000", Duration.from("2h10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-23T00:10:00.000", Duration.from("2d10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-24T01:00:00.000", Duration.from("2d25h").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-10-21T00:00:00.000", Duration.from("1mo").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2017-11-21T00:00:00.000", Duration.from("14mo").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2017-02-28T00:00:00.000", Duration.from("12mo").addTo(toMillis("2016-02-29T00:00:00")));
     }
 
     @Test
     public void testAddToWithNegativeDurations()
     {
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("-0m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("-10us").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T23:50:00"), Duration.from("-10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T22:30:00"), Duration.from("-90m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T21:50:00"), Duration.from("-2h10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-18T23:50:00"), Duration.from("-2d10m").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-17T23:00:00"), Duration.from("-2d25h").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-08-21T00:00:00"), Duration.from("-1mo").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2015-07-21T00:00:00"), Duration.from("-14mo").addTo(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2015-02-28T00:00:00"), Duration.from("-12mo").addTo(toMillis("2016-02-29T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("-0m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("-10us").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T23:50:00.000", Duration.from("-10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T22:30:00.000", Duration.from("-90m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T21:50:00.000", Duration.from("-2h10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-18T23:50:00.000", Duration.from("-2d10m").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-17T23:00:00.000", Duration.from("-2d25h").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-08-21T00:00:00.000", Duration.from("-1mo").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2015-07-21T00:00:00.000", Duration.from("-14mo").addTo(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2015-02-28T00:00:00.000", Duration.from("-12mo").addTo(toMillis("2016-02-29T00:00:00")));
     }
 
     @Test
     public void testSubstractFrom()
     {
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("0m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("10us").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T23:50:00"), Duration.from("10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T22:30:00"), Duration.from("90m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-20T21:50:00"), Duration.from("2h10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-18T23:50:00"), Duration.from("2d10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-17T23:00:00"), Duration.from("2d25h").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-08-21T00:00:00"), Duration.from("1mo").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2015-07-21T00:00:00"), Duration.from("14mo").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2015-02-28T00:00:00"), Duration.from("12mo").substractFrom(toMillis("2016-02-29T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("0m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("10us").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T23:50:00.000", Duration.from("10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T22:30:00.000", Duration.from("90m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-20T21:50:00.000", Duration.from("2h10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-18T23:50:00.000", Duration.from("2d10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-17T23:00:00.000", Duration.from("2d25h").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-08-21T00:00:00.000", Duration.from("1mo").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2015-07-21T00:00:00.000", Duration.from("14mo").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2015-02-28T00:00:00.000", Duration.from("12mo").substractFrom(toMillis("2016-02-29T00:00:00")));
     }
 
     @Test
     public void testSubstractWithNegativeDurations()
     {
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("-0m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:00:00"), Duration.from("-10us").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T00:10:00"), Duration.from("-10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T01:30:00"), Duration.from("-90m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-21T02:10:00"), Duration.from("-2h10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-23T00:10:00"), Duration.from("-2d10m").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-09-24T01:00:00"), Duration.from("-2d25h").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2016-10-21T00:00:00"), Duration.from("-1mo").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2017-11-21T00:00:00"), Duration.from("-14mo").substractFrom(toMillis("2016-09-21T00:00:00")));
-        assertEquals(toMillis("2017-02-28T00:00:00"), Duration.from("-12mo").substractFrom(toMillis("2016-02-29T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("-0m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:00:00.000", Duration.from("-10us").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T00:10:00.000", Duration.from("-10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T01:30:00.000", Duration.from("-90m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-21T02:10:00.000", Duration.from("-2h10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-23T00:10:00.000", Duration.from("-2d10m").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-09-24T01:00:00.000", Duration.from("-2d25h").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2016-10-21T00:00:00.000", Duration.from("-1mo").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2017-11-21T00:00:00.000", Duration.from("-14mo").substractFrom(toMillis("2016-09-21T00:00:00")));
+        assertTimeEquals("2017-02-28T00:00:00.000", Duration.from("-12mo").substractFrom(toMillis("2016-02-29T00:00:00")));
+    }
+
+    @Test
+    public void testInvalidFloorTimestamp()
+    {
+        try
+        {
+            floorTimestamp("2016-09-27T16:12:00", "2h", "2017-09-01T00:00:00");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            assertEquals("The floor function starting time is greater than the provided time", e.getMessage());
+        }
+
+        try
+        {
+            floorTimestamp("2016-09-27T16:12:00", "-2h", "2016-09-27T00:00:00");
+            Assert.fail();
+        }
+        catch (InvalidRequestException e)
+        {
+            assertEquals("Negative durations are not supported by the floor function", e.getMessage());
+        }
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationInHours()
+    {
+        // Test floor with a timestamp equals to the start time
+        long result = floorTimestamp("2016-09-27T16:12:00", "2h", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T16:12:00.000", result);
+
+        // Test floor with a duration equals to zero
+        result = floorTimestamp("2016-09-27T18:12:00", "0h", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T18:12:00.000", result);
+
+        // Test floor with a timestamp exactly equals to the start time + (1 x duration)
+        result = floorTimestamp("2016-09-27T18:12:00", "2h", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T18:12:00.000", result);
+
+        // Test floor with a timestamp in the first bucket
+        result = floorTimestamp("2016-09-27T16:13:00", "2h", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T16:12:00.000", result);
+
+        // Test floor with a timestamp in another bucket
+        result = floorTimestamp("2016-09-27T16:12:00", "2h", "2016-09-27T00:00:00");
+        assertTimeEquals("2016-09-27T16:00:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationInDays()
+    {
+        // Test floor with a start time at the beginning of the month
+        long result = floorTimestamp("2016-09-27T16:12:00", "2d", "2016-09-01T00:00:00");
+        assertTimeEquals("2016-09-27T00:00:00.000", result);
+
+        // Test floor with a start time in the previous month
+        result = floorTimestamp("2016-09-27T16:12:00", "2d", "2016-08-01T00:00:00");
+        assertTimeEquals("2016-09-26T00:00:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationInDaysAndHours()
+    {
+        long result = floorTimestamp("2016-09-27T16:12:00", "2d12h", "2016-09-01T00:00:00");
+        assertTimeEquals("2016-09-26T00:00:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationInMonths()
+    {
+        // Test floor with a timestamp equals to the start time
+        long result = floorTimestamp("2016-09-01T00:00:00", "2mo", "2016-09-01T00:00:00");
+        assertTimeEquals("2016-09-01T00:00:00.000", result);
+
+        // Test floor with a timestamp in the first bucket
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo", "2016-09-01T00:00:00");
+        assertTimeEquals("2016-09-01T00:00:00.000", result);
+
+        // Test floor with a start time at the beginning of the year (LEAP YEAR)
+        result = floorTimestamp("2016-09-27T16:12:00", "1mo", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-09-01T00:00:00.000", result);
+
+        // Test floor with a start time at the beginning of the previous year
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo", "2015-01-01T00:00:00");
+        assertTimeEquals("2016-09-01T00:00:00.000", result);
+
+        // Test floor with a start time in the previous year
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo", "2015-02-02T00:00:00");
+        assertTimeEquals("2016-08-02T00:00:00.000", result);
+
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationInMonthsAndDays()
+    {
+        long result = floorTimestamp("2016-09-27T16:12:00", "2mo2d", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-09-09T00:00:00.000", result);
+
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo5d", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-09-21T00:00:00.000", result);
+
+        // Test floor with a timestamp in the first bucket
+        result = floorTimestamp("2016-09-04T16:12:00", "2mo5d", "2016-07-01T00:00:00");
+        assertTimeEquals("2016-07-01T00:00:00.000", result);
+
+        // Test floor with a timestamp in a bucket starting on the last day of the month
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo10d", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-07-31T00:00:00.000", result);
+
+        // Test floor with a timestamp in a bucket starting on the first day of the month
+        result = floorTimestamp("2016-09-27T16:12:00", "2mo12d", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-08-06T00:00:00.000", result);
+
+        // Test leap years
+        result = floorTimestamp("2016-04-27T16:12:00", "1mo30d", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-03-02T00:00:00.000", result);
+
+        result = floorTimestamp("2015-04-27T16:12:00", "1mo30d", "2015-01-01T00:00:00");
+        assertTimeEquals("2015-03-03T00:00:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithDurationSmallerThanPrecision()
+    {
+        long result = floorTimestamp("2016-09-27T18:14:00", "5us", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T18:14:00.000", result);
+
+        result = floorTimestamp("2016-09-27T18:14:00", "1h5us", "2016-09-27T16:12:00");
+        assertTimeEquals("2016-09-27T18:12:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithLeapSecond()
+    {
+        long result = floorTimestamp("2016-07-02T00:00:00", "2m", "2016-06-30T23:58:00");
+        assertTimeEquals("2016-07-02T00:00:00.000", result);
+    }
+
+    @Test
+    public void testFloorTimestampWithComplexDuration()
+    {
+        long result = floorTimestamp("2016-07-02T00:00:00", "2mo2d8h", "2016-01-01T00:00:00");
+        assertTimeEquals("2016-05-05T16:00:00.000", result);
+    }
+
+    @Test
+    public void testInvalidFloorTime()

Review Comment:
   There is no happy path test case for `floorTime`. Maybe add this one too
   
   ```java
       @Test
       public void testFloorTime()
       {
           long time = floorTime("16:12:00", "2h");
           Duration result = Duration.newInstance(0, 0, time);
           Duration expected = Duration.from("16h");
           assertEquals(expected, result);
       }
   ```



##########
src/java/org/apache/cassandra/cql3/selection/AbstractFunctionSelector.java:
##########
@@ -17,24 +17,96 @@
  */
 package org.apache.cassandra.cql3.selection;
 
+import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
+import java.util.Optional;
 
+import com.google.common.base.Objects;
 import com.google.common.collect.Iterables;
 
 import org.apache.commons.lang3.text.StrBuilder;
 import org.apache.cassandra.schema.ColumnMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.Schema;
+import org.apache.cassandra.schema.TableMetadata;
+import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.cql3.ColumnSpecification;
 import org.apache.cassandra.cql3.QueryOptions;
 import org.apache.cassandra.cql3.functions.Function;
+import org.apache.cassandra.cql3.functions.FunctionName;
+import org.apache.cassandra.cql3.functions.PartialScalarFunction;
+import org.apache.cassandra.cql3.functions.ScalarFunction;
 import org.apache.cassandra.cql3.statements.RequestValidations;
+import org.apache.cassandra.db.TypeSizes;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.DataOutputPlus;
+
+import static java.util.stream.Collectors.joining;
 
 abstract class AbstractFunctionSelector<T extends Function> extends Selector
 {
+    protected static abstract class AbstractFunctionSelectorDeserializer extends SelectorDeserializer
+    {
+        protected Selector deserialize(DataInputPlus in, int version, TableMetadata metadata) throws IOException
+        {
+            FunctionName name = new FunctionName(in.readUTF(), in.readUTF());
+
+            KeyspaceMetadata keyspace = Schema.instance.getKeyspaceMetadata(metadata.keyspace);
+
+            int numberOfArguments = (int) in.readUnsignedVInt();
+            List<AbstractType<?>> argTypes = new ArrayList<>(numberOfArguments);
+            for (int i = 0; i < numberOfArguments; i++)
+            {
+                argTypes.add(readType(keyspace, in));

Review Comment:
   nit: `readType` has an overload to accept `TableMetadata`.
   
   ```suggestion
               int numberOfArguments = (int) in.readUnsignedVInt();
               List<AbstractType<?>> argTypes = new ArrayList<>(numberOfArguments);
               for (int i = 0; i < numberOfArguments; i++)
               {
                   argTypes.add(readType(metadata, in));
   ```
   



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