You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by "walterddr (via GitHub)" <gi...@apache.org> on 2023/12/15 18:05:03 UTC

[PR] [POC] function registry unification [pinot]

walterddr opened a new pull request, #12163:
URL: https://github.com/apache/pinot/pull/12163

   working POC


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [multistage][feature] function registry unification [pinot]

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #12163:
URL: https://github.com/apache/pinot/pull/12163#discussion_r1442370706


##########
pinot-common/src/main/java/org/apache/pinot/common/function/sql/PinotFunctionRegistry.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.common.function.sql;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.NameMultimap;
+import org.apache.pinot.common.function.TransformFunctionType;
+import org.apache.pinot.common.function.registry.PinotFunction;
+import org.apache.pinot.common.function.registry.PinotScalarFunction;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+import org.apache.pinot.spi.utils.PinotReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Registry for scalar functions.
+ */
+public class PinotFunctionRegistry {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PinotFunctionRegistry.class);
+  private static final NameMultimap<SqlOperator> OPERATOR_MAP = new NameMultimap<>();
+  private static final NameMultimap<PinotFunction> FUNCTION_MAP = new NameMultimap<>();
+
+  private PinotFunctionRegistry() {
+  }
+
+  /**
+   * Registers the scalar functions via reflection.
+   * NOTE: In order to plugin methods using reflection, the methods should be inside a class that includes ".function."
+   *       in its class path. This convention can significantly reduce the time of class scanning.
+   */
+  static {
+    // REGISTER FUNCTIONS
+    long startTimeMs = System.currentTimeMillis();
+    Set<Method> methods = PinotReflectionUtils.getMethodsThroughReflection(".*\\.function\\..*", ScalarFunction.class);
+    for (Method method : methods) {
+      if (!Modifier.isPublic(method.getModifiers())) {
+        continue;
+      }
+      ScalarFunction scalarFunction = method.getAnnotation(ScalarFunction.class);
+      if (scalarFunction.enabled()) {
+        // Parse annotated function names and alias
+        Set<String> scalarFunctionNames = Arrays.stream(scalarFunction.names()).collect(Collectors.toSet());
+        if (scalarFunctionNames.size() == 0) {
+          scalarFunctionNames.add(method.getName());
+        }
+        boolean nullableParameters = scalarFunction.nullableParameters();
+        PinotFunctionRegistry.registerFunction(method, scalarFunctionNames, nullableParameters);
+      }
+    }
+    LOGGER.info("Initialized FunctionRegistry with {} functions: {} in {}ms", FUNCTION_MAP.map().size(),
+        FUNCTION_MAP.map().keySet(), System.currentTimeMillis() - startTimeMs);
+
+    // REGISTER OPERATORS

Review Comment:
   Note to reviewer: 
   this part will be maintain until all Transform function are registered in scalar form via PinotScalarFunction



##########
pinot-common/src/main/java/org/apache/pinot/common/function/sql/PinotFunctionRegistry.java:
##########
@@ -0,0 +1,237 @@
+/**
+ * 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.common.function.sql;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Locale;
+import java.util.Set;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.SqlUtil;
+import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
+import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
+import org.apache.calcite.util.NameMultimap;
+import org.apache.pinot.common.function.TransformFunctionType;
+import org.apache.pinot.common.function.registry.PinotFunction;
+import org.apache.pinot.common.function.registry.PinotScalarFunction;
+import org.apache.pinot.common.utils.DataSchema;
+import org.apache.pinot.segment.spi.AggregationFunctionType;
+import org.apache.pinot.spi.annotations.ScalarFunction;
+import org.apache.pinot.spi.utils.PinotReflectionUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * Registry for scalar functions.
+ */
+public class PinotFunctionRegistry {
+  private static final Logger LOGGER = LoggerFactory.getLogger(PinotFunctionRegistry.class);
+  private static final NameMultimap<SqlOperator> OPERATOR_MAP = new NameMultimap<>();
+  private static final NameMultimap<PinotFunction> FUNCTION_MAP = new NameMultimap<>();
+
+  private PinotFunctionRegistry() {
+  }
+
+  /**
+   * Registers the scalar functions via reflection.
+   * NOTE: In order to plugin methods using reflection, the methods should be inside a class that includes ".function."
+   *       in its class path. This convention can significantly reduce the time of class scanning.
+   */
+  static {
+    // REGISTER FUNCTIONS
+    long startTimeMs = System.currentTimeMillis();
+    Set<Method> methods = PinotReflectionUtils.getMethodsThroughReflection(".*\\.function\\..*", ScalarFunction.class);
+    for (Method method : methods) {
+      if (!Modifier.isPublic(method.getModifiers())) {
+        continue;
+      }
+      ScalarFunction scalarFunction = method.getAnnotation(ScalarFunction.class);
+      if (scalarFunction.enabled()) {
+        // Parse annotated function names and alias
+        Set<String> scalarFunctionNames = Arrays.stream(scalarFunction.names()).collect(Collectors.toSet());
+        if (scalarFunctionNames.size() == 0) {
+          scalarFunctionNames.add(method.getName());
+        }
+        boolean nullableParameters = scalarFunction.nullableParameters();
+        PinotFunctionRegistry.registerFunction(method, scalarFunctionNames, nullableParameters);
+      }
+    }
+    LOGGER.info("Initialized FunctionRegistry with {} functions: {} in {}ms", FUNCTION_MAP.map().size(),
+        FUNCTION_MAP.map().keySet(), System.currentTimeMillis() - startTimeMs);
+
+    // REGISTER OPERATORS
+    // Walk through all the Pinot aggregation types and
+    //   1. register those that are supported in multistage in addition to calcite standard opt table.
+    //   2. register special handling that differs from calcite standard.
+    for (AggregationFunctionType aggregationFunctionType : AggregationFunctionType.values()) {
+      if (aggregationFunctionType.getSqlKind() != null) {
+        // 1. Register the aggregation function with Calcite
+        registerAggregateFunction(aggregationFunctionType.getName(), aggregationFunctionType);
+        // 2. Register the aggregation function with Calcite on all alternative names
+        List<String> alternativeFunctionNames = aggregationFunctionType.getAlternativeNames();
+        for (String alternativeFunctionName : alternativeFunctionNames) {
+          registerAggregateFunction(alternativeFunctionName, aggregationFunctionType);
+        }
+      }
+    }
+
+    // Walk through all the Pinot transform types and
+    //   1. register those that are supported in multistage in addition to calcite standard opt table.
+    //   2. register special handling that differs from calcite standard.
+    for (TransformFunctionType transformFunctionType : TransformFunctionType.values()) {
+      if (transformFunctionType.getSqlKind() != null) {
+        // 1. Register the transform function with Calcite
+        registerTransformFunction(transformFunctionType.getName(), transformFunctionType);
+        // 2. Register the transform function with Calcite on all alternative names
+        List<String> alternativeFunctionNames = transformFunctionType.getAlternativeNames();
+        for (String alternativeFunctionName : alternativeFunctionNames) {
+          registerTransformFunction(alternativeFunctionName, transformFunctionType);
+        }
+      }
+    }
+  }
+
+  public static void init() {
+  }
+
+  @VisibleForTesting
+  public static void registerFunction(Method method, boolean nullableParameters) {
+    registerFunction(method, Collections.singleton(method.getName()), nullableParameters);
+  }
+
+  public static NameMultimap<PinotFunction> getFunctionMap() {
+    return FUNCTION_MAP;
+  }
+
+  public static NameMultimap<SqlOperator> getOperatorMap() {
+    return OPERATOR_MAP;
+  }
+
+  @Nullable
+  public static PinotScalarFunction getScalarFunction(SqlOperatorTable operatorTable, RelDataTypeFactory typeFactory,
+      String functionName, List<DataSchema.ColumnDataType> argTypes) {
+    List<RelDataType> relArgTypes = convertArgumentTypes(typeFactory, argTypes);
+    SqlOperator sqlOperator = SqlUtil.lookupRoutine(operatorTable, typeFactory,
+        new SqlIdentifier(functionName, SqlParserPos.QUOTED_ZERO), relArgTypes, null, null, SqlSyntax.FUNCTION,
+        SqlKind.OTHER_FUNCTION, SqlNameMatchers.withCaseSensitive(false), true);
+    if (sqlOperator instanceof SqlUserDefinedFunction) {
+      Function function = ((SqlUserDefinedFunction) sqlOperator).getFunction();
+      if (function instanceof PinotScalarFunction) {
+        return (PinotScalarFunction) function;
+      }
+    }
+    return null;
+  }
+
+  private static void registerFunction(Method method, Set<String> alias, boolean nullableParameters) {
+    if (method.getAnnotation(Deprecated.class) == null) {
+      for (String name : alias) {
+        registerCalciteNamedFunctionMap(name, method, nullableParameters);
+      }
+    }
+  }
+
+  private static void registerCalciteNamedFunctionMap(String name, Method method, boolean nullableParameters) {
+    FUNCTION_MAP.put(name, new PinotScalarFunction(name, method, nullableParameters));
+  }
+
+  private static List<RelDataType> convertArgumentTypes(RelDataTypeFactory typeFactory,
+      List<DataSchema.ColumnDataType> argTypes) {
+    return argTypes.stream().map(type -> toRelType(typeFactory, type)).collect(Collectors.toList());
+  }
+
+  private static RelDataType toRelType(RelDataTypeFactory typeFactory, DataSchema.ColumnDataType dataType) {

Review Comment:
   Note to reviewer: 
   this and several other type conversion utils should be unified, we will follow up in separate PRs



##########
pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java:
##########
@@ -38,19 +42,14 @@
 
 /**
  * Registry for scalar functions.
- * <p>TODO: Merge FunctionRegistry and FunctionDefinitionRegistry to provide one single registry for all functions.
  */
 public class FunctionRegistry {

Review Comment:
   Note to reviewer:  this class will be maintain until
   1. all Transform/Scalar functions are unified
   2. all legacy, arg-count based function resolution logics are deprecated



##########
pinot-common/src/main/java/org/apache/pinot/common/function/sql/PinotOperatorTable.java:
##########
@@ -0,0 +1,101 @@
+/**
+ * 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.common.function.sql;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
+import org.apache.pinot.common.function.FunctionRegistry;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+
+/**
+ * Temporary implementation of all dynamic arg/return type inference operators.

Review Comment:
   This will be maintained until all operators can be merged into catalog



##########
pinot-query-planner/src/main/java/org/apache/calcite/sql/util/PinotSqlStdOperatorTable.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.calcite.sql.util;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.sql.PinotSqlCoalesceFunction;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
+import org.apache.calcite.util.Util;
+import org.apache.pinot.common.function.FunctionRegistry;
+
+
+public class PinotSqlStdOperatorTable extends SqlStdOperatorTable {

Review Comment:
   this class now only handles overries against SqlStdOperatorTable. it doesn't handle ANY Pinot special operators/functions



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [poc][multistage][feature] function registry unification [pinot]

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on PR #12163:
URL: https://github.com/apache/pinot/pull/12163#issuecomment-1879132975

   closing in favor of #12229


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [poc][multistage][feature] function registry unification [pinot]

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr closed pull request #12163: [poc][multistage][feature] function registry unification
URL: https://github.com/apache/pinot/pull/12163


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [multistage][feature] function registry unification [pinot]

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on code in PR #12163:
URL: https://github.com/apache/pinot/pull/12163#discussion_r1442371361


##########
pinot-common/src/main/java/org/apache/pinot/common/function/sql/PinotOperatorTable.java:
##########
@@ -0,0 +1,101 @@
+/**
+ * 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.common.function.sql;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorTable;
+import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
+import org.apache.pinot.common.function.FunctionRegistry;
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+
+/**
+ * Temporary implementation of all dynamic arg/return type inference operators.

Review Comment:
   Note to reviewer:
   This will be maintained until all operators can be merged into catalog



##########
pinot-query-planner/src/main/java/org/apache/calcite/sql/util/PinotSqlStdOperatorTable.java:
##########
@@ -0,0 +1,98 @@
+/**
+ * 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.calcite.sql.util;
+
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.calcite.sql.PinotSqlCoalesceFunction;
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
+import org.apache.calcite.util.Util;
+import org.apache.pinot.common.function.FunctionRegistry;
+
+
+public class PinotSqlStdOperatorTable extends SqlStdOperatorTable {

Review Comment:
   Note to reviewer:
   this class now only handles overries against SqlStdOperatorTable. it doesn't handle ANY Pinot special operators/functions



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [POC] function registry unification [pinot]

Posted by "walterddr (via GitHub)" <gi...@apache.org>.
walterddr commented on PR #12163:
URL: https://github.com/apache/pinot/pull/12163#issuecomment-1861248100

   - Follow-up POC for unified polymorphism: https://github.com/walterddr/pinot/pull/91
   - Follow-up POC for parameter-type-based lookup: https://github.com/walterddr/pinot/pull/91 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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


Re: [PR] [POC] function registry unification [pinot]

Posted by "codecov-commenter (via GitHub)" <gi...@apache.org>.
codecov-commenter commented on PR #12163:
URL: https://github.com/apache/pinot/pull/12163#issuecomment-1861089527

   ## [Codecov](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) Report
   Attention: `154 lines` in your changes are missing coverage. Please review.
   > Comparison is base [(`56ecafc`)](https://app.codecov.io/gh/apache/pinot/commit/56ecafc17e141b4cd9404255452b654231cc1247?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 61.60% compared to head [(`f15c394`)](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) 27.61%.
   > Report is 21 commits behind head on master.
   
   > :exclamation: Current head f15c394 differs from pull request most recent head 5f65e91. Consider uploading reports for the commit 5f65e91 to get more accurate results
   
   | [Files](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Patch % | Lines |
   |---|---|---|
   | [...not/common/function/sql/PinotFunctionRegistry.java](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc3FsL1Bpbm90RnVuY3Rpb25SZWdpc3RyeS5qYXZh) | 0.00% | [86 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [...apache/pinot/common/function/FunctionRegistry.java](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vRnVuY3Rpb25SZWdpc3RyeS5qYXZh) | 0.00% | [27 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [.../pinot/common/function/sql/PinotOperatorTable.java](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vc3FsL1Bpbm90T3BlcmF0b3JUYWJsZS5qYXZh) | 0.00% | [25 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [.../common/function/registry/PinotScalarFunction.java](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtY29tbW9uL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9waW5vdC9jb21tb24vZnVuY3Rpb24vcmVnaXN0cnkvUGlub3RTY2FsYXJGdW5jdGlvbi5qYXZh) | 0.00% | [15 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   | [.../java/org/apache/pinot/query/QueryEnvironment.java](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#diff-cGlub3QtcXVlcnktcGxhbm5lci9zcmMvbWFpbi9qYXZhL29yZy9hcGFjaGUvcGlub3QvcXVlcnkvUXVlcnlFbnZpcm9ubWVudC5qYXZh) | 0.00% | [1 Missing :warning: ](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) |
   
   <details><summary>Additional details and impacted files</summary>
   
   
   ```diff
   @@              Coverage Diff              @@
   ##             master   #12163       +/-   ##
   =============================================
   - Coverage     61.60%   27.61%   -33.99%     
   + Complexity     1152      207      -945     
   =============================================
     Files          2406     2413        +7     
     Lines        130859   131246      +387     
     Branches      20218    20282       +64     
   =============================================
   - Hits          80611    36246    -44365     
   - Misses        44373    92261    +47888     
   + Partials       5875     2739     -3136     
   ```
   
   | [Flag](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flags&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | Coverage Δ | |
   |---|---|---|
   | [custom-integration1](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration1](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-0.01%)` | :arrow_down: |
   | [integration2](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `0.00% <0.00%> (ø)` | |
   | [java-11](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `<0.01% <0.00%> (-61.56%)` | :arrow_down: |
   | [java-21](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-33.86%)` | :arrow_down: |
   | [skip-bytebuffers-false](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-33.97%)` | :arrow_down: |
   | [skip-bytebuffers-true](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-33.85%)` | :arrow_down: |
   | [temurin](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-33.99%)` | :arrow_down: |
   | [unittests](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-33.99%)` | :arrow_down: |
   | [unittests1](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `?` | |
   | [unittests2](https://app.codecov.io/gh/apache/pinot/pull/12163/flags?src=pr&el=flag&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache) | `27.61% <0.00%> (-0.04%)` | :arrow_down: |
   
   Flags with carried forward coverage won't be shown. [Click here](https://docs.codecov.io/docs/carryforward-flags?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache#carryforward-flags-in-the-pull-request-comment) to find out more.
   
   
   </details>
   
   [:umbrella: View full report in Codecov by Sentry](https://app.codecov.io/gh/apache/pinot/pull/12163?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).   
   :loudspeaker: Have feedback on the report? [Share it here](https://about.codecov.io/codecov-pr-comment-feedback/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=apache).
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pinot.apache.org

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