You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2020/03/20 13:18:19 UTC

[GitHub] [drill] cgivre opened a new pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

cgivre opened a new pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033
 
 
   # [DRILL-7652](https://issues.apache.org/jira/browse/DRILL-7652): Add time_bucket() function for Time Series Analysis
   
   ## Description
   
   This PR adds two UDFs which facilitate time series analysis.  This PR also includes updates to the `README.md` in the `contrib/udf` folder to reflect the new UDF.
   
   ## Documentation
   These functions are useful for doing time series analysis by grouping the data into arbitrary intervals.  See: https://blog.timescale.com/blog/simplified-time-series-analytics
   -using-the-time_bucket-function/ for more examples. 
   
   There are two versions of the function:
   * `time_bucket(<timestamp>, <interval>)`
   * `time_bucket_ns(<timestamp>,<interval>)`
   
   Both functions accept a `BIGINT` timestamp and an interval in milliseconds as arguments. The `time_bucket_ns()` function accepts timestamps in nanoseconds and `time_bucket
   ()` accepts timestamps in milliseconds.  Both return timestamps in the original format.
   
   ### Example:
   The query below calculates the average for the `cpu` metric for every five minute interval.
   
   ```sql
   SELECT time_bucket(time_stamp, 30000) AS five_min, avg(cpu)
     FROM metrics
     GROUP BY five_min
     ORDER BY five_min DESC LIMIT 12;
   ```
   
   ## Testing
   There are a series of unit tests included with this PR.

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


With regards,
Apache Git Services

[GitHub] [drill] arina-ielchiieva commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
arina-ielchiieva commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601744663
 
 
   Igor will run all tests for you branch and let us know if there are any issues.
   @cgivre if everything is OK, PR can be merged tomorrow.

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


With regards,
Apache Git Services

[GitHub] [drill] ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395664288
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   no problems, you are a little bit in a rush I guess since in your second commit you left commented line. Also, it would be good to rerun your test just to be sure that the suggestion works fine. 

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


With regards,
Apache Git Services

[GitHub] [drill] cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395689299
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   @ihuzenko 
   Thanks for the review.  Can you (or I) go ahead and commit this to master?  I'm working on a project over the weekend and that would be very helpful.
   

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


With regards,
Apache Git Services

[GitHub] [drill] cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395660292
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   @ihuzenko 
   Wow... yes.... that is a lot simpler.  That is what happens when you are coding at 2AM... ;-)
   (Fixed)

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


With regards,
Apache Git Services

[GitHub] [drill] cgivre commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
cgivre commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601745932
 
 
   @arina-ielchiieva, @ihuzenko 
   Thank you very much for the quick turnaround.  You don't have to give up your weekend however for this.  Monday is fine. 

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


With regards,
Apache Git Services

[GitHub] [drill] arina-ielchiieva merged pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
arina-ielchiieva merged pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033
 
 
   

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


With regards,
Apache Git Services

[GitHub] [drill] ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395657446
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   isn't simpler formula ```timestamp - (timestamp % interval)``` ? 

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


With regards,
Apache Git Services

[GitHub] [drill] ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
ihuzenko commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395688425
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   yes, looks good. Thanks for the changes.

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


With regards,
Apache Git Services

[GitHub] [drill] arina-ielchiieva commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
arina-ielchiieva commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601743413
 
 
   @cgivre it's a common rule / courtesy to keep PRs open at least for one day so others can take a look if they want to, exceptions apply to something really minor or emergency bug fix.

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


With regards,
Apache Git Services

[GitHub] [drill] arina-ielchiieva edited a comment on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
arina-ielchiieva edited a comment on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601744663
 
 
   Igor will run all tests for your branch and let us know if there are any issues.
   @cgivre if everything is OK, PR can be merged tomorrow.

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


With regards,
Apache Git Services

[GitHub] [drill] cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
cgivre commented on a change in pull request #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#discussion_r395667595
 
 

 ##########
 File path: contrib/udfs/src/main/java/org/apache/drill/exec/udfs/TimeBucketFunctions.java
 ##########
 @@ -0,0 +1,105 @@
+/*
+ * 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.drill.exec.udfs;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+
+
+public class TimeBucketFunctions {
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in nanoseconds
+   * 2. The desired bucket interval IN MILLISECONDS
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket_ns",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketNSFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in nanoseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds and convert to nanoseconds
+      long intervalToAdd = interval.value * 1000000;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
+    }
+  }
+
+  /**
+   * This function is used for facilitating time series analysis by creating buckets of time intervals.  See
+   * https://blog.timescale.com/blog/simplified-time-series-analytics-using-the-time_bucket-function/ for usage. The function takes two arguments:
+   * 1. The timestamp in milliseconds
+   * 2. The desired bucket interval IN milliseconds
+   *
+   * The function returns a BIGINT of the nearest time bucket.
+   */
+  @FunctionTemplate(name = "time_bucket",
+    scope = FunctionTemplate.FunctionScope.SIMPLE,
+    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class TimeBucketFunction implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDate;
+
+    @Param
+    BigIntHolder interval;
+
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // Get the timestamp in milliseconds
+      long timestamp = inputDate.value;
+
+      // Get the interval in milliseconds
+      long intervalToAdd = interval.value;
+
+      out.value = (long) java.lang.Math.floor(timestamp / intervalToAdd) * intervalToAdd;
 
 Review comment:
   @ihuzenko 
   I did rerun all the tests before I submitted and everything passed..  But I did forget to remove the commented out line.  I saw that naturally after I pushed, but I got rid of it.  Commits are squashed.  Are we good to go? 

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


With regards,
Apache Git Services

[GitHub] [drill] cgivre commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
cgivre commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601744021
 
 
   @arina-ielchiieva 
   Thanks for the response.  In that case, we can wait.  Definitely not an emergency. 

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


With regards,
Apache Git Services

[GitHub] [drill] ihuzenko commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis

Posted by GitBox <gi...@apache.org>.
ihuzenko commented on issue #2033: DRILL-7652: Add time_bucket() function for time series analysis
URL: https://github.com/apache/drill/pull/2033#issuecomment-601805823
 
 
   @cgivre @arina-ielchiieva 
   tests passed on a test cluster. 

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


With regards,
Apache Git Services