You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by GitBox <gi...@apache.org> on 2020/05/03 01:31:00 UTC

[GitHub] [incubator-pinot] reallocf opened a new pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

reallocf opened a new pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326


   Working toward #5313 - still need to implement toLocalDateTime and fromLocalDateTime. This commit just refactors the FunctionRegistry to allow non-static functions to be registered.


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r422550841



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -27,6 +32,8 @@
  */
 public class DateTimeFunctions {
 
+  private Map<String, DateTimeFormatter> _dateTimeFormatterMap = new HashMap<>();

Review comment:
       Good catch! Yeah, I'm still new to the concurrency model here - thanks for keeping an eye out for this! :)




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623039504


   Question: how would y'all test something like this locally? I ran the immediately relevant tests in IntelliJ, then triggered a `mvn test` (still running a while later). What command would y'all run to feel confident?


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419034116



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -34,69 +35,72 @@
   @Test
   public void testExpressionWithColumn()
       throws Exception {
-    Method method = MyFunc.class.getDeclaredMethod("reverseString", String.class);
-    FunctionRegistry.registerStaticFunction(method);
-    FunctionInfo functionInfo = FunctionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("reverseString", String.class)));
+    FunctionInfo functionInfo = functionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
     System.out.println(functionInfo);
     String expression = "reverseString(testColumn)";
 
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertEquals(evaluator.getArguments(), Lists.newArrayList("testColumn"));
     GenericRow row = new GenericRow();
     for (int i = 0; i < 5; i++) {
       String value = "testValue" + i;
       row.putField("testColumn", value);
       Object result = evaluator.evaluate(row);
-      Assert.assertEquals(result, new StringBuilder(value).reverse().toString());
+      Assert.assertEquals(result, myFunc.reverseString(value));
     }
   }
 
   @Test
   public void testExpressionWithConstant()
       throws Exception {
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch('%s', '%s')", input, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
   @Test
   public void testMultiFunctionExpression()
       throws Exception {
-    FunctionRegistry.registerStaticFunction(MyFunc.class.getDeclaredMethod("reverseString", String.class));
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(Arrays
+        .asList(myFunc.getClass().getDeclaredMethod("reverseString", String.class),

Review comment:
       `Lists.newArrayList(...)`




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r422551022



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -40,4 +47,14 @@ static Long toEpochHours(Long millis) {
   static Long toEpochMinutes(Long millis, String bucket) {
     return TimeUnit.MILLISECONDS.toMinutes(millis) / Integer.parseInt(bucket);
   }
+
+  DateTime toDateTime(String dateTimeString, String pattern) {
+    if (!_dateTimeFormatterMap.containsKey(pattern)) {
+      _dateTimeFormatterMap.put(pattern, DateTimeFormat.forPattern(pattern));
+    }
+
+    DateTimeFormatter dateTimeFormatter = _dateTimeFormatterMap.get(pattern);
+
+    return dateTimeFormatter.parseDateTime(dateTimeString);

Review comment:
       Okay, adjusted this so toDateTime takes in a Long millis and returns a String formattedDateTime and fromDateTime takes in a String formattedDateTime and outputs a Long millis - does that seem right to you?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r425407301



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimePatternHandler.java
##########
@@ -0,0 +1,45 @@
+/**
+ * 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.pinot.core.data.function;
+
+import java.util.concurrent.ConcurrentHashMap;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+
+
+public class DateTimePatternHandler {
+  private final ConcurrentHashMap<String, DateTimeFormatter> patternDateTimeFormatterCache = new ConcurrentHashMap<>();
+
+  public Long parseDateTimeStringToMillis(String dateTimeString, String pattern) {
+    DateTimeFormatter dateTimeFormatter = getDateTimeFormatterFromCache(pattern);
+    return dateTimeFormatter.parseMillis(dateTimeString);
+  }
+
+  public String parseMillisToDateTimeString(Long millis, String pattern) {
+    DateTimeFormatter dateTimeFormatter = getDateTimeFormatterFromCache(pattern);
+    return dateTimeFormatter.print(millis);
+  }
+
+  private DateTimeFormatter getDateTimeFormatterFromCache(String pattern) {
+    DateTimeFormatter dateTimeFormatter =
+        patternDateTimeFormatterCache.getOrDefault(pattern, DateTimeFormat.forPattern(pattern));

Review comment:
       Ahh, very slick! :)




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar edited a comment on pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar edited a comment on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623586440


   Could you please also add to the SchemaUtilsTest#testSourceFieldExtractorName, to make sure we're extracting arguments right.
   


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419042856



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -34,69 +35,72 @@
   @Test
   public void testExpressionWithColumn()
       throws Exception {
-    Method method = MyFunc.class.getDeclaredMethod("reverseString", String.class);
-    FunctionRegistry.registerStaticFunction(method);
-    FunctionInfo functionInfo = FunctionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("reverseString", String.class)));
+    FunctionInfo functionInfo = functionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
     System.out.println(functionInfo);
     String expression = "reverseString(testColumn)";
 
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertEquals(evaluator.getArguments(), Lists.newArrayList("testColumn"));
     GenericRow row = new GenericRow();
     for (int i = 0; i < 5; i++) {
       String value = "testValue" + i;
       row.putField("testColumn", value);
       Object result = evaluator.evaluate(row);
-      Assert.assertEquals(result, new StringBuilder(value).reverse().toString());
+      Assert.assertEquals(result, myFunc.reverseString(value));

Review comment:
       There's some testing value to keeping the `new StringBuilder...` - will change back




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419034064



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -19,7 +19,8 @@
 package org.apache.pinot.core.data.function;
 
 import com.google.common.collect.Lists;
-import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.Collections;

Review comment:
       Use `Lists` instead

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -34,69 +35,72 @@
   @Test
   public void testExpressionWithColumn()
       throws Exception {
-    Method method = MyFunc.class.getDeclaredMethod("reverseString", String.class);
-    FunctionRegistry.registerStaticFunction(method);
-    FunctionInfo functionInfo = FunctionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("reverseString", String.class)));

Review comment:
       `Lists.newArrayList(...)`

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -34,69 +35,72 @@
   @Test
   public void testExpressionWithColumn()
       throws Exception {
-    Method method = MyFunc.class.getDeclaredMethod("reverseString", String.class);
-    FunctionRegistry.registerStaticFunction(method);
-    FunctionInfo functionInfo = FunctionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("reverseString", String.class)));
+    FunctionInfo functionInfo = functionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
     System.out.println(functionInfo);
     String expression = "reverseString(testColumn)";
 
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertEquals(evaluator.getArguments(), Lists.newArrayList("testColumn"));
     GenericRow row = new GenericRow();
     for (int i = 0; i < 5; i++) {
       String value = "testValue" + i;
       row.putField("testColumn", value);
       Object result = evaluator.evaluate(row);
-      Assert.assertEquals(result, new StringBuilder(value).reverse().toString());
+      Assert.assertEquals(result, myFunc.reverseString(value));
     }
   }
 
   @Test
   public void testExpressionWithConstant()
       throws Exception {
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));

Review comment:
       `Lists.newArrayList(...)`




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419031711



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/FunctionRegistry.java
##########
@@ -18,36 +18,26 @@
  */
 package org.apache.pinot.core.data.function;
 
-import com.google.common.base.Preconditions;
 import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
  * Registry for inbuilt Pinot functions
  */
 public class FunctionRegistry {

Review comment:
       Thoughts on changing the name of this to `DefaultFunctionRegistry` now that it only applies to the DefaultFunctionEvaluator?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar merged pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar merged pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326


   


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r422548708



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -54,49 +54,73 @@ public void testExpressionWithColumn()
   @Test
   public void testExpressionWithConstant()
       throws Exception {
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Lists.newArrayList(myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch('%s', '%s')", input, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
   @Test
   public void testMultiFunctionExpression()
       throws Exception {
-    FunctionRegistry.registerStaticFunction(MyFunc.class.getDeclaredMethod("reverseString", String.class));
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(Lists
+        .newArrayList(myFunc.getClass().getDeclaredMethod("reverseString", String.class),
+            myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
-    String reversedInput = MyFunc.reverseString(input);
+    String reversedInput = myFunc.reverseString(input);
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch(reverseString('%s'), '%s')", reversedInput, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
-  private static class MyFunc {
-    static String reverseString(String input) {
-      return new StringBuilder(input).reverse().toString();
-    }
+  @Test
+  public void testStateSharedBetweenRowsForExecution()
+      throws Exception {

Review comment:
       This test basically just confirms that the internal state of the FunctionRegistry is shared between each row. I agree with the current implementation it's fairly self-explanatory, but you can imagine an implementation where the internals of the FunctionRegistry are different for each row. This is to make sure we don't somehow regress to that, because then we'd see a big performance hit for creating a SDF for each row.
   
   But can definitely remove if it just feels like clutter to you ☺️ 




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623588587


   > > Question: how would y'all test something like this locally? I ran the immediately relevant tests in IntelliJ, then triggered a `mvn test` (still running a while later). What command would y'all run to feel confident?
   > 
   > You could
   > 
   > 1. create a sample test file, and a schema with your transform functions
   > 2. create a segment w/ your schema and data
   > 3. read the segment using PinotSegmentRecordReader, verify values OR load the segment in a local setup and query
   
   Would also be nice to have an integration test, specifically designed to test transform functions. We could have groovy, static date time, non-static date time functions. This can be a followup 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



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r425424268



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DefaultFunctionRegistryFactory.java
##########
@@ -0,0 +1,70 @@
+/**
+ * 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.pinot.core.data.function;
+
+import com.google.common.collect.Lists;
+import java.lang.reflect.Method;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class DefaultFunctionRegistryFactory {
+  private static final Logger LOGGER = LoggerFactory.getLogger(DefaultFunctionRegistryFactory.class);
+
+  private DefaultFunctionRegistryFactory() {
+
+  }
+
+  public static DefaultFunctionRegistry getDefaultFunctionRegistry() {

Review comment:
       Now that there's also a JsonFunction function, won't mention where the functions come from.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419031828



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/FunctionRegistryFactory.java
##########
@@ -0,0 +1,37 @@
+package org.apache.pinot.core.data.function;
+
+import java.lang.reflect.Method;
+import java.util.Arrays;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Factory class to create a {@link FunctionRegistry}
+ */
+public class FunctionRegistryFactory {
+  private static final Logger LOGGER = LoggerFactory.getLogger(FunctionRegistryFactory.class);
+
+  private FunctionRegistryFactory() {
+
+  }
+
+  static List<Method> functionsToRegister;
+
+  static {
+    try {
+      functionsToRegister = Arrays.asList(DateTimeFunctions.class.getDeclaredMethod("toEpochHours", Long.class),
+          DateTimeFunctions.class.getDeclaredMethod("toEpochMinutes", Long.class, String.class));
+    } catch (NoSuchMethodException e) {
+      LOGGER.error("Caught exception when registering function", e);
+    }
+  }

Review comment:
       Blech. Any cleaner way to do this? Oh Java... haha




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] mcvsubbu commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
mcvsubbu commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419601234



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -27,6 +32,8 @@
  */
 public class DateTimeFunctions {
 
+  private Map<String, DateTimeFormatter> _dateTimeFormatterMap = new HashMap<>();

Review comment:
       It seems to me that this should be a concurrent hashmap. Consumers from different partitions may call this  across threads, am I right?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419565919



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -40,4 +47,14 @@ static Long toEpochHours(Long millis) {
   static Long toEpochMinutes(Long millis, String bucket) {
     return TimeUnit.MILLISECONDS.toMinutes(millis) / Integer.parseInt(bucket);
   }
+
+  DateTime toDateTime(String dateTimeString, String pattern) {
+    if (!_dateTimeFormatterMap.containsKey(pattern)) {

Review comment:
       use getOrDefault?

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -40,4 +47,14 @@ static Long toEpochHours(Long millis) {
   static Long toEpochMinutes(Long millis, String bucket) {
     return TimeUnit.MILLISECONDS.toMinutes(millis) / Integer.parseInt(bucket);
   }
+
+  DateTime toDateTime(String dateTimeString, String pattern) {
+    if (!_dateTimeFormatterMap.containsKey(pattern)) {
+      _dateTimeFormatterMap.put(pattern, DateTimeFormat.forPattern(pattern));
+    }
+
+    DateTimeFormatter dateTimeFormatter = _dateTimeFormatterMap.get(pattern);
+
+    return dateTimeFormatter.parseDateTime(dateTimeString);

Review comment:
       These functions will be applied during record transformation, and then the GenericRecord  is directly sent to the indexer
   RecordReader -> GenericRecord -> RecordTransformer -> GenericRecord -> Indexer.
   DateTime is not a datatype that Pinot can understand. Inputs and outputs from these transform functions should be STRING, INT, LONG.

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/FunctionRegistry.java
##########
@@ -18,36 +18,26 @@
  */
 package org.apache.pinot.core.data.function;
 
-import com.google.common.base.Preconditions;
 import java.lang.reflect.Method;
-import java.lang.reflect.Modifier;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 
 /**
  * Registry for inbuilt Pinot functions
  */
 public class FunctionRegistry {

Review comment:
       yes that works

##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluator.java
##########
@@ -50,11 +50,16 @@
 
   public DefaultFunctionEvaluator(String expression)
       throws Exception {
+    this(expression, FunctionRegistryFactory.getFunctionRegistry());

Review comment:
       An instance of `DefaultFunctionEvaluator` is created per transform function in the schema. We don't want to call getFunctionRegistry multiple times, we'll just end up creating many registries right?
   

##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -54,49 +54,73 @@ public void testExpressionWithColumn()
   @Test
   public void testExpressionWithConstant()
       throws Exception {
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Lists.newArrayList(myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch('%s', '%s')", input, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
   @Test
   public void testMultiFunctionExpression()
       throws Exception {
-    FunctionRegistry.registerStaticFunction(MyFunc.class.getDeclaredMethod("reverseString", String.class));
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(Lists
+        .newArrayList(myFunc.getClass().getDeclaredMethod("reverseString", String.class),
+            myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
-    String reversedInput = MyFunc.reverseString(input);
+    String reversedInput = myFunc.reverseString(input);
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch(reverseString('%s'), '%s')", reversedInput, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
-  private static class MyFunc {
-    static String reverseString(String input) {
-      return new StringBuilder(input).reverse().toString();
-    }
+  @Test
+  public void testStateSharedBetweenRowsForExecution()
+      throws Exception {

Review comment:
       i didn't understand why this test was added




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623586440


   Could you please also add to the SchemaUtilsTest, to make sure we're extracting arguments right.
   


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623587872


   > Question: how would y'all test something like this locally? I ran the immediately relevant tests in IntelliJ, then triggered a `mvn test` (still running a while later). What command would y'all run to feel confident?
   
   You could 
   1) create a sample test file, and a schema with your transform functions
   2) create a segment w/ your schema and data
   3) read the segment using PinotSegmentRecordReader, verify values OR load the segment in a local setup and query


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419724574



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -27,6 +32,8 @@
  */
 public class DateTimeFunctions {
 
+  private Map<String, DateTimeFormatter> _dateTimeFormatterMap = new HashMap<>();

Review comment:
       While this is working great for dateTimeFormat functions,  I wonder if we need to instead have multiple constructors (or init methods), and create DateTimeFunction object for every function we encounter in the schema.
   For example, say we would want to add `round(millis, 15:MINUTES)`. The logic for this is (millis / (15 * 60 * 1000)) * (15* 60 * 1000). We don't want to interpret "15:MINUTES" on every row. Instead, we can create a DateTimeFunction that saves "15:MINUTES" as a member variable, and then invoke `round()` on this object.
   We can have init methods like
   initBucket(int bucket), initRounding(int rounding), initSDF(String pattern), etc.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r422550932



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluator.java
##########
@@ -50,11 +50,16 @@
 
   public DefaultFunctionEvaluator(String expression)
       throws Exception {
+    this(expression, FunctionRegistryFactory.getFunctionRegistry());

Review comment:
       I think I'm creating individual `DefaultFunctionRegistry`s in the right place now, but still not 100% sure. Let me know - thanks! :)




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r422564834



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DateTimeFunctionEvaluatorTest.java
##########
@@ -52,12 +54,14 @@ public void testDateTimeTransformFunctions(String transformFunction, List<String
     // toEpochSeconds w/ rounding
     GenericRow row1_1 = new GenericRow();
     row1_1.putValue("timestamp", 1578685189000L);
-    inputs.add(new Object[]{"toEpochSecondsRounded(timestamp, 10)", Lists.newArrayList("timestamp"), row1_1, 1578685180L});
+    inputs.add(
+        new Object[]{"toEpochSecondsRounded(timestamp, 10)", Lists.newArrayList("timestamp"), row1_1, 1578685180L});

Review comment:
       Ended up getting a good number of formatting differences when I auto-formatted this file. Is my setup wrong or do y'all get these changes when formatting too?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] codecov-io commented on pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
codecov-io commented on pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#issuecomment-623055433


   # [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=h1) Report
   > Merging [#5326](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=desc) into [master](https://codecov.io/gh/apache/incubator-pinot/commit/c87be271bfe4c923b59f4e68b85a51e518112e54&el=desc) will **decrease** coverage by `0.03%`.
   > The diff coverage is `78.78%`.
   
   [![Impacted file tree graph](https://codecov.io/gh/apache/incubator-pinot/pull/5326/graphs/tree.svg?width=650&height=150&src=pr&token=4ibza2ugkz)](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=tree)
   
   ```diff
   @@            Coverage Diff             @@
   ##           master    #5326      +/-   ##
   ==========================================
   - Coverage   66.09%   66.06%   -0.04%     
   ==========================================
     Files        1072     1073       +1     
     Lines       54552    54573      +21     
     Branches     8137     8138       +1     
   ==========================================
   - Hits        36058    36054       -4     
   - Misses      15858    15876      +18     
   - Partials     2636     2643       +7     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=tree) | Coverage Δ | |
   |---|---|---|
   | [...ot/core/data/function/FunctionRegistryFactory.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL2Z1bmN0aW9uL0Z1bmN0aW9uUmVnaXN0cnlGYWN0b3J5LmphdmE=) | `66.66% <66.66%> (ø)` | |
   | [...he/pinot/core/data/function/DateTimeFunctions.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL2Z1bmN0aW9uL0RhdGVUaW1lRnVuY3Rpb25zLmphdmE=) | `87.50% <80.00%> (+20.83%)` | :arrow_up: |
   | [...t/core/data/function/DefaultFunctionEvaluator.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL2Z1bmN0aW9uL0RlZmF1bHRGdW5jdGlvbkV2YWx1YXRvci5qYXZh) | `95.91% <100.00%> (+0.17%)` | :arrow_up: |
   | [...che/pinot/core/data/function/FunctionRegistry.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL2Z1bmN0aW9uL0Z1bmN0aW9uUmVnaXN0cnkuamF2YQ==) | `80.00% <100.00%> (+9.16%)` | :arrow_up: |
   | [...a/manager/realtime/RealtimeSegmentDataManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9kYXRhL21hbmFnZXIvcmVhbHRpbWUvUmVhbHRpbWVTZWdtZW50RGF0YU1hbmFnZXIuamF2YQ==) | `50.00% <0.00%> (-25.00%)` | :arrow_down: |
   | [...e/impl/dictionary/LongOnHeapMutableDictionary.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yZWFsdGltZS9pbXBsL2RpY3Rpb25hcnkvTG9uZ09uSGVhcE11dGFibGVEaWN0aW9uYXJ5LmphdmE=) | `64.63% <0.00%> (-12.20%)` | :arrow_down: |
   | [...r/dociditerators/RangelessBitmapDocIdIterator.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9vcGVyYXRvci9kb2NpZGl0ZXJhdG9ycy9SYW5nZWxlc3NCaXRtYXBEb2NJZEl0ZXJhdG9yLmphdmE=) | `72.22% <0.00%> (-11.12%)` | :arrow_down: |
   | [...impl/dictionary/FloatOffHeapMutableDictionary.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yZWFsdGltZS9pbXBsL2RpY3Rpb25hcnkvRmxvYXRPZmZIZWFwTXV0YWJsZURpY3Rpb25hcnkuamF2YQ==) | `64.51% <0.00%> (-7.53%)` | :arrow_down: |
   | [.../realtime/impl/ThreadSafeMutableRoaringBitmap.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29yZS9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29yZS9yZWFsdGltZS9pbXBsL1RocmVhZFNhZmVNdXRhYmxlUm9hcmluZ0JpdG1hcC5qYXZh) | `92.85% <0.00%> (-7.15%)` | :arrow_down: |
   | [...lix/core/realtime/PinotRealtimeSegmentManager.java](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree#diff-cGlub3QtY29udHJvbGxlci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvY29udHJvbGxlci9oZWxpeC9jb3JlL3JlYWx0aW1lL1Bpbm90UmVhbHRpbWVTZWdtZW50TWFuYWdlci5qYXZh) | `78.97% <0.00%> (-4.62%)` | :arrow_down: |
   | ... and [15 more](https://codecov.io/gh/apache/incubator-pinot/pull/5326/diff?src=pr&el=tree-more) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=continue).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=footer). Last update [c87be27...0e551b2](https://codecov.io/gh/apache/incubator-pinot/pull/5326?src=pr&el=lastupdated). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments).
   


----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] reallocf commented on a change in pull request #5326: Update FunctionRegistry to avoid static initialization (#5313)

Posted by GitBox <gi...@apache.org>.
reallocf commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419032029



##########
File path: pinot-core/src/test/java/org/apache/pinot/core/data/function/DefaultFunctionEvaluatorTest.java
##########
@@ -34,69 +35,72 @@
   @Test
   public void testExpressionWithColumn()
       throws Exception {
-    Method method = MyFunc.class.getDeclaredMethod("reverseString", String.class);
-    FunctionRegistry.registerStaticFunction(method);
-    FunctionInfo functionInfo = FunctionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("reverseString", String.class)));
+    FunctionInfo functionInfo = functionRegistry.resolve("reverseString", new Class<?>[]{Object.class});
     System.out.println(functionInfo);
     String expression = "reverseString(testColumn)";
 
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertEquals(evaluator.getArguments(), Lists.newArrayList("testColumn"));
     GenericRow row = new GenericRow();
     for (int i = 0; i < 5; i++) {
       String value = "testValue" + i;
       row.putField("testColumn", value);
       Object result = evaluator.evaluate(row);
-      Assert.assertEquals(result, new StringBuilder(value).reverse().toString());
+      Assert.assertEquals(result, myFunc.reverseString(value));
     }
   }
 
   @Test
   public void testExpressionWithConstant()
       throws Exception {
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(
+        Collections.singletonList(myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch('%s', '%s')", input, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
 
   @Test
   public void testMultiFunctionExpression()
       throws Exception {
-    FunctionRegistry.registerStaticFunction(MyFunc.class.getDeclaredMethod("reverseString", String.class));
-    FunctionRegistry
-        .registerStaticFunction(MyFunc.class.getDeclaredMethod("daysSinceEpoch", String.class, String.class));
+    MyFunc myFunc = new MyFunc();
+    FunctionRegistry functionRegistry = new FunctionRegistry(Arrays
+        .asList(myFunc.getClass().getDeclaredMethod("reverseString", String.class),
+            myFunc.getClass().getDeclaredMethod("daysSinceEpoch", String.class, String.class)));
     String input = "1980-01-01";
-    String reversedInput = MyFunc.reverseString(input);
+    String reversedInput = myFunc.reverseString(input);
     String format = "yyyy-MM-dd";
     String expression = String.format("daysSinceEpoch(reverseString('%s'), '%s')", reversedInput, format);
-    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression);
+    DefaultFunctionEvaluator evaluator = new DefaultFunctionEvaluator(expression, functionRegistry);
     Assert.assertTrue(evaluator.getArguments().isEmpty());
     GenericRow row = new GenericRow();
     Object result = evaluator.evaluate(row);
-    Assert.assertEquals(result, MyFunc.daysSinceEpoch(input, format));
+    Assert.assertEquals(result, myFunc.daysSinceEpoch(input, format));
   }
+}
 
-  private static class MyFunc {
-    static String reverseString(String input) {
-      return new StringBuilder(input).reverse().toString();
-    }
+class MyFunc {

Review comment:
       Had to pull this out into its own function (as opposed to a private function) in order to allow for the `clazz.newInstance()` call on line 56 of FunctionInvoker. Should I move this to a separate file?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419603427



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -40,4 +47,14 @@ static Long toEpochHours(Long millis) {
   static Long toEpochMinutes(Long millis, String bucket) {
     return TimeUnit.MILLISECONDS.toMinutes(millis) / Integer.parseInt(bucket);
   }
+
+  DateTime toDateTime(String dateTimeString, String pattern) {
+    if (!_dateTimeFormatterMap.containsKey(pattern)) {
+      _dateTimeFormatterMap.put(pattern, DateTimeFormat.forPattern(pattern));
+    }
+
+    DateTimeFormatter dateTimeFormatter = _dateTimeFormatterMap.get(pattern);
+
+    return dateTimeFormatter.parseDateTime(dateTimeString);

Review comment:
       I guess this answers your question about fromDateTime?




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org


[GitHub] [incubator-pinot] npawar commented on a change in pull request #5326: Add toDateTime DateTimeFunction (#5313)

Posted by GitBox <gi...@apache.org>.
npawar commented on a change in pull request #5326:
URL: https://github.com/apache/incubator-pinot/pull/5326#discussion_r419724574



##########
File path: pinot-core/src/main/java/org/apache/pinot/core/data/function/DateTimeFunctions.java
##########
@@ -27,6 +32,8 @@
  */
 public class DateTimeFunctions {
 
+  private Map<String, DateTimeFormatter> _dateTimeFormatterMap = new HashMap<>();

Review comment:
       While this is working great for dateTimeFormat functions,  I wonder if we need to instead have multiple constructors (or init methods), and create DateTimeFunction object for every function we encounter in the schema.
   For example, say we would want to add `round(millis, 15:MINUTES)`. The logic for this is (millis / (15*60*1000)) * (15*60*1000). We don't want to interpret "15:MINUTES" on every row. Instead, we can create a DateTimeFunction that saves "15:MINUTES" as a member variable, and then invoke `round()` on this object.
   We can have init methods like
   initBucket(int bucket), initRounding(int rounding), initSDF(String pattern), etc.




----------------------------------------------------------------
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: commits-unsubscribe@pinot.apache.org
For additional commands, e-mail: commits-help@pinot.apache.org