You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pinot.apache.org by ro...@apache.org on 2023/07/27 17:36:00 UTC

[pinot] branch master updated: dont duplicate register scalar function in CalciteSchema (#11190)

This is an automated email from the ASF dual-hosted git repository.

rongr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 4a12ebf2df dont duplicate register scalar function in CalciteSchema (#11190)
4a12ebf2df is described below

commit 4a12ebf2dfee0800c7aa88a90840564de00a549a
Author: Rong Rong <ro...@apache.org>
AuthorDate: Thu Jul 27 10:35:55 2023 -0700

    dont duplicate register scalar function in CalciteSchema (#11190)
    
    Schema (PinotCatalog) and FunctionMap (CalciteSchema.SchemaPlus) both registers functions from FunctionRegistry. previously we registered in both, thus causes confusion. the solution is to only do it in SchemaPlus b/c it handles case sensitivity
    
    this fixes: #11189
    
    Co-authored-by: Rong Rong <ro...@startree.ai>
---
 .../org/apache/pinot/common/function/FunctionRegistry.java  |  5 -----
 .../java/org/apache/pinot/query/catalog/PinotCatalog.java   | 13 ++++++++++---
 .../src/test/resources/queries/SpecialSyntax.json           |  8 ++++++++
 3 files changed, 18 insertions(+), 8 deletions(-)

diff --git a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
index b6edfdec07..e4c3b251de 100644
--- a/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
+++ b/pinot-common/src/main/java/org/apache/pinot/common/function/FunctionRegistry.java
@@ -21,7 +21,6 @@ package org.apache.pinot.common.function;
 import com.google.common.base.Preconditions;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
-import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -128,10 +127,6 @@ public class FunctionRegistry {
     return FUNCTION_MAP.map();
   }
 
-  public static Collection<Function> getRegisteredCalciteFunctions(String name) {
-    return FUNCTION_MAP.map().get(name);
-  }
-
   public static Set<String> getRegisteredCalciteFunctionNames() {
     return FUNCTION_MAP.map().keySet();
   }
diff --git a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java
index 0505b876fa..9b6b1c657a 100644
--- a/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java
+++ b/pinot-query-planner/src/main/java/org/apache/pinot/query/catalog/PinotCatalog.java
@@ -31,7 +31,6 @@ import org.apache.calcite.schema.SchemaVersion;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.schema.Table;
 import org.apache.pinot.common.config.provider.TableCache;
-import org.apache.pinot.common.function.FunctionRegistry;
 import org.apache.pinot.spi.utils.builder.TableNameBuilder;
 
 import static java.util.Objects.requireNonNull;
@@ -92,14 +91,22 @@ public class PinotCatalog implements Schema {
     return Collections.emptySet();
   }
 
+  /**
+   * {@code PinotCatalog} doesn't need to return function collections b/c they are already registered.
+   * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema)}
+   */
   @Override
   public Collection<Function> getFunctions(String name) {
-    return FunctionRegistry.getRegisteredCalciteFunctions(name);
+    return Collections.emptyList();
   }
 
+  /**
+   * {@code PinotCatalog} doesn't need to return function name set b/c they are already registered.
+   * see: {@link org.apache.calcite.jdbc.CalciteSchemaBuilder#asRootSchema(Schema)}
+   */
   @Override
   public Set<String> getFunctionNames() {
-    return FunctionRegistry.getRegisteredCalciteFunctionNames();
+    return Collections.emptySet();
   }
 
   @Override
diff --git a/pinot-query-runtime/src/test/resources/queries/SpecialSyntax.json b/pinot-query-runtime/src/test/resources/queries/SpecialSyntax.json
index f312015b41..30fd01455f 100644
--- a/pinot-query-runtime/src/test/resources/queries/SpecialSyntax.json
+++ b/pinot-query-runtime/src/test/resources/queries/SpecialSyntax.json
@@ -40,6 +40,14 @@
           ["foo", 4]
         ]
       },
+      {
+        "description": "test scalar function with STD SQL operator, and scalar function exactly match should be found properly",
+        "sql": "SELECT upper(col1), roundDecimal(col3_r), add(plus(CAST(col3_l AS DOUBLE), cast(col3_r AS DOUBLE)), cast(10 AS DOUBLE)) FROM (SELECT {tbl1}.col1 AS col1, {tbl1}.col3 AS col3_l, {tbl2}.col3 AS col3_r FROM {tbl1} JOIN {tbl2} USING (col2))",
+        "outputs": [
+          ["BAR", 3, 15.0],
+          ["FOO", 4, 15.0]
+        ]
+      },
       {
         "description": "test scalar function with STD SQL operator, and scalar function without STD SQL operator can be found properly",
         "sql": "SELECT UpPeR(col1), round_deCiMal(col3_r), aDD(pluS(CAST(col3_l AS DOUBLE), CAST(col3_r AS DOUBLE)), CAST(10 AS DOUBLE)) FROM (SELECT {tbl1}.col1 AS col1, {tbl1}.col3 AS col3_l, {tbl2}.col3 AS col3_r FROM {tbl1} JOIN {tbl2} USING (col2))",


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