You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by hsuanyi <gi...@git.apache.org> on 2016/02/29 08:04:12 UTC

[GitHub] drill pull request: Drill 4372 review

GitHub user hsuanyi opened a pull request:

    https://github.com/apache/drill/pull/397

    Drill 4372 review

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/hsuanyi/incubator-drill DRILL-4372_Review

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/397.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #397
    
----
commit 4c75fd6ae22e26dcac37d5da1f41020d25f885d8
Author: Sudheesh Katkam <sk...@maprtech.com>
Date:   2015-12-22T04:38:59Z

    Validate Drill functions (argument and return types). WIP.

commit 4aa34bf41982a4491f6eea347165436fe9dfd853
Author: Hsuan-Yi Chu <hs...@usc.edu>
Date:   2015-12-30T22:21:10Z

    DRILL-4372: Expose the functions return type to Drill
    
    - Drill-Calite version update:
    This commit needs to have Calcite's patch (CALCITE-1062) to plugin customized SqlOperator.
    
    - FunctionTemplate
    Add FunctionArgumentNumber annotation. This annotation element tells if the number of argument(s) is fixed or arbitrary (e.g., String concatenation function).
    
    Due to this modification, there are some minor changes in DrillFuncHolder, DrillFunctionRegistry and FunctionAttributes.
    
    - Checker
    Add a new Checker (which Calcite uses to validate the legitimacy of the number of argument(s) for a function) to allow functions with arbitrary arguments to pass Caclite's validation
    
    - Type conversion between Drill and Calcite
    DrillConstExector is given a static method getDrillTypeFromCalcite() to convert Calcite types to Drill's.
    
    - Extract function's return type inference
    Unlike other functions, Extract function's return type can be determined solely based on the first argument. A logic is added in to allow this inference to happen
    
    - DrillCalcite wrapper:
    From the aspects of return type inference and argument type checks, Calcite's mechanism is very different from Drill's. In addition, currently, there is no straightforward way for Drill to plug-in customized mechanisms to Calcite. Thus, wrappers are provided to serve the objective.
    
    Except for the mechanisms of type inference and argument type checks, these wrappers just forward any method calls to the wrapped SqlOpertor, SqlFuncion or SqlAggFunction to respond.
    
    A interface DrillCalciteSqlWrapper is also added for the callers of the three wrappers to get the wrapped objects easier.
    
    Due to these wrappers, UnsupportedOperatorsVisitor is modified in a minor manner.
    
    - Calcite's SqlOpertor, SqlFuncion or SqlAggFunction are wrapped in DrillOperatorTable
    Instead of returning Caclite's native SqlOpertor, SqlFuncion or SqlAggFunction, return the wrapped ones to ensure customized behaviors can be adopted.
    
    - Type inference mechanism
    This mechanism is used across all SqlOpertor, SqlFuncion or SqlAggFunction. Thus, it is factored out as its own method in TypeInferenceUtils
    
    - Upgrade Drill-Calcite
    
    Bump version number to 1.4.0-drill-test-r15

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56528868
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
     
       public DrillOperatorTable(FunctionImplementationRegistry registry) {
    -    operators = Lists.newArrayList();
    -    operators.addAll(inner.getOperatorList());
    +    this(registry, null);
    +  }
     
    +  public DrillOperatorTable(FunctionImplementationRegistry registry, SystemOptionManager systemOptionManager) {
         registry.register(this);
    +    operatorsCalcite.addAll(inner.getOperatorList());
    +    populateWrappedCalciteOperators();
    +    this.systemOptionManager = systemOptionManager;
    +  }
    +
    +  public void addDefault(String name, SqlOperator op) {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56456667
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    --- End diff --
    
    How about calciteOperators?
    
    Also, I meant it for all lists and maps here.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922555
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +112,58 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange;
    --- End diff --
    
    argNumberRange


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922761
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java ---
    @@ -100,8 +108,13 @@ public void onMatch(RelOptRuleCall ruleCall) {
        */
       private boolean containsAvgStddevVarCall(List<AggregateCall> aggCallList) {
         for (AggregateCall call : aggCallList) {
    -      if (call.getAggregation() instanceof SqlAvgAggFunction
    -          || call.getAggregation() instanceof SqlSumAggFunction) {
    +      SqlAggFunction sqlAggFunction = call.getAggregation();
    +      if(sqlAggFunction instanceof DrillCalciteSqlWrapper) {
    --- End diff --
    
    There are a bunch of unwrap calls like this. Is there a way to avoid this? Otherwise, devs need to aware of this unwrapping for *all* Calcite (SQL standard) functions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54743281
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    --- End diff --
    
    Right, most of them are singleton, except for the default one, which needs a DrillFuncHolder list to be associated with them.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426363
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperatorNotInfer.java ---
    @@ -0,0 +1,44 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class HiveUDFOperatorNotInfer extends HiveUDFOperator {
    --- End diff --
    
    The class name is not very meaningful. Can we use other names to different the concept of type inference? 
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455160
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java ---
    @@ -204,4 +220,46 @@ private HiveFuncHolder matchAndCreateUDFHolder(String udfName,
         return null;
       }
     
    +  public class HiveSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private HiveSqlReturnTypeInference() {
    +
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        final TypeProtos.MinorType minorType = TypeInferenceUtils.getDrillTypeFromCalciteType(type);
    +        if(minorType == TypeProtos.MinorType.LATE) {
    +          return opBinding.getTypeFactory()
    +              .createTypeWithNullability(
    +                  opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY),
    +                  true);
    +        }
    +      }
    +
    +      final FunctionCall functionCall = TypeInferenceUtils.convertSqlOperatorBindingToFunctionCall(opBinding);
    +      final HiveFuncHolder hiveFuncHolder = getFunction(functionCall);
    +      if(hiveFuncHolder == null) {
    +        String operandTypes = "";
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54956465
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java ---
    @@ -100,8 +108,13 @@ public void onMatch(RelOptRuleCall ruleCall) {
        */
       private boolean containsAvgStddevVarCall(List<AggregateCall> aggCallList) {
         for (AggregateCall call : aggCallList) {
    -      if (call.getAggregation() instanceof SqlAvgAggFunction
    -          || call.getAggregation() instanceof SqlSumAggFunction) {
    +      SqlAggFunction sqlAggFunction = call.getAggregation();
    +      if(sqlAggFunction instanceof DrillCalciteSqlWrapper) {
    --- End diff --
    
    The exception you might see is like this one:
    
    Error: SYSTEM ERROR: AssertionError: Internal error: Conversion to relational algebra failed to preserve datatypes:
    validated type:
    RecordType(DOUBLE NOT NULL EXPR$0) NOT NULL
    converted type:
    RecordType(INTEGER NOT NULL EXPR$0) NOT NULL


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426515
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -122,6 +127,13 @@ public int size(){
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    +    registerForDefault(operatorTable);
    --- End diff --
    
    "Default" is not a good name (You choose "default", just because that's the existing behavior). Better to change to more meaningful name. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54680708
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,43 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.google.common.base.Preconditions;
    -import org.apache.drill.common.types.TypeProtos.MajorType;
    -import org.apache.drill.common.types.TypeProtos.MinorType;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
    +import java.util.ArrayList;
    +import java.util.List;
    +
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    -
    -  private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, isDeterministic);
    --- End diff --
    
    It does not make sense to that that a DrillSqlOperator would have an empty list of DrillFuncHolder.  From the code, several places will call this constructor, like "convert_fromTIMESTAMP_IMPALA", "CastHigh", all should have corresponding DrillFuncHolder.  
    
    Also, DrillDefaultSqlReturnTypeInference would return "any" in case of empty DrillFuncHolder, while the above cases probably should return non-any types. 
    
    Sounds like when we call this constructor to create a DrillSqlOperator, at minimum, we need verify that there is at least one DrillFuncHolder exists. Otherwise, will it hit problem in execution, if no DrillFuncHolder could be found?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455344
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -122,6 +127,13 @@ public int size(){
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    +    registerForDefault(operatorTable);
    --- End diff --
    
    Renamed one as registerOperatorsWithoutInference; and the other as registerOperatorsWithInference


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454252
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperator.java ---
    @@ -18,28 +18,20 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.fasterxml.jackson.databind.type.TypeFactory;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
     import org.apache.calcite.sql.SqlCallBinding;
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
     import org.apache.calcite.sql.SqlOperandCountRange;
     import org.apache.calcite.sql.SqlOperator;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
     import org.apache.calcite.sql.type.SqlOperandCountRanges;
     import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
     
     public class HiveUDFOperator extends SqlFunction {
    -
    -  public HiveUDFOperator(String name) {
    -    super(new SqlIdentifier(name, SqlParserPos.ZERO), DynamicReturnType.INSTANCE, null, new ArgChecker(), null,
    +  public HiveUDFOperator(String name, SqlReturnTypeInference sqlReturnTypeInference) {
    +    super(new SqlIdentifier(name, SqlParserPos.ZERO), sqlReturnTypeInference, null, new ArgChecker(), null,
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922552
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +112,58 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    --- End diff --
    
    Remove unused method (and FunctionImplementationRegistry#getAllMethods).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56453956
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54923253
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    --- End diff --
    
    final and ImmutableMap.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54963651
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDeferToExecSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDeferToExecSqlReturnTypeInference INSTANCE = new DrillDeferToExecSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      return factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +    }
    +  }
    +
    +  private static class DrillSumSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillSumSqlReturnTypeInference INSTANCE = new DrillSumSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      // If there is group-by and the imput type is Non-nullable,
    +      // the output is Non-nullable;
    +      // Otherwise, the output is nullable.
    +      final boolean isNullable = opBinding.getGroupCount() == 0
    +          || opBinding.getOperandType(0).isNullable();
    +
    +      final SqlTypeName sqlTypeName = opBinding.getOperandType(0).getSqlTypeName();
    +      if(sqlTypeName == SqlTypeName.ANY) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.ANY,
    +            isNullable);
    +      }
    +
    +      final TypeProtos.MinorType inputMinorType = getDrillTypeFromCalciteType(opBinding.getOperandType(0));
    +      if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.BIGINT))
    +          == TypeProtos.MinorType.BIGINT) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.BIGINT,
    +            isNullable);
    +      } else if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT8))
    +          == TypeProtos.MinorType.FLOAT8) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.DOUBLE,
    +            isNullable);
    +      } else {
    +        throw UserException
    +            .functionError()
    +            .message(String.format("%s does not support operand types (%s)",
    +                opBinding.getOperator().getName(),
    +                sqlTypeName))
    +            .build(logger);
    +      }
    +    }
    +  }
    +
    +  private static class DrillCountSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillCountSqlReturnTypeInference INSTANCE = new DrillCountSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName type = SqlTypeName.BIGINT;
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          type,
    +          false);
    +    }
    +  }
    +
    +  private static class DrillConcatSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillConcatSqlReturnTypeInference INSTANCE = new DrillConcatSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      boolean isNullable = true;
    +      int precision = 0;
    +      for(RelDataType relDataType : opBinding.collectOperandTypes()) {
    +        if(!relDataType.isNullable()) {
    +          isNullable = false;
    +        }
    +
    +        // If the underlying columns cannot offer information regarding the precision (i.e., the length) of the VarChar,
    +        // Drill uses the largest to represent it
    +        if(relDataType.getPrecision() == TypeHelper.VARCHAR_DEFAULT_CAST_LEN
    +            || relDataType.getPrecision() == RelDataType.PRECISION_NOT_SPECIFIED) {
    +          precision = TypeHelper.VARCHAR_DEFAULT_CAST_LEN;
    +        } else {
    +          precision += relDataType.getPrecision();
    +        }
    +      }
    +
    +      return factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.VARCHAR, precision),
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillLengthSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillLengthSqlReturnTypeInference INSTANCE = new DrillLengthSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName sqlTypeName = SqlTypeName.BIGINT;
    +
    +      // We need to check only the first argument because
    +      // the second one is used to represent encoding type
    +      final boolean isNullable = opBinding.getOperandType(0).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillPadTrimSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillPadTrimSqlReturnTypeInference INSTANCE = new DrillPadTrimSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName sqlTypeName = SqlTypeName.VARCHAR;
    +
    +      for(int i = 0; i < opBinding.getOperandCount(); ++i) {
    +        if(opBinding.getOperandType(i).isNullable()) {
    +          return DrillConstExecutor.createCalciteTypeWithNullability(
    +              factory, sqlTypeName, true);
    +        }
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory, sqlTypeName, false);
    +    }
    +  }
    +
    +  private static class DrillConvertToSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillConvertToSqlReturnTypeInference INSTANCE = new DrillConvertToSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName type = SqlTypeName.VARBINARY;
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory, type, opBinding.getOperandType(0).isNullable());
    +    }
    +  }
    +
    +  private static class DrillExtractSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillExtractSqlReturnTypeInference INSTANCE = new DrillExtractSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final TimeUnit timeUnit = opBinding.getOperandType(0).getIntervalQualifier().getStartUnit();
    +      final boolean isNullable = opBinding.getOperandType(1).isNullable();
    +
    +      final SqlTypeName sqlTypeName = getSqlTypeNameForTimeUnit(timeUnit.name());
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillSqrtSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillSqrtSqlReturnTypeInference INSTANCE = new DrillSqrtSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final boolean isNullable = opBinding.getOperandType(0).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          SqlTypeName.DOUBLE,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDatePartSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDatePartSqlReturnTypeInference INSTANCE = new DrillDatePartSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      final SqlNode firstOperand = ((SqlCallBinding) opBinding).operand(0);
    +      if(!(firstOperand instanceof SqlCharStringLiteral)) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(factory,
    +            SqlTypeName.ANY,
    +            opBinding.getOperandType(1).isNullable());
    +      }
    +
    +      final String part = ((SqlCharStringLiteral) firstOperand)
    +          .getNlsString()
    +          .getValue()
    +          .toUpperCase();
    +
    +      final SqlTypeName sqlTypeName = getSqlTypeNameForTimeUnit(part);
    +      final boolean isNullable = opBinding.getOperandType(1).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillCastSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillCastSqlReturnTypeInference INSTANCE = new DrillCastSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final boolean isNullable = opBinding
    +          .getOperandType(0)
    +          .isNullable();
    +
    +      RelDataType ret = factory.createTypeWithNullability(
    +          opBinding.getOperandType(1),
    +          isNullable);
    +
    +      if (opBinding instanceof SqlCallBinding) {
    +        SqlCallBinding callBinding = (SqlCallBinding) opBinding;
    +        SqlNode operand0 = callBinding.operand(0);
    +
    +        // dynamic parameters and null constants need their types assigned
    +        // to them using the type they are casted to.
    +        if (((operand0 instanceof SqlLiteral)
    +                && (((SqlLiteral) operand0).getValue() == null))
    +                || (operand0 instanceof SqlDynamicParam)) {
    +          callBinding.getValidator().setValidatedNodeType(
    +                  operand0,
    +                  ret);
    +        }
    +      }
    +
    +      return ret;
    +    }
    +  }
    +
    +  private static DrillFuncHolder resolveDrillFuncHolder(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    final List<LogicalExpression> args = Lists.newArrayList();
    +    for (final RelDataType type : opBinding.collectOperandTypes()) {
    +      final TypeProtos.MinorType minorType = getDrillTypeFromCalciteType(type);
    +      final TypeProtos.MajorType majorType;
    +      if (type.isNullable()) {
    +        majorType =  Types.optional(minorType);
    +      } else {
    +        majorType = Types.required(minorType);
    +      }
    +
    +      args.add(new MajorTypeInLogicalExpression(majorType));
    +    }
    +    final FunctionCall functionCall = new FunctionCall(opBinding.getOperator().getName(), args, ExpressionPosition.UNKNOWN);
    +    final FunctionResolver functionResolver = FunctionResolverFactory.getResolver();
    +    final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
    +
    +    // Throw an exception
    +    // if no DrillFuncHolder matched for the given list of operand types
    +    if(func == null) {
    +      String operandTypes = "";
    +      for(int i = 0; i < opBinding.getOperandCount(); ++i) {
    +        operandTypes += opBinding.getOperandType(i).getSqlTypeName();
    +        if(i < opBinding.getOperandCount() - 1) {
    +          operandTypes += ",";
    +        }
    +      }
    +
    +      throw UserException
    +          .functionError()
    +          .message(String.format("%s does not support operand types (%s)",
    +              opBinding.getOperator().getName(),
    +              operandTypes))
    +          .build(logger);
    +    }
    +    return func;
    +  }
    +
    +  /**
    +   * This class is not intended to be initiated
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454322
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,66 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
     import java.util.Set;
     
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
     import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
    +import org.apache.drill.common.types.TypeProtos;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
    +import org.apache.drill.exec.planner.sql.DrillSqlAggOperatorNotInfer;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
    +import org.apache.drill.exec.planner.sql.DrillSqlOperatorNotInfer;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final ImmutableMap<String, Pair<Integer, Integer>> drillFuncToRange = ImmutableMap.<String, Pair<Integer, Integer>> builder()
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54923241
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    --- End diff --
    
    Please add a comment to:
    SqlTypeName.CHAR -> TypeProtos.MinorType.VARCHAR
    
    This one is incorrect?
    SqlTypeName.DECIMAL -> TypeProtos.MinorType.FLOAT8
    
    Also, aren't there more types here too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54786125
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    @jacques-n I think we can do that for all, except for Concat.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on the pull request:

    https://github.com/apache/drill/pull/397#issuecomment-198462911
  
    +1
    
    I have some minor comments. Also, I think a utility class that unwraps *SqlWrapper classes would be helpful.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455794
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    --- End diff --
    
    operatorsCalcite is native Calcite operators. I change it to operatorsNativeCalcite. Hopefully, it gives better information.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54957475
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56433120
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    --- End diff --
    
    Also, do we need so many structures to track all this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431177
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperator.java ---
    @@ -18,28 +18,20 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.fasterxml.jackson.databind.type.TypeFactory;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
     import org.apache.calcite.sql.SqlCallBinding;
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
     import org.apache.calcite.sql.SqlOperandCountRange;
     import org.apache.calcite.sql.SqlOperator;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
     import org.apache.calcite.sql.type.SqlOperandCountRanges;
     import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
     
     public class HiveUDFOperator extends SqlFunction {
    -
    -  public HiveUDFOperator(String name) {
    -    super(new SqlIdentifier(name, SqlParserPos.ZERO), DynamicReturnType.INSTANCE, null, new ArgChecker(), null,
    +  public HiveUDFOperator(String name, SqlReturnTypeInference sqlReturnTypeInference) {
    +    super(new SqlIdentifier(name, SqlParserPos.ZERO), sqlReturnTypeInference, null, new ArgChecker(), null,
    --- End diff --
    
    Use ArgChecker.INSTANCE (and make that final)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on the pull request:

    https://github.com/apache/drill/pull/397#issuecomment-190864394
  
    @amansinha100, @jinfengni  can you review it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54915765
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    @jacques-n I implemented them.
    Can you review? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56456027
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,740 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlRankFunction;
    +import org.apache.calcite.sql.fun.SqlAvgAggFunction;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static final ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING = ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56456844
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    --- End diff --
    
    I think we had this discussion on the review procedure before. Say, in the Collection<DrillFuncHolder> we have some DrillFuncHolder which are deterministic and the other are non-deterministic. 
    
    By the logic here, we will group the entire Collection<DrillFuncHolder>  as a DrillSqlOperator and claim it is non-deterministic.
    
    In fact, separating them into two DrillSqlOperator (one being deterministic and the other being non-deterministic does not help). 
    
    Please see DrillOperatorTable.lookupOperatorOverloads(). As you can see, parameter list is not passed in. So even if we have two DrillSqlOperator, DrillOperatorTable.lookupOperatorOverloads() does not have the enough information to pick the one. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56704712
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java ---
    @@ -209,6 +212,10 @@ public static long getInitialPlanningMemorySize() {
         return INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES;
       }
     
    +  public boolean isTypeInferenceEnabled() {
    +    return options.getOption(TYPE_INFERENCE.getOptionName()).bool_val;
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431299
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
    --- End diff --
    
    Use interface `OptionManager systemOptions;`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54916226
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    --- End diff --
    
    Comment added into the code


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54955232
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlWrapper.java ---
    @@ -0,0 +1,33 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.sql.SqlOperator;
    +/**
    + * This interface is meant for the users of the wrappers, {@link DrillCalciteSqlOperatorWrapper},
    + * {@link DrillCalciteSqlFunctionWrapper} and {@link DrillCalciteSqlAggFunctionWrapper}, to access the wrapped Calcite
    + * {@link SqlOperator} without knowing exactly which wrapper it is.
    + */
    +public interface DrillCalciteSqlWrapper {
    --- End diff --
    
    This is unfortunately not possible. Take DrillCalciteSqlAggFunctionWrapper as an example:
    
    In order to let Calcite believe this is a SqlAggFunction, we have to claim DrillCalciteSqlAggFunctionWrapper is "extended" from SqlAggFunction. Not being able to do multi-inherentence in JAVA prevents code sharing from another base implementation


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on the pull request:

    https://github.com/apache/drill/pull/397#issuecomment-191911233
  
    Also, commit says I am the author ;)
    
    [caused due to squashing]


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54923238
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    --- End diff --
    
    final


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426782
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    +    registerForDefault(operatorTable);
    +  }
    +
    +  public void registerForInference(DrillOperatorTable operatorTable) {
    --- End diff --
    
    These register methods should be private.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54680959
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory
    +            .createSqlType(SqlTypeName.BOOLEAN);
    --- End diff --
    
    What about the nullability of the return type?  Are you assume that it's non-nullable Boolean ?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431169
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java ---
    @@ -204,4 +220,46 @@ private HiveFuncHolder matchAndCreateUDFHolder(String udfName,
         return null;
       }
     
    +  public class HiveSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private HiveSqlReturnTypeInference() {
    +
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        final TypeProtos.MinorType minorType = TypeInferenceUtils.getDrillTypeFromCalciteType(type);
    +        if(minorType == TypeProtos.MinorType.LATE) {
    +          return opBinding.getTypeFactory()
    +              .createTypeWithNullability(
    +                  opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY),
    +                  true);
    +        }
    +      }
    +
    +      final FunctionCall functionCall = TypeInferenceUtils.convertSqlOperatorBindingToFunctionCall(opBinding);
    +      final HiveFuncHolder hiveFuncHolder = getFunction(functionCall);
    +      if(hiveFuncHolder == null) {
    +        String operandTypes = "";
    --- End diff --
    
    Nit: use StringBuilder


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426814
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java ---
    @@ -100,8 +108,13 @@ public void onMatch(RelOptRuleCall ruleCall) {
        */
       private boolean containsAvgStddevVarCall(List<AggregateCall> aggCallList) {
         for (AggregateCall call : aggCallList) {
    -      if (call.getAggregation() instanceof SqlAvgAggFunction
    -          || call.getAggregation() instanceof SqlSumAggFunction) {
    +      SqlAggFunction sqlAggFunction = call.getAggregation();
    +      if(sqlAggFunction instanceof DrillCalciteSqlWrapper) {
    --- End diff --
    
    Would a utility class that does the unwrapping be helpful?
    
    Different util functions unwrap different operators.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54952796
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +112,58 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54954482
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java ---
    @@ -770,6 +770,65 @@ public void eval() {
         } // end of eval
       }
     
    +  /*
    +   * Fill up the string to length 'length' by prepending the character ' ' in the beginning of 'text'.
    +   * If the string is already longer than length, then it is truncated (on the right).
    +   */
    +  @FunctionTemplate(name = "lpad", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
    +  public static class LpadTwoArg implements DrillSimpleFunc {
    --- End diff --
    
    I already have some in TestFunctionsWithTypeExpoQueries
    I added more to TestStringFunctions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54689413
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    --- End diff --
    
    Similar to the previous comment, it would be preferable to not have switch statements with lots of cases (the number of such functions that need to be handled here could keep growing).  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455114
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    +    registerForDefault(operatorTable);
    +  }
    +
    +  public void registerForInference(DrillOperatorTable operatorTable) {
    +    final Map<String, DrillSqlOperator.DrillSqlOperatorBuilder> map = Maps.newHashMap();
    +    final Map<String, DrillSqlAggOperator.DrillSqlAggOperatorBuilder> mapAgg = Maps.newHashMap();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
    +      for (DrillFuncHolder func : function.getValue()) {
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumberRange;
    +          if(drillFuncToRange.containsKey(name)) {
    +            argNumberRange = drillFuncToRange.get(name);
    +          } else {
    +            argNumberRange = Pair.of(func.getParamCount(), func.getParamCount());
    +          }
    +          functions.put(argNumberRange, func);
    +        }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        if(map.containsKey(name)) {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426944
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperatorNotInfer.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlAggOperatorNotInfer extends DrillSqlAggOperator {
    --- End diff --
    
    Nit: DrillSqlAggOperatorWithoutInference?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54916066
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory
    +            .createSqlType(SqlTypeName.BOOLEAN);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY || type.getSqlTypeName() == SqlTypeName.DECIMAL) {
    --- End diff --
    
    Right, since limit-0 patch should be able to know whether the shortcut can be taken for Decimal, this patch does not need to worry about it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54786023
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    --- End diff --
    
    @jacques-n Yes, once per registry. 
    
    This patch has not changed that behavior.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56527875
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus schema) {
         }
       }
     
    +  private static class DrillRexBuilder extends RexBuilder {
    +    private DrillRexBuilder(RelDataTypeFactory typeFactory) {
    +      super(typeFactory);
    +    }
    +
    +    @Override
    +    public RexNode ensureType(
    +        RelDataType type,
    +        RexNode node,
    +        boolean matchNullability) {
    +      RelDataType targetType = type;
    +      if (matchNullability) {
    +        targetType = matchNullability(type, node);
    +      }
    +      if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
    +        return node;
    +      }
    +      if (!node.getType().equals(targetType)) {
    +        if(!targetType.isStruct()) {
    +          final RelDataType anyType = TypeInferenceUtils.createCalciteTypeWithNullability(
    --- End diff --
    
    I thought Calcite might have mechanism to further catch the cases where implicit casting is not added in. 
    
    Let me try if we can find any issue when there is no such a cast to ANY


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54955354
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56692124
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java ---
    @@ -40,7 +43,16 @@
     public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
     
       private final SqlAvgAggFunction.Subtype subtype;
    -  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false);
    +  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false,
    +      new SqlReturnTypeInference() {
    +        @Override
    +        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +          return TypeInferenceUtils.createCalciteTypeWithNullability(
    +              opBinding.getTypeFactory(),
    +              SqlTypeName.ANY,
    --- End diff --
    
    So regarding 2, what guarantees that return types at validation and at execution are same, since we add a `CastHigh` after validation?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56458266
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -140,6 +140,10 @@ public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
         }
     
         public DrillSqlOperator build() {
    +      if(name == null || functions.isEmpty()) {
    +        throw new AssertionError("The fields, name and functions, need to be set before build DrillSqlAggOperator");
    --- End diff --
    
    build gets called before fields are properly set.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56440041
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    +  private static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  private final TypeProtos.MajorType returnType;
    +
    +  public DrillSqlOperatorNotInfer(String name, int argCount, TypeProtos.MajorType returnType, boolean isDeterminisitic) {
    +    super(name,
    +        new ArrayList< DrillFuncHolder>(),
    +        argCount,
    +        argCount,
    +        isDeterminisitic,
    +        DynamicReturnType.INSTANCE);
    --- End diff --
    
    This is legacy code [1] (DrillSqlOperatorNotInfer corresponds to the DrillSqlOperator in the old code). 
    [1] https://github.com/apache/drill/blob/d7eebec41a1636055be1b2c79b693d76c52d8932/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java#L48


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54916158
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory
    +            .createSqlType(SqlTypeName.BOOLEAN);
    --- End diff --
    
    Addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54765563
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    @jacques-n , here is my understanding about the dummy functions @hsuanyi is talking about.
    
    For 'lpad' function, Calcite allows both 2 arguments or 3 arguments. Drill has only implementation for the case of 3 arguments. What Drill did is DrillOptiq will rewrite the case of 2 arguments into 3 arguments [2]. The dummy function is for the case of 2 arguments, since Drill's run-time generate logic does not need the implementation of 2 arguments at all, due to the rewrite in DrillOptiq. 
    
    Another option is 1) to remove the rewrite in DrillOptiq and 2) add the implementation for the case of 2 arguments (which is essentially same as the one for 3 arguments).  The new implementation for 2 arguments case is not dummy at all. 
    
      
    
    [1] https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java#L702-L707
    [2] https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java#L429-L434


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56453885
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperatorNotInfer.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlAggOperatorNotInfer extends DrillSqlAggOperator {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56458586
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus schema) {
         }
       }
     
    +  private static class DrillRexBuilder extends RexBuilder {
    +    private DrillRexBuilder(RelDataTypeFactory typeFactory) {
    +      super(typeFactory);
    +    }
    +
    +    @Override
    +    public RexNode ensureType(
    +        RelDataType type,
    +        RexNode node,
    +        boolean matchNullability) {
    +      RelDataType targetType = type;
    +      if (matchNullability) {
    +        targetType = matchNullability(type, node);
    +      }
    +      if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
    +        return node;
    +      }
    +      if (!node.getType().equals(targetType)) {
    +        if(!targetType.isStruct()) {
    +          final RelDataType anyType = TypeInferenceUtils.createCalciteTypeWithNullability(
    --- End diff --
    
    This is to avoid implicit casting added by Calcite.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56439803
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    +  private static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  private final TypeProtos.MajorType returnType;
    +
    +  public DrillSqlOperatorNotInfer(String name, int argCount, TypeProtos.MajorType returnType, boolean isDeterminisitic) {
    +    super(name,
    +        new ArrayList< DrillFuncHolder>(),
    +        argCount,
    +        argCount,
    +        isDeterminisitic,
    +        DynamicReturnType.INSTANCE);
    +    this.returnType = Preconditions.checkNotNull(returnType);
    +  }
    +
    +  protected RelDataType getReturnDataType(final RelDataTypeFactory factory) {
    +    if (TypeProtos.MinorType.BIT.equals(returnType.getMinorType())) {
    +      return factory.createSqlType(SqlTypeName.BOOLEAN);
    +    }
    +    return factory.createTypeWithNullability(factory.createSqlType(SqlTypeName.ANY), true);
    +  }
    +
    +  private RelDataType getNullableReturnDataType(final RelDataTypeFactory factory) {
    +    return factory.createTypeWithNullability(getReturnDataType(factory), true);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    +    if (NONE.equals(returnType)) {
    +      return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
    --- End diff --
    
    Similar to the one in DrillSqlAggOperatorNotInfer.java. Please see [1]
    
    [1] https://github.com/apache/drill/blob/d7eebec41a1636055be1b2c79b693d76c52d8932/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java#L71


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922784
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java ---
    @@ -17,18 +17,51 @@
      */
     package org.apache.drill.exec.planner.sql;
     
    +import com.google.common.collect.Maps;
     import org.apache.calcite.sql.SqlCallBinding;
     import org.apache.calcite.sql.SqlOperandCountRange;
     import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.type.SqlOperandCountRanges;
     import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.commons.lang3.tuple.Pair;
    +
    +import java.util.Map;
     
     class Checker implements SqlOperandTypeChecker {
       private SqlOperandCountRange range;
     
    -  public Checker(int size) {
    +  public static Checker ANY_CHECKER = new Checker();
    --- End diff --
    
    final


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54923222
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    --- End diff --
    
    final


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54925324
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDeferToExecSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDeferToExecSqlReturnTypeInference INSTANCE = new DrillDeferToExecSqlReturnTypeInference();
    --- End diff --
    
    mark all the static instances here and below _final_.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454944
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
     
       public DrillOperatorTable(FunctionImplementationRegistry registry) {
    -    operators = Lists.newArrayList();
    -    operators.addAll(inner.getOperatorList());
    +    this(registry, null);
    --- End diff --
    
    I removed this constructor


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426761
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperatorNotInfer.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlAggOperatorNotInfer extends DrillSqlAggOperator {
    +  public DrillSqlAggOperatorNotInfer(String name, int argCount) {
    +    super(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, DynamicReturnType.INSTANCE);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    +    return getAny(validator.getTypeFactory());
    +  }
    +
    +  private RelDataType getAny(RelDataTypeFactory factory){
    +    return factory.createSqlType(SqlTypeName.ANY);
    --- End diff --
    
    what about nullability of 'ANY" type here? Are you returning non-nullable "any"? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54744340
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    --- End diff --
    
    I replied this one in the previous pull request. But let me state again here. 
    Even for boolean, I did not feel comfortable to keep this logic (this logic has been in the code before this patch) because ANY type as input means it is unknown.
    
    However, if we have a boolean output function in the WHERE-CLAUSE, this logic can help pass calcite's check
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426767
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    --- End diff --
    
    Also, is there a way without registering the two variants of the same UDF?
    
    Implementations of PluggableFunctionRegistry need to know call operatorTable#addInference and operatorTable#addDefault methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54681010
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory
    +            .createSqlType(SqlTypeName.BOOLEAN);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY || type.getSqlTypeName() == SqlTypeName.DECIMAL) {
    --- End diff --
    
    Can you explain why you will return ANY, if one of the input type is DECIMAL? Probably put some comments here. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56527238
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus schema) {
         }
       }
     
    +  private static class DrillRexBuilder extends RexBuilder {
    +    private DrillRexBuilder(RelDataTypeFactory typeFactory) {
    +      super(typeFactory);
    +    }
    +
    +    @Override
    +    public RexNode ensureType(
    +        RelDataType type,
    +        RexNode node,
    +        boolean matchNullability) {
    +      RelDataType targetType = type;
    +      if (matchNullability) {
    +        targetType = matchNullability(type, node);
    +      }
    +      if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
    +        return node;
    +      }
    +      if (!node.getType().equals(targetType)) {
    +        if(!targetType.isStruct()) {
    +          final RelDataType anyType = TypeInferenceUtils.createCalciteTypeWithNullability(
    --- End diff --
    
    As you can see line 418, if one side is ANY, then even if the types are different, Calcite would not complain, nor fail.
    
    Here, what we want to do is "not letting Calcite doing anything" even when the types are not matched. So I think we should borrow the solution in line 418.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426971
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    --- End diff --
    
    Nit: rename


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54954517
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java ---
    @@ -17,18 +17,51 @@
      */
     package org.apache.drill.exec.planner.sql;
     
    +import com.google.common.collect.Maps;
     import org.apache.calcite.sql.SqlCallBinding;
     import org.apache.calcite.sql.SqlOperandCountRange;
     import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.type.SqlOperandCountRanges;
     import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.commons.lang3.tuple.Pair;
    +
    +import java.util.Map;
     
     class Checker implements SqlOperandTypeChecker {
       private SqlOperandCountRange range;
     
    -  public Checker(int size) {
    +  public static Checker ANY_CHECKER = new Checker();
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426878
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    --- End diff --
    
    Nits: better names for these variables?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on the pull request:

    https://github.com/apache/drill/pull/397#issuecomment-198539051
  
    +1. 
    As a separate exercise, I have suggested to the performance QA team to add planning specific perf tests with large number of expressions to measure type inferencing performance. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56692131
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,93 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.OptionManager;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> calciteOperators = Lists.newArrayList();
    +  private final List<SqlOperator> drillOperatorsWithoutInference = Lists.newArrayList();
    +  private final List<SqlOperator> drillOperatorsWithInference = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> drillOperatorsWithoutInferenceMap = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> drillOperatorsWithInferenceMap = ArrayListMultimap.create();
     
    -  public DrillOperatorTable(FunctionImplementationRegistry registry) {
    -    operators = Lists.newArrayList();
    -    operators.addAll(inner.getOperatorList());
    +  private final OptionManager systemOptionManager;
     
    +  public DrillOperatorTable(FunctionImplementationRegistry registry, SystemOptionManager systemOptionManager) {
    --- End diff --
    
    `OptionManager systemOptionManager`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jacques-n <gi...@git.apache.org>.
Github user jacques-n commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54762638
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    Let's just fix this by adding the required functions. Having exceptions here and in DrillOptiq seems to be hard to maintain hack.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54763522
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    Then, we have to add dummy functions. That could work for most of them, but not Concat (which accepts 1 to infinite number of arguments).
    
    And that is why we chose moving away from DummyFunctions


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jacques-n <gi...@git.apache.org>.
Github user jacques-n commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54762584
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    --- End diff --
    
    Can you confirm that we are only creating DrillSqlOperators once per registry as opposed to per query/parsing?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56427110
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,740 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlKind;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlRankFunction;
    +import org.apache.calcite.sql.fun.SqlAvgAggFunction;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.FunctionCallFactory;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static final ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING = ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    --- End diff --
    
    Nit: wrap around to next line (here and below)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56692111
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlannerSettings.java ---
    @@ -209,6 +212,10 @@ public static long getInitialPlanningMemorySize() {
         return INITIAL_OFF_HEAP_ALLOCATION_IN_BYTES;
       }
     
    +  public boolean isTypeInferenceEnabled() {
    +    return options.getOption(TYPE_INFERENCE.getOptionName()).bool_val;
    --- End diff --
    
    ```java
    return option.getOption(TYPE_INFERENCE);
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922890
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    --- End diff --
    
    I recollect there were more types here, albeit commented out. Can you add them back (including comments)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56458540
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus schema) {
         }
       }
     
    +  private static class DrillRexBuilder extends RexBuilder {
    +    private DrillRexBuilder(RelDataTypeFactory typeFactory) {
    +      super(typeFactory);
    +    }
    +
    +    @Override
    +    public RexNode ensureType(
    +        RelDataType type,
    +        RexNode node,
    +        boolean matchNullability) {
    +      RelDataType targetType = type;
    +      if (matchNullability) {
    +        targetType = matchNullability(type, node);
    +      }
    +      if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
    +        return node;
    +      }
    +      if (!node.getType().equals(targetType)) {
    +        if(!targetType.isStruct()) {
    +          final RelDataType anyType = TypeInferenceUtils.createCalciteTypeWithNullability(
    --- End diff --
    
    why do you extend RexBuilder and override ensureType() here? Most of the code is identical in RexBuilder.ensureType, except for line 422 - 427. Are they Drill-specific logic? If the answer is no, then we should put this logic in Calcite, not in Drill. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56459723
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54965064
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,60 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
    -import java.util.Set;
     
    -import org.apache.calcite.sql.SqlOperator;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final Map<String, Pair<Integer, Integer>> drillFuncToRange = Maps.newHashMap();
    +  static {
    +    // CONCAT is allowed to take [1, infinity) number of arguments.
    +    // Currently, this flexibility is offered by DrillOptiq to rewrite it as
    +    // a nested structure
    +    drillFuncToRange.put("CONCAT", Pair.of(1, Integer.MAX_VALUE));
    +
    +    // When LENGTH is given two arguments, this function relies on DrillOptiq to rewrite it as
    +    // another function based on the second argument (encodingType)
    +    drillFuncToRange.put("LENGTH", Pair.of(1, 2));
    +
    +    // Dummy functions
    --- End diff --
    
    Concat is due to the flexibility of variable # of arguments
    LENGTH is.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56459546
  
    --- Diff: contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java ---
    @@ -43,4 +43,32 @@ public void testEncode() throws Exception {
             .baselineValues(new Object[] { null })
             .go();
       }
    +
    +  @Test
    +  public void testReflect() throws Exception {
    +    final String query = "select reflect('java.lang.Math', 'round', cast(2 as float)) as col \n" +
    --- End diff --
    
    Addressed. But I choose a function which type can be resolved in planning time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54964989
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDeferToExecSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDeferToExecSqlReturnTypeInference INSTANCE = new DrillDeferToExecSqlReturnTypeInference();
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54952798
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,60 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
    -import java.util.Set;
     
    -import org.apache.calcite.sql.SqlOperator;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final Map<String, Pair<Integer, Integer>> drillFuncToRange = Maps.newHashMap();
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54952793
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +112,58 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange;
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jacques-n <gi...@git.apache.org>.
Github user jacques-n commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54763735
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    Why do we have to add dummy functions? I understand in the case that we don't support variable arguments. Why is that the case here?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56427768
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    --- End diff --
    
    pls consider using a different class name. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56427623
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    +  private static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  private final TypeProtos.MajorType returnType;
    +
    +  public DrillSqlOperatorNotInfer(String name, int argCount, TypeProtos.MajorType returnType, boolean isDeterminisitic) {
    +    super(name,
    +        new ArrayList< DrillFuncHolder>(),
    +        argCount,
    +        argCount,
    +        isDeterminisitic,
    +        DynamicReturnType.INSTANCE);
    +    this.returnType = Preconditions.checkNotNull(returnType);
    +  }
    +
    +  protected RelDataType getReturnDataType(final RelDataTypeFactory factory) {
    +    if (TypeProtos.MinorType.BIT.equals(returnType.getMinorType())) {
    +      return factory.createSqlType(SqlTypeName.BOOLEAN);
    +    }
    +    return factory.createTypeWithNullability(factory.createSqlType(SqlTypeName.ANY), true);
    +  }
    +
    +  private RelDataType getNullableReturnDataType(final RelDataTypeFactory factory) {
    +    return factory.createTypeWithNullability(getReturnDataType(factory), true);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    +    if (NONE.equals(returnType)) {
    +      return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
    --- End diff --
    
    Nullability of "any" type? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426746
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperatorNotInfer.java ---
    @@ -0,0 +1,44 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class HiveUDFOperatorNotInfer extends HiveUDFOperator {
    --- End diff --
    
    Nit: better class name? HiveUDFOperatorWithoutInference?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454893
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    --- End diff --
    
    We want to safely fall back to original behavior (which does not do inference). So the safest way is to have the ones which do not do inference to have their own operators. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431301
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
     
       public DrillOperatorTable(FunctionImplementationRegistry registry) {
    -    operators = Lists.newArrayList();
    -    operators.addAll(inner.getOperatorList());
    +    this(registry, null);
    --- End diff --
    
    Is this to get around unit tests? If so, can you please deprecate this c'tor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56457961
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    --- End diff --
    
    ok


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54924830
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    --- End diff --
    
    Move createCalciteTypeWithNullability to this class.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56453886
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -62,11 +123,70 @@ public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory ca
     
       @Override
       public List<SqlOperator> getOperatorList() {
    -    return operators;
    +    final List<SqlOperator> sqlOperators = Lists.newArrayList();
    +    sqlOperators.addAll(operatorsCalcite);
    +    if(isEnableInference()) {
    +      sqlOperators.addAll(operatorsInferernce);
    +    } else {
    +      sqlOperators.addAll(operatorsDefault);
    +    }
    +
    +    return sqlOperators;
       }
     
       // Get the list of SqlOperator's with the given name.
       public List<SqlOperator> getSqlOperator(String name) {
    -    return opMap.get(name.toLowerCase());
    +    if(isEnableInference()) {
    +      return opMapInferernce.get(name.toLowerCase());
    +    } else {
    +      return opMapDefault.get(name.toLowerCase());
    +    }
    +  }
    +
    +  private void populateWrappedCalciteOperators() {
    +    for(SqlOperator calciteOperator : inner.getOperatorList()) {
    +      final SqlOperator wrapper;
    +      if(calciteOperator instanceof SqlAggFunction) {
    +        wrapper = new DrillCalciteSqlAggFunctionWrapper((SqlAggFunction) calciteOperator,
    +            getFunctionListWithInference(calciteOperator.getName()));
    +      } else if(calciteOperator instanceof SqlFunction) {
    +        wrapper = new DrillCalciteSqlFunctionWrapper((SqlFunction) calciteOperator,
    +            getFunctionListWithInference(calciteOperator.getName()));
    +      } else {
    +        final String drillOpName = FunctionCallFactory.replaceOpWithFuncName(calciteOperator.getName());
    +        final List<DrillFuncHolder> drillFuncHolders = getFunctionListWithInference(drillOpName);
    +        if(drillFuncHolders.isEmpty() || calciteOperator == SqlStdOperatorTable.UNARY_MINUS || calciteOperator == SqlStdOperatorTable.UNARY_PLUS) {
    +          continue;
    +        }
    +
    +        wrapper = new DrillCalciteSqlOperatorWrapper(calciteOperator, drillOpName, drillFuncHolders);
    +      }
    +      calciteToWrapper.put(calciteOperator, wrapper);
    +    }
    +  }
    +
    +  private List<DrillFuncHolder> getFunctionListWithInference(String name) {
    +    final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    for(SqlOperator sqlOperator : opMapInferernce.get(name.toLowerCase())) {
    +      if(sqlOperator instanceof DrillSqlOperator) {
    +        final List<DrillFuncHolder> list = ((DrillSqlOperator) sqlOperator).getFunctions();
    +        if(list != null) {
    +          functions.addAll(list);
    +        }
    +      }
    +
    +      if(sqlOperator instanceof DrillSqlAggOperator) {
    +        final List<DrillFuncHolder> list = ((DrillSqlAggOperator) sqlOperator).getFunctions();
    +        if(list != null) {
    +          functions.addAll(list);
    +        }
    +      }
    +    }
    +    return functions;
    +  }
    +
    +  private boolean isEnableInference() {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54956169
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillReduceAggregatesRule.java ---
    @@ -100,8 +108,13 @@ public void onMatch(RelOptRuleCall ruleCall) {
        */
       private boolean containsAvgStddevVarCall(List<AggregateCall> aggCallList) {
         for (AggregateCall call : aggCallList) {
    -      if (call.getAggregation() instanceof SqlAvgAggFunction
    -          || call.getAggregation() instanceof SqlSumAggFunction) {
    +      SqlAggFunction sqlAggFunction = call.getAggregation();
    +      if(sqlAggFunction instanceof DrillCalciteSqlWrapper) {
    --- End diff --
    
    Yeah,... I see the point. But this is necessary to still have wrapper here. 
    
    For example, the type of avg(integer_type_col) in Calcite is defined as Integer. Thus, say, if the wrapper is removed somehow before this rule is fired, Calcite will complain the type it sees here (Integer) does not match with that (Double) we inferred before.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/drill/pull/397


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56435145
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    +        this.isDeterministic = isDeterministic;
    +      }
    +      return this;
    +    }
    +
    +    public DrillSqlOperator build() {
    +      return new DrillSqlOperator(
    +          name,
    +          functions,
    +          argCountMin,
    +          argCountMax,
    +          isDeterministic,
    +          TypeInferenceUtils.getDrillSqlReturnTypeInference(
    +              name,
    --- End diff --
    
    Can we ensure that name and functions are always set, before call this constructor this builder? If not, will it hit problem?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54955290
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56692104
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +114,100 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerOperatorsWithInference(operatorTable);
    +    registerOperatorsWithoutInference(operatorTable);
    +  }
    +
    +  private void registerOperatorsWithInference(DrillOperatorTable operatorTable) {
    +    final Map<String, DrillSqlOperator.DrillSqlOperatorBuilder> map = Maps.newHashMap();
    +    final Map<String, DrillSqlAggOperator.DrillSqlAggOperatorBuilder> mapAgg = Maps.newHashMap();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
    +      for (DrillFuncHolder func : function.getValue()) {
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumberRange;
    +          if(registeredFuncNameToArgRange.containsKey(name)) {
    +            argNumberRange = registeredFuncNameToArgRange.get(name);
    +          } else {
    +            argNumberRange = Pair.of(func.getParamCount(), func.getParamCount());
    +          }
    +          functions.put(argNumberRange, func);
    +        }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        if(!map.containsKey(name)) {
    +          map.put(name, new DrillSqlOperator.DrillSqlOperatorBuilder()
    +              .setName(name));
    +        }
    +
    +        final DrillSqlOperator.DrillSqlOperatorBuilder drillSqlOperatorBuilder = map.get(name);
    +        drillSqlOperatorBuilder
    +            .addFunctions(entry.getValue())
    +            .setArgumentCount(min, max)
    +            .setDeterministic(isDeterministic);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        if(mapAgg.containsKey(name)) {
    --- End diff --
    
    simplify this `if..else` as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426901
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
     
       public DrillOperatorTable(FunctionImplementationRegistry registry) {
    -    operators = Lists.newArrayList();
    -    operators.addAll(inner.getOperatorList());
    +    this(registry, null);
    +  }
     
    +  public DrillOperatorTable(FunctionImplementationRegistry registry, SystemOptionManager systemOptionManager) {
         registry.register(this);
    +    operatorsCalcite.addAll(inner.getOperatorList());
    +    populateWrappedCalciteOperators();
    +    this.systemOptionManager = systemOptionManager;
    +  }
    +
    +  public void addDefault(String name, SqlOperator op) {
    --- End diff --
    
    From names, the functionality of addDefault and addInference is not obvious.
    
    Please add docs to these methods and class, since implementations of pluggable function registry have to use these methods.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56439767
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -183,10 +190,40 @@ public SchemaPlus getDefaultSchema() {
       }
     
       private class DrillValidator extends SqlValidatorImpl {
    +    private final Set<SqlValidatorScope> identitySet = Sets.newIdentityHashSet();
    +
         protected DrillValidator(SqlOperatorTable opTab, SqlValidatorCatalogReader catalogReader,
             RelDataTypeFactory typeFactory, SqlConformance conformance) {
           super(opTab, catalogReader, typeFactory, conformance);
         }
    +
    +    @Override
    +    public SqlValidatorScope getSelectScope(final SqlSelect select) {
    --- End diff --
    
    We need better solution here. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431228
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java ---
    @@ -135,6 +135,16 @@ public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin>
         }
       },
     
    +  SUM_CONVERSION("Convert SUM to $SUM0") {
    --- End diff --
    
    Is this phase enforced?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54690939
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    --- End diff --
    
    I am a little worried that type inferencing for each of these functions is instantiating a new object.  I haven't looked into the details of these classes, but we should keep type inferencing as lightweight as possible. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56434976
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperatorNotInfer.java ---
    @@ -0,0 +1,43 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlAggOperatorNotInfer extends DrillSqlAggOperator {
    +  public DrillSqlAggOperatorNotInfer(String name, int argCount) {
    +    super(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, DynamicReturnType.INSTANCE);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    +    return getAny(validator.getTypeFactory());
    +  }
    +
    +  private RelDataType getAny(RelDataTypeFactory factory){
    +    return factory.createSqlType(SqlTypeName.ANY);
    --- End diff --
    
    DrillSqlAggOperatorNotInfer.java corresponds to the DrillSqlAggOperator in the old code[1]. When the inference option is being turned off, we want to fall back to the original behavior as close as possible. With that in mind, I was trying to not change the logic too much.
    
    [1]https://github.com/apache/drill/blob/d7eebec41a1636055be1b2c79b693d76c52d8932/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java#L50


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454989
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,34 +33,88 @@
     import org.apache.calcite.sql.SqlOperatorTable;
     import org.apache.calcite.sql.SqlSyntax;
     import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.drill.exec.planner.physical.PlannerSettings;
    +import org.apache.drill.exec.server.options.SystemOptionManager;
     
     import java.util.List;
    +import java.util.Map;
     
    +/**
    + * Implementation of {@link SqlOperatorTable} that contains standard operators and functions provided through
    + * {@link #inner SqlStdOperatorTable}, and Drill User Defined Functions.
    + */
     public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    -
    +//  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators;
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
    +  private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922840
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlWrapper.java ---
    @@ -0,0 +1,33 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.sql.SqlOperator;
    +/**
    + * This interface is meant for the users of the wrappers, {@link DrillCalciteSqlOperatorWrapper},
    + * {@link DrillCalciteSqlFunctionWrapper} and {@link DrillCalciteSqlAggFunctionWrapper}, to access the wrapped Calcite
    + * {@link SqlOperator} without knowing exactly which wrapper it is.
    + */
    +public interface DrillCalciteSqlWrapper {
    --- End diff --
    
    There is redundant code in DrillCalciteSqlAggFunctionWrapper, DrillCalciteSqlFunctionWrapper and DrillCalciteSqlOperatorWrapper.
    
    Can these extend from a base implementation which has common methods?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454445
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -183,10 +190,40 @@ public SchemaPlus getDefaultSchema() {
       }
     
       private class DrillValidator extends SqlValidatorImpl {
    +    private final Set<SqlValidatorScope> identitySet = Sets.newIdentityHashSet();
    +
         protected DrillValidator(SqlOperatorTable opTab, SqlValidatorCatalogReader catalogReader,
             RelDataTypeFactory typeFactory, SqlConformance conformance) {
           super(opTab, catalogReader, typeFactory, conformance);
         }
    +
    +    @Override
    +    public SqlValidatorScope getSelectScope(final SqlSelect select) {
    --- End diff --
    
    will move this logic to calcite.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455697
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    --- End diff --
    
    addressed as above


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54831804
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    --- End diff --
    
    This part of code is added to eliminate our usage of Dummy functions. (DummyConvertTo is an example).
    
    I think for now, let me just revert it because it might introduce unnecessary code change and blur the objective. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922538
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,60 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
    -import java.util.Set;
     
    -import org.apache.calcite.sql.SqlOperator;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final Map<String, Pair<Integer, Integer>> drillFuncToRange = Maps.newHashMap();
    --- End diff --
    
    ImmutableMap?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jacques-n <gi...@git.apache.org>.
Github user jacques-n commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54765921
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    Your last suggestion is what I'm saying. Just implement a 2 argument implementation then everything works the same as everything else. The 2 argument version can share code/delegate the three argument version. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454010
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java ---
    @@ -17,47 +17,76 @@
      */
     package org.apache.drill.exec.planner.sql;
     
    -import java.util.List;
    -
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import com.google.common.collect.Lists;
     import org.apache.calcite.sql.SqlAggFunction;
    -import org.apache.calcite.sql.SqlCall;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
     import org.apache.calcite.sql.SqlKind;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
    -import com.google.common.collect.ImmutableList;
    +import java.util.Collection;
    +import java.util.List;
     
     public class DrillSqlAggOperator extends SqlAggFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);
    +  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);
    +  private final List<DrillFuncHolder> functions;
     
    -
    -  public DrillSqlAggOperator(String name, int argCount) {
    -    super(name, new SqlIdentifier(name, SqlParserPos.ZERO), SqlKind.OTHER_FUNCTION, DynamicReturnType.INSTANCE, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
    +  protected DrillSqlAggOperator(String name, List<DrillFuncHolder> functions, int argCountMin, int argCountMax, SqlReturnTypeInference sqlReturnTypeInference) {
    +    super(name,
    +        new SqlIdentifier(name, SqlParserPos.ZERO),
    +        SqlKind.OTHER_FUNCTION,
    +        sqlReturnTypeInference,
    +        null,
    +        Checker.getChecker(argCountMin, argCountMax),
    +        SqlFunctionCategory.USER_DEFINED_FUNCTION);
    +    this.functions = functions;
       }
     
    -  @Override
    -  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    -    return getAny(validator.getTypeFactory());
    +  private DrillSqlAggOperator(String name, List<DrillFuncHolder> functions, int argCountMin, int argCountMax) {
    +    this(name,
    +        functions,
    +        argCountMin,
    +        argCountMax,
    +        TypeInferenceUtils.getDrillSqlReturnTypeInference(
    +            name,
    +            functions));
       }
     
    -  private RelDataType getAny(RelDataTypeFactory factory){
    -    return factory.createSqlType(SqlTypeName.ANY);
    -//    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
    +  public List<DrillFuncHolder> getFunctions() {
    +    return functions;
       }
     
    -//  @Override
    -//  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
    -//    return ImmutableList.of(typeFactory.createSqlType(SqlTypeName.ANY));
    -//  }
    -//
    -//  @Override
    -//  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
    -//    return getAny(typeFactory);
    -//  }
    +  public static class DrillSqlAggOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56435067
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    --- End diff --
    
    why do you have this check?
    
    If someone uses this builder, setDeterminsitic(false), then setDeterministic(true). The second call will not get the desired result.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54926134
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,60 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
    -import java.util.Set;
     
    -import org.apache.calcite.sql.SqlOperator;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final Map<String, Pair<Integer, Integer>> drillFuncToRange = Maps.newHashMap();
    +  static {
    +    // CONCAT is allowed to take [1, infinity) number of arguments.
    +    // Currently, this flexibility is offered by DrillOptiq to rewrite it as
    +    // a nested structure
    +    drillFuncToRange.put("CONCAT", Pair.of(1, Integer.MAX_VALUE));
    +
    +    // When LENGTH is given two arguments, this function relies on DrillOptiq to rewrite it as
    +    // another function based on the second argument (encodingType)
    +    drillFuncToRange.put("LENGTH", Pair.of(1, 2));
    +
    +    // Dummy functions
    --- End diff --
    
    Are these rewritten too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56427872
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/expr/fn/HiveFunctionRegistry.java ---
    @@ -70,7 +85,8 @@ public HiveFunctionRegistry(DrillConfig config) {
       @Override
       public void register(DrillOperatorTable operatorTable) {
         for (String name : Sets.union(methodsGenericUDF.asMap().keySet(), methodsUDF.asMap().keySet())) {
    -      operatorTable.add(name, new HiveUDFOperator(name.toUpperCase()));
    +      operatorTable.addDefault(name, new HiveUDFOperatorNotInfer(name.toUpperCase()));
    --- End diff --
    
    Please update the docs for PluggableFunctionRegistry accordingly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426918
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -62,11 +123,70 @@ public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory ca
     
       @Override
       public List<SqlOperator> getOperatorList() {
    -    return operators;
    +    final List<SqlOperator> sqlOperators = Lists.newArrayList();
    +    sqlOperators.addAll(operatorsCalcite);
    +    if(isEnableInference()) {
    +      sqlOperators.addAll(operatorsInferernce);
    +    } else {
    +      sqlOperators.addAll(operatorsDefault);
    +    }
    +
    +    return sqlOperators;
       }
     
       // Get the list of SqlOperator's with the given name.
       public List<SqlOperator> getSqlOperator(String name) {
    -    return opMap.get(name.toLowerCase());
    +    if(isEnableInference()) {
    +      return opMapInferernce.get(name.toLowerCase());
    +    } else {
    +      return opMapDefault.get(name.toLowerCase());
    +    }
    +  }
    +
    +  private void populateWrappedCalciteOperators() {
    +    for(SqlOperator calciteOperator : inner.getOperatorList()) {
    +      final SqlOperator wrapper;
    +      if(calciteOperator instanceof SqlAggFunction) {
    +        wrapper = new DrillCalciteSqlAggFunctionWrapper((SqlAggFunction) calciteOperator,
    +            getFunctionListWithInference(calciteOperator.getName()));
    +      } else if(calciteOperator instanceof SqlFunction) {
    +        wrapper = new DrillCalciteSqlFunctionWrapper((SqlFunction) calciteOperator,
    +            getFunctionListWithInference(calciteOperator.getName()));
    +      } else {
    +        final String drillOpName = FunctionCallFactory.replaceOpWithFuncName(calciteOperator.getName());
    +        final List<DrillFuncHolder> drillFuncHolders = getFunctionListWithInference(drillOpName);
    +        if(drillFuncHolders.isEmpty() || calciteOperator == SqlStdOperatorTable.UNARY_MINUS || calciteOperator == SqlStdOperatorTable.UNARY_PLUS) {
    +          continue;
    +        }
    +
    +        wrapper = new DrillCalciteSqlOperatorWrapper(calciteOperator, drillOpName, drillFuncHolders);
    +      }
    +      calciteToWrapper.put(calciteOperator, wrapper);
    +    }
    +  }
    +
    +  private List<DrillFuncHolder> getFunctionListWithInference(String name) {
    +    final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    for(SqlOperator sqlOperator : opMapInferernce.get(name.toLowerCase())) {
    +      if(sqlOperator instanceof DrillSqlOperator) {
    +        final List<DrillFuncHolder> list = ((DrillSqlOperator) sqlOperator).getFunctions();
    +        if(list != null) {
    +          functions.addAll(list);
    +        }
    +      }
    +
    +      if(sqlOperator instanceof DrillSqlAggOperator) {
    +        final List<DrillFuncHolder> list = ((DrillSqlAggOperator) sqlOperator).getFunctions();
    +        if(list != null) {
    +          functions.addAll(list);
    +        }
    +      }
    +    }
    +    return functions;
    +  }
    +
    +  private boolean isEnableInference() {
    --- End diff --
    
    Nit: rename to isInferenceEnabled


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54680335
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,43 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.google.common.base.Preconditions;
    -import org.apache.drill.common.types.TypeProtos.MajorType;
    -import org.apache.drill.common.types.TypeProtos.MinorType;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
    +import java.util.ArrayList;
    +import java.util.List;
    +
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    -
    -  private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, isDeterministic);
       }
     
    -  public DrillSqlOperator(String name, int argCount, MajorType returnType, boolean isDeterminisitic) {
    -    super(new SqlIdentifier(name, SqlParserPos.ZERO), DynamicReturnType.INSTANCE, null, new Checker(argCount), null, SqlFunctionCategory.USER_DEFINED_FUNCTION);
    -    this.returnType = Preconditions.checkNotNull(returnType);
    -    this.isDeterministic = isDeterminisitic;
    +  public DrillSqlOperator(String name, List<DrillFuncHolder> functions, int argCountMin, int argCountMax, boolean isDeterministic) {
    +    super(new SqlIdentifier(name, SqlParserPos.ZERO),
    +        TypeInferenceUtils.getDrillSqlReturnTypeInference(
    --- End diff --
    
    It's good you made to pass in a SqlReturnTypeInference object to DrillSqlOperator.  
    
    However, I have couple of follow-up comments. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54964785
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by amansinha100 <gi...@git.apache.org>.
Github user amansinha100 commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54688304
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    --- End diff --
    
    There ought to be a more declarative way to populate the operator table with these types of functions.  Here you have listed 4 such functions but what if there are a lot more ?  You should only need to call operatorTable.add(arg1, arg2) once inside a loop and pass in different values of arg1, arg2 based on a statically populated structure. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56454823
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    +        this.isDeterministic = isDeterministic;
    +      }
    +      return this;
    +    }
    +
    +    public DrillSqlOperator build() {
    +      return new DrillSqlOperator(
    +          name,
    +          functions,
    +          argCountMin,
    +          argCountMax,
    +          isDeterministic,
    +          TypeInferenceUtils.getDrillSqlReturnTypeInference(
    +              name,
    --- End diff --
    
    Add a check in the build()


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455331
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperatorNotInfer.java ---
    @@ -0,0 +1,44 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class HiveUDFOperatorNotInfer extends HiveUDFOperator {
    --- End diff --
    
    renamed as HiveUDFOperatorWithoutInference


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426779
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    --- End diff --
    
    Nit: the default is to infer. So these function names are confusing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56432884
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    +  private static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  private final TypeProtos.MajorType returnType;
    +
    +  public DrillSqlOperatorNotInfer(String name, int argCount, TypeProtos.MajorType returnType, boolean isDeterminisitic) {
    +    super(name,
    +        new ArrayList< DrillFuncHolder>(),
    +        argCount,
    +        argCount,
    +        isDeterminisitic,
    +        DynamicReturnType.INSTANCE);
    --- End diff --
    
    This is not right. You pass DynamicReturnType.INSTANCE as SqlReturnTypeInference, yet also override inferReturnType. Why do you have to pass DynamicReturnType here? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56453928
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperatorNotInfer.java ---
    @@ -0,0 +1,76 @@
    +/**
    + * 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
    + * <p/>
    + * http://www.apache.org/licenses/LICENSE-2.0
    + * <p/>
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +
    +import java.util.ArrayList;
    +
    +public class DrillSqlOperatorNotInfer extends DrillSqlOperator {
    --- End diff --
    
    Addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54766049
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,110 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
    +  }
    +
    +  public Collection<DrillFuncHolder> getAllMethods() {
    +    return Collections.unmodifiableCollection(registeredFunctions.values());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    -    SqlOperator op;
    -    for (Entry<String, Collection<DrillFuncHolder>> function : methods.asMap().entrySet()) {
    -      Set<Integer> argCounts = Sets.newHashSet();
    -      String name = function.getKey().toUpperCase();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
           for (DrillFuncHolder func : function.getValue()) {
    -        if (argCounts.add(func.getParamCount())) {
    -          if (func.isAggregating()) {
    -            op = new DrillSqlAggOperator(name, func.getParamCount());
    -          } else {
    -            boolean isDeterministic;
    -            // prevent Drill from folding constant functions with types that cannot be materialized
    -            // into literals
    -            if (DrillConstExecutor.NON_REDUCIBLE_TYPES.contains(func.getReturnType().getMinorType())) {
    -              isDeterministic = false;
    -            } else {
    -              isDeterministic = func.isDeterministic();
    -            }
    -            op = new DrillSqlOperator(name, func.getParamCount(), func.getReturnType(), isDeterministic);
    -          }
    -          operatorTable.add(function.getKey(), op);
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumerRange = getArgNumerRange(name, func);
    +          functions.put(argNumerRange, func);
             }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final DrillSqlOperator drillSqlOperator;
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        drillSqlOperator = new DrillSqlOperator(
    +            name,
    +            Lists.newArrayList(entry.getValue()),
    +            min,
    +            max,
    +            isDeterministic);
    +        operatorTable.add(name, drillSqlOperator);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        operatorTable.add(name, new DrillSqlAggOperator(name, Lists.newArrayList(entry.getValue()), entry.getKey()));
           }
         }
    +
    +    registerCalcitePlaceHolderFunction(operatorTable);
    +  }
    +
    +  /**
    +   * These {@link DrillSqlOperator} merely act as a placeholder so that Calcite
    +   * allows convert_to(), convert_from(), flatten(), date_part() functions in SQL.
    +   */
    +  private void registerCalcitePlaceHolderFunction(DrillOperatorTable operatorTable) {
    +    final String convert_to = "CONVERT_TO";
    +    final String convert_from = "CONVERT_FROM";
    +    final String flatten = "FLATTEN";
    +    final String date_part = "DATE_PART";
    +
    +    operatorTable.add(convert_to,
    +        new DrillSqlOperator(convert_to,
    +            2,
    +            true));
    +    operatorTable.add(convert_from,
    +        new DrillSqlOperator(convert_from,
    +            2,
    +            true));
    +    operatorTable.add(flatten,
    +        new DrillSqlOperator(flatten,
    +            1,
    +            true));
    +    operatorTable.add(date_part,
    +        new DrillSqlOperator(date_part,
    +            2,
    +            true));
       }
     
    +  private Pair<Integer, Integer> getArgNumerRange(final String name, final DrillFuncHolder func) {
    +    switch(name.toUpperCase()) {
    +      case "CONCAT":
    +        return Pair.of(1, Integer.MAX_VALUE);
    +
    +      // Drill does not have a FunctionTemplate for the lpad/rpad with two arguments.
    +      // It relies on DrillOptiq.java to add a third dummy argument to be acceptable
    +      // by the FunctionTemplate in StringFunctions.java
    +      case "LPAD":
    +      case "RPAD":
    +        return Pair.of(2, 3);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return Pair.of(1, 2);
    +
    +      // Similar to the reason above, DrillOptiq.java is used for rewritting
    +      case "LENGTH":
    +        return Pair.of(1, 2);
    +
    +      default:
    +        return Pair.of(func.getParamCount(), func.getParamCount());
    +    }
    +  }
    --- End diff --
    
    We support variable arguments for these few cases. For example, you can give one or two arguments for LTRIM, RTRIM, BTRIM. And if you just give one argument, DrillOptiq will fill the second argument for users (see [1]).
    
    However, in order to stop Calcite failing the LTRIM, RTRIM, BTRIM with single argument, we have to either add a dummy function or add the code here
    [1] https://github.com/apache/drill/blob/d7eebec41a1636055be1b2c79b693d76c52d8932/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java#L364


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455347
  
    --- Diff: contrib/storage-hive/core/src/main/java/org/apache/drill/exec/planner/sql/HiveUDFOperatorNotInfer.java ---
    @@ -0,0 +1,44 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class HiveUDFOperatorNotInfer extends HiveUDFOperator {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56523015
  
    --- Diff: contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java ---
    @@ -43,4 +43,32 @@ public void testEncode() throws Exception {
             .baselineValues(new Object[] { null })
             .go();
       }
    +
    +  @Test
    +  public void testReflect() throws Exception {
    +    final String query = "select reflect('java.lang.Math', 'round', cast(2 as float)) as col \n" +
    --- End diff --
    
    BTW, the correct way to verify this is to check the plan produced by your limit 0 shortcut. 
    But apparently, we can enhance that after yours gets in.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54916249
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,43 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.google.common.base.Preconditions;
    -import org.apache.drill.common.types.TypeProtos.MajorType;
    -import org.apache.drill.common.types.TypeProtos.MinorType;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
    +import java.util.ArrayList;
    +import java.util.List;
    +
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    -
    -  private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, isDeterministic);
    --- End diff --
    
    Will file a jira


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54787960
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,43 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.google.common.base.Preconditions;
    -import org.apache.drill.common.types.TypeProtos.MajorType;
    -import org.apache.drill.common.types.TypeProtos.MinorType;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
    +import java.util.ArrayList;
    +import java.util.List;
    +
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    -
    -  private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, isDeterministic);
    --- End diff --
    
    This constructor exists for the legacy reason. In theory, if Drill needs a DrillSqlOperator, it is supposed to go to DrillOperatorTable for pickup. 
    
    However, I think it is because Drill cannot access to DrillOperatorTable at the place where this constructor is being called. An example can be found in [1]. 
    
    (Note rexBuilder.getOpTab() will only give calcite's SqlStdOperatorTable)
    
    [1]
    https://github.com/apache/drill/blob/d7eebec41a1636055be1b2c79b693d76c52d8932/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java#L240
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54680921
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    --- End diff --
    
    Why do you treat boolean type specially here? Sounds like the logic could be applied to other type as wells.  If all the DrillFuncHolder's return type is  BigInt, should we return BigInt? 
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431206
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerForInference(operatorTable);
    +    registerForDefault(operatorTable);
    +  }
    +
    +  public void registerForInference(DrillOperatorTable operatorTable) {
    +    final Map<String, DrillSqlOperator.DrillSqlOperatorBuilder> map = Maps.newHashMap();
    +    final Map<String, DrillSqlAggOperator.DrillSqlAggOperatorBuilder> mapAgg = Maps.newHashMap();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
    +      for (DrillFuncHolder func : function.getValue()) {
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumberRange;
    +          if(drillFuncToRange.containsKey(name)) {
    +            argNumberRange = drillFuncToRange.get(name);
    +          } else {
    +            argNumberRange = Pair.of(func.getParamCount(), func.getParamCount());
    +          }
    +          functions.put(argNumberRange, func);
    +        }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        if(map.containsKey(name)) {
    --- End diff --
    
    Nit: simplify if..else
    ```java
    if (!map.containsKey(name) ) {
      map.put(name, new DrillSqlOperator.DrillSqlOperatorBuilder().setName(name));
    }
    map.get(name)...;
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56704717
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +114,100 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    +    registerOperatorsWithInference(operatorTable);
    +    registerOperatorsWithoutInference(operatorTable);
    +  }
    +
    +  private void registerOperatorsWithInference(DrillOperatorTable operatorTable) {
    +    final Map<String, DrillSqlOperator.DrillSqlOperatorBuilder> map = Maps.newHashMap();
    +    final Map<String, DrillSqlAggOperator.DrillSqlAggOperatorBuilder> mapAgg = Maps.newHashMap();
    +    for (Entry<String, Collection<DrillFuncHolder>> function : registeredFunctions.asMap().entrySet()) {
    +      final ArrayListMultimap<Pair<Integer, Integer>, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = true;
    +      for (DrillFuncHolder func : function.getValue()) {
    +        final int paramCount = func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          final Pair<Integer, Integer> argNumberRange;
    +          if(registeredFuncNameToArgRange.containsKey(name)) {
    +            argNumberRange = registeredFuncNameToArgRange.get(name);
    +          } else {
    +            argNumberRange = Pair.of(func.getParamCount(), func.getParamCount());
    +          }
    +          functions.put(argNumberRange, func);
    +        }
    +
    +        if(!func.isDeterministic()) {
    +          isDeterministic = false;
    +        }
    +      }
    +      for (Entry<Pair<Integer, Integer>, Collection<DrillFuncHolder>> entry : functions.asMap().entrySet()) {
    +        final Pair<Integer, Integer> range = entry.getKey();
    +        final int max = range.getRight();
    +        final int min = range.getLeft();
    +        if(!map.containsKey(name)) {
    +          map.put(name, new DrillSqlOperator.DrillSqlOperatorBuilder()
    +              .setName(name));
    +        }
    +
    +        final DrillSqlOperator.DrillSqlOperatorBuilder drillSqlOperatorBuilder = map.get(name);
    +        drillSqlOperatorBuilder
    +            .addFunctions(entry.getValue())
    +            .setArgumentCount(min, max)
    +            .setDeterministic(isDeterministic);
    +      }
    +      for (Entry<Integer, Collection<DrillFuncHolder>> entry : aggregateFunctions.asMap().entrySet()) {
    +        if(mapAgg.containsKey(name)) {
    --- End diff --
    
    addressed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54922569
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java ---
    @@ -770,6 +770,65 @@ public void eval() {
         } // end of eval
       }
     
    +  /*
    +   * Fill up the string to length 'length' by prepending the character ' ' in the beginning of 'text'.
    +   * If the string is already longer than length, then it is truncated (on the right).
    +   */
    +  @FunctionTemplate(name = "lpad", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
    +  public static class LpadTwoArg implements DrillSimpleFunc {
    --- End diff --
    
    Please add unit test for these functions.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431658
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/PlannerPhase.java ---
    @@ -135,6 +135,16 @@ public RuleSet getRules(OptimizerRulesContext context, Collection<StoragePlugin>
         }
       },
     
    +  SUM_CONVERSION("Convert SUM to $SUM0") {
    --- End diff --
    
    Yes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54923299
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,568 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import com.google.common.collect.Maps;
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +import org.apache.drill.exec.resolver.TypeCastRules;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.DECIMAL, TypeProtos.MinorType.FLOAT8)
    +          .build();
    +
    +  private static Map<String, SqlReturnTypeInference> funcNameToInference = Maps.newHashMap();
    +  static {
    +    funcNameToInference.put("DATE_PART", DrillDatePartSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SUM", DrillSumSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("COUNT", DrillCountSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONCAT", DrillConcatSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LENGTH", DrillLengthSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RPAD", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("LTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("RTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("BTRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("TRIM", DrillPadTrimSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_TO", DrillConvertToSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("EXTRACT", DrillExtractSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("SQRT", DrillSqrtSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CAST", DrillCastSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("FLATTEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("KVGEN", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +    funcNameToInference.put("CONVERT_FROM", DrillDeferToExecSqlReturnTypeInference.INSTANCE);
    +  }
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +
    +    final String nameCap = name.toUpperCase();
    +    if(funcNameToInference.containsKey(nameCap)) {
    +      return funcNameToInference.get(nameCap);
    +    } else {
    +      return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      // This code for boolean output type is added for addressing DRILL-1729
    +      // In summary, if we have a boolean output function in the WHERE-CLAUSE,
    +      // this logic can validate and execute user queries seamlessly
    +      boolean allBooleanOutput = true;
    +      for (DrillFuncHolder function : functions) {
    +        if (function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +          allBooleanOutput = false;
    +          break;
    +        }
    +      }
    +      if (allBooleanOutput) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.BOOLEAN), true);
    +      }
    +
    +      // The following logic is just a safe play:
    +      // Even if any of the input arguments has ANY type,
    +      // it "might" still be possible to determine the return type based on other non-ANY types
    +      for (RelDataType type : opBinding.collectOperandTypes()) {
    +        if (type.getSqlTypeName() == SqlTypeName.ANY) {
    +          return factory.createTypeWithNullability(
    +              factory.createSqlType(SqlTypeName.ANY),
    +              true);
    +        }
    +      }
    +
    +      final DrillFuncHolder func = resolveDrillFuncHolder(opBinding, functions);
    +      final RelDataType returnType = getReturnType(opBinding, func);
    +      return returnType;
    +    }
    +
    +    private static RelDataType getReturnType(final SqlOperatorBinding opBinding, final DrillFuncHolder func) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      // least restrictive type (nullable ANY type)
    +      final RelDataType nullableAnyType = factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +
    +      final TypeProtos.MajorType returnType = func.getReturnType();
    +      if (UNKNOWN_TYPE.equals(returnType)) {
    +        return nullableAnyType;
    +      }
    +
    +      final TypeProtos.MinorType minorType = returnType.getMinorType();
    +      final SqlTypeName sqlTypeName = getCalciteTypeFromDrillType(minorType);
    +      if (sqlTypeName == null) {
    +        return nullableAnyType;
    +      }
    +
    +      final boolean isNullable;
    +      switch (returnType.getMode()) {
    +        case REPEATED:
    +        case OPTIONAL:
    +          isNullable = true;
    +          break;
    +
    +        case REQUIRED:
    +          switch (func.getNullHandling()) {
    +            case INTERNAL:
    +              isNullable = false;
    +              break;
    +
    +            case NULL_IF_NULL:
    +              boolean isNull = false;
    +              for (int i = 0; i < opBinding.getOperandCount(); ++i) {
    +                if (opBinding.getOperandType(i).isNullable()) {
    +                  isNull = true;
    +                  break;
    +                }
    +              }
    +
    +              isNullable = isNull;
    +              break;
    +            default:
    +              throw new UnsupportedOperationException();
    +          }
    +          break;
    +
    +        default:
    +          throw new UnsupportedOperationException();
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDeferToExecSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDeferToExecSqlReturnTypeInference INSTANCE = new DrillDeferToExecSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      return factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.ANY),
    +          true);
    +    }
    +  }
    +
    +  private static class DrillSumSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillSumSqlReturnTypeInference INSTANCE = new DrillSumSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      // If there is group-by and the imput type is Non-nullable,
    +      // the output is Non-nullable;
    +      // Otherwise, the output is nullable.
    +      final boolean isNullable = opBinding.getGroupCount() == 0
    +          || opBinding.getOperandType(0).isNullable();
    +
    +      final SqlTypeName sqlTypeName = opBinding.getOperandType(0).getSqlTypeName();
    +      if(sqlTypeName == SqlTypeName.ANY) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.ANY,
    +            isNullable);
    +      }
    +
    +      final TypeProtos.MinorType inputMinorType = getDrillTypeFromCalciteType(opBinding.getOperandType(0));
    +      if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.BIGINT))
    +          == TypeProtos.MinorType.BIGINT) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.BIGINT,
    +            isNullable);
    +      } else if(TypeCastRules.getLeastRestrictiveType(Lists.newArrayList(inputMinorType, TypeProtos.MinorType.FLOAT8))
    +          == TypeProtos.MinorType.FLOAT8) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(
    +            factory,
    +            SqlTypeName.DOUBLE,
    +            isNullable);
    +      } else {
    +        throw UserException
    +            .functionError()
    +            .message(String.format("%s does not support operand types (%s)",
    +                opBinding.getOperator().getName(),
    +                sqlTypeName))
    +            .build(logger);
    +      }
    +    }
    +  }
    +
    +  private static class DrillCountSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillCountSqlReturnTypeInference INSTANCE = new DrillCountSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName type = SqlTypeName.BIGINT;
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          type,
    +          false);
    +    }
    +  }
    +
    +  private static class DrillConcatSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillConcatSqlReturnTypeInference INSTANCE = new DrillConcatSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      boolean isNullable = true;
    +      int precision = 0;
    +      for(RelDataType relDataType : opBinding.collectOperandTypes()) {
    +        if(!relDataType.isNullable()) {
    +          isNullable = false;
    +        }
    +
    +        // If the underlying columns cannot offer information regarding the precision (i.e., the length) of the VarChar,
    +        // Drill uses the largest to represent it
    +        if(relDataType.getPrecision() == TypeHelper.VARCHAR_DEFAULT_CAST_LEN
    +            || relDataType.getPrecision() == RelDataType.PRECISION_NOT_SPECIFIED) {
    +          precision = TypeHelper.VARCHAR_DEFAULT_CAST_LEN;
    +        } else {
    +          precision += relDataType.getPrecision();
    +        }
    +      }
    +
    +      return factory.createTypeWithNullability(
    +          factory.createSqlType(SqlTypeName.VARCHAR, precision),
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillLengthSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillLengthSqlReturnTypeInference INSTANCE = new DrillLengthSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName sqlTypeName = SqlTypeName.BIGINT;
    +
    +      // We need to check only the first argument because
    +      // the second one is used to represent encoding type
    +      final boolean isNullable = opBinding.getOperandType(0).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillPadTrimSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillPadTrimSqlReturnTypeInference INSTANCE = new DrillPadTrimSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName sqlTypeName = SqlTypeName.VARCHAR;
    +
    +      for(int i = 0; i < opBinding.getOperandCount(); ++i) {
    +        if(opBinding.getOperandType(i).isNullable()) {
    +          return DrillConstExecutor.createCalciteTypeWithNullability(
    +              factory, sqlTypeName, true);
    +        }
    +      }
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory, sqlTypeName, false);
    +    }
    +  }
    +
    +  private static class DrillConvertToSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillConvertToSqlReturnTypeInference INSTANCE = new DrillConvertToSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final SqlTypeName type = SqlTypeName.VARBINARY;
    +
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory, type, opBinding.getOperandType(0).isNullable());
    +    }
    +  }
    +
    +  private static class DrillExtractSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillExtractSqlReturnTypeInference INSTANCE = new DrillExtractSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final TimeUnit timeUnit = opBinding.getOperandType(0).getIntervalQualifier().getStartUnit();
    +      final boolean isNullable = opBinding.getOperandType(1).isNullable();
    +
    +      final SqlTypeName sqlTypeName = getSqlTypeNameForTimeUnit(timeUnit.name());
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillSqrtSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillSqrtSqlReturnTypeInference INSTANCE = new DrillSqrtSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final boolean isNullable = opBinding.getOperandType(0).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          SqlTypeName.DOUBLE,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillDatePartSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillDatePartSqlReturnTypeInference INSTANCE = new DrillDatePartSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +
    +      final SqlNode firstOperand = ((SqlCallBinding) opBinding).operand(0);
    +      if(!(firstOperand instanceof SqlCharStringLiteral)) {
    +        return DrillConstExecutor.createCalciteTypeWithNullability(factory,
    +            SqlTypeName.ANY,
    +            opBinding.getOperandType(1).isNullable());
    +      }
    +
    +      final String part = ((SqlCharStringLiteral) firstOperand)
    +          .getNlsString()
    +          .getValue()
    +          .toUpperCase();
    +
    +      final SqlTypeName sqlTypeName = getSqlTypeNameForTimeUnit(part);
    +      final boolean isNullable = opBinding.getOperandType(1).isNullable();
    +      return DrillConstExecutor.createCalciteTypeWithNullability(
    +          factory,
    +          sqlTypeName,
    +          isNullable);
    +    }
    +  }
    +
    +  private static class DrillCastSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private static DrillCastSqlReturnTypeInference INSTANCE = new DrillCastSqlReturnTypeInference();
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      final boolean isNullable = opBinding
    +          .getOperandType(0)
    +          .isNullable();
    +
    +      RelDataType ret = factory.createTypeWithNullability(
    +          opBinding.getOperandType(1),
    +          isNullable);
    +
    +      if (opBinding instanceof SqlCallBinding) {
    +        SqlCallBinding callBinding = (SqlCallBinding) opBinding;
    +        SqlNode operand0 = callBinding.operand(0);
    +
    +        // dynamic parameters and null constants need their types assigned
    +        // to them using the type they are casted to.
    +        if (((operand0 instanceof SqlLiteral)
    +                && (((SqlLiteral) operand0).getValue() == null))
    +                || (operand0 instanceof SqlDynamicParam)) {
    +          callBinding.getValidator().setValidatedNodeType(
    +                  operand0,
    +                  ret);
    +        }
    +      }
    +
    +      return ret;
    +    }
    +  }
    +
    +  private static DrillFuncHolder resolveDrillFuncHolder(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    final List<LogicalExpression> args = Lists.newArrayList();
    +    for (final RelDataType type : opBinding.collectOperandTypes()) {
    +      final TypeProtos.MinorType minorType = getDrillTypeFromCalciteType(type);
    +      final TypeProtos.MajorType majorType;
    +      if (type.isNullable()) {
    +        majorType =  Types.optional(minorType);
    +      } else {
    +        majorType = Types.required(minorType);
    +      }
    +
    +      args.add(new MajorTypeInLogicalExpression(majorType));
    +    }
    +    final FunctionCall functionCall = new FunctionCall(opBinding.getOperator().getName(), args, ExpressionPosition.UNKNOWN);
    +    final FunctionResolver functionResolver = FunctionResolverFactory.getResolver();
    +    final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
    +
    +    // Throw an exception
    +    // if no DrillFuncHolder matched for the given list of operand types
    +    if(func == null) {
    +      String operandTypes = "";
    +      for(int i = 0; i < opBinding.getOperandCount(); ++i) {
    +        operandTypes += opBinding.getOperandType(i).getSqlTypeName();
    +        if(i < opBinding.getOperandCount() - 1) {
    +          operandTypes += ",";
    +        }
    +      }
    +
    +      throw UserException
    +          .functionError()
    +          .message(String.format("%s does not support operand types (%s)",
    +              opBinding.getOperator().getName(),
    +              operandTypes))
    +          .build(logger);
    +    }
    +    return func;
    +  }
    +
    +  /**
    +   * This class is not intended to be initiated
    --- End diff --
    
    instantiated


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431197
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,66 @@
      */
     package org.apache.drill.exec.expr.fn;
     
    -import java.util.Arrays;
     import java.util.Collection;
    +import java.util.Collections;
     import java.util.HashMap;
    -import java.util.HashSet;
     import java.util.List;
    +import java.util.Map;
     import java.util.Map.Entry;
     import java.util.Set;
     
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +import com.google.common.collect.Maps;
    +import com.google.common.collect.Sets;
     import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.commons.lang3.tuple.Pair;
     import org.apache.drill.common.scanner.persistence.AnnotatedClassDescriptor;
     import org.apache.drill.common.scanner.persistence.ScanResult;
    -import org.apache.drill.exec.expr.DrillFunc;
    +import org.apache.drill.common.types.TypeProtos;
     import org.apache.drill.exec.planner.logical.DrillConstExecutor;
     import org.apache.drill.exec.planner.sql.DrillOperatorTable;
     import org.apache.drill.exec.planner.sql.DrillSqlAggOperator;
    +import org.apache.drill.exec.planner.sql.DrillSqlAggOperatorNotInfer;
     import org.apache.drill.exec.planner.sql.DrillSqlOperator;
     
     import com.google.common.collect.ArrayListMultimap;
    -import com.google.common.collect.Sets;
    +import org.apache.drill.exec.planner.sql.DrillSqlOperatorNotInfer;
     
    +/**
    + * Registry of Drill functions.
    + */
     public class DrillFunctionRegistry {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  private ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  // key: function name (lowercase) value: list of functions with that name
    +  private final ArrayListMultimap<String, DrillFuncHolder> registeredFunctions = ArrayListMultimap.create();
     
    -  /* Hash map to prevent registering functions with exactly matching signatures
    -   * key: Function Name + Input's Major Type
    -   * Value: Class name where function is implemented
    -   */
    -  private HashMap<String, String> functionSignatureMap = new HashMap<>();
    +  private static final ImmutableMap<String, Pair<Integer, Integer>> drillFuncToRange = ImmutableMap.<String, Pair<Integer, Integer>> builder()
    --- End diff --
    
    Nit: rename


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431148
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java ---
    @@ -17,47 +17,76 @@
      */
     package org.apache.drill.exec.planner.sql;
     
    -import java.util.List;
    -
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import com.google.common.collect.Lists;
     import org.apache.calcite.sql.SqlAggFunction;
    -import org.apache.calcite.sql.SqlCall;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
     import org.apache.calcite.sql.SqlKind;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
    -import com.google.common.collect.ImmutableList;
    +import java.util.Collection;
    +import java.util.List;
     
     public class DrillSqlAggOperator extends SqlAggFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);
    +  // private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);
    +  private final List<DrillFuncHolder> functions;
     
    -
    -  public DrillSqlAggOperator(String name, int argCount) {
    -    super(name, new SqlIdentifier(name, SqlParserPos.ZERO), SqlKind.OTHER_FUNCTION, DynamicReturnType.INSTANCE, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
    +  protected DrillSqlAggOperator(String name, List<DrillFuncHolder> functions, int argCountMin, int argCountMax, SqlReturnTypeInference sqlReturnTypeInference) {
    +    super(name,
    +        new SqlIdentifier(name, SqlParserPos.ZERO),
    +        SqlKind.OTHER_FUNCTION,
    +        sqlReturnTypeInference,
    +        null,
    +        Checker.getChecker(argCountMin, argCountMax),
    +        SqlFunctionCategory.USER_DEFINED_FUNCTION);
    +    this.functions = functions;
       }
     
    -  @Override
    -  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
    -    return getAny(validator.getTypeFactory());
    +  private DrillSqlAggOperator(String name, List<DrillFuncHolder> functions, int argCountMin, int argCountMax) {
    +    this(name,
    +        functions,
    +        argCountMin,
    +        argCountMax,
    +        TypeInferenceUtils.getDrillSqlReturnTypeInference(
    +            name,
    +            functions));
       }
     
    -  private RelDataType getAny(RelDataTypeFactory factory){
    -    return factory.createSqlType(SqlTypeName.ANY);
    -//    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
    +  public List<DrillFuncHolder> getFunctions() {
    +    return functions;
       }
     
    -//  @Override
    -//  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
    -//    return ImmutableList.of(typeFactory.createSqlType(SqlTypeName.ANY));
    -//  }
    -//
    -//  @Override
    -//  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
    -//    return getAny(typeFactory);
    -//  }
    +  public static class DrillSqlAggOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    --- End diff --
    
    unused?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56456528
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java ---
    @@ -40,7 +43,16 @@
     public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
     
       private final SqlAvgAggFunction.Subtype subtype;
    -  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false);
    +  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false,
    +      new SqlReturnTypeInference() {
    +        @Override
    +        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +          return TypeInferenceUtils.createCalciteTypeWithNullability(
    +              opBinding.getTypeFactory(),
    +              SqlTypeName.ANY,
    --- End diff --
    
    1.This is the legacy code. In theory, we should not create DrillSqlOperator outside of DrillFunctionRegistry. 
    
    2. Convertlet follows after validation.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56455656
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,23 +118,106 @@ public DrillFunctionRegistry(ScanResult classpathScan) {
       }
     
       public int size(){
    -    return methods.size();
    +    return registeredFunctions.size();
       }
     
       /** Returns functions with given name. Function name is case insensitive. */
       public List<DrillFuncHolder> getMethods(String name) {
    -    return this.methods.get(name.toLowerCase());
    +    return this.registeredFunctions.get(name.toLowerCase());
       }
     
       public void register(DrillOperatorTable operatorTable) {
    --- End diff --
    
    It is for safely falling back. The more we diverge from the original mechanism, the more risky we cannot fall back to the original behavior. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56397383
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -43,24 +46,49 @@
     public class DrillOperatorTable extends SqlStdOperatorTable {
     //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
       private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
    -  private List<SqlOperator> operators = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsCalcite = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsDefault = Lists.newArrayList();
    +  private final List<SqlOperator> operatorsInferernce = Lists.newArrayList();
       private final Map<SqlOperator, SqlOperator> calciteToWrapper = Maps.newIdentityHashMap();
    -  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
    +
    +  private final ArrayListMultimap<String, SqlOperator> opMapDefault = ArrayListMultimap.create();
    +  private final ArrayListMultimap<String, SqlOperator> opMapInferernce = ArrayListMultimap.create();
    +
    +  private final SystemOptionManager systemOptionManager;
     
       public DrillOperatorTable(FunctionImplementationRegistry registry) {
    +    this(registry, null);
    --- End diff --
    
    Why do we have this constructor, with systemOptionManager = null? Can we remove it ? 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56431255
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillAvgVarianceConvertlet.java ---
    @@ -40,7 +43,16 @@
     public class DrillAvgVarianceConvertlet implements SqlRexConvertlet {
     
       private final SqlAvgAggFunction.Subtype subtype;
    -  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false);
    +  private static final DrillSqlOperator CastHighOp = new DrillSqlOperator("CastHigh", 1, false,
    +      new SqlReturnTypeInference() {
    +        @Override
    +        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +          return TypeInferenceUtils.createCalciteTypeWithNullability(
    +              opBinding.getTypeFactory(),
    +              SqlTypeName.ANY,
    --- End diff --
    
    (1) Can the return type be inferred based on TypeCastRules?
    (2) Are *Convertlet part of validation phase? If not, what guarantees that return type at validation and at execution are same?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by hsuanyi <gi...@git.apache.org>.
Github user hsuanyi commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54915844
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    --- End diff --
    
    I used Map to structure it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56458191
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -140,6 +140,10 @@ public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
         }
     
         public DrillSqlOperator build() {
    +      if(name == null || functions.isEmpty()) {
    +        throw new AssertionError("The fields, name and functions, need to be set before build DrillSqlAggOperator");
    --- End diff --
    
    Why assertionError here? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on the pull request:

    https://github.com/apache/drill/pull/397#issuecomment-198407062
  
    Looks good to me. 
    
    +1
    
    Some additional info
    
    1. This PR has added an option to enable/disable the type inference feature (by default, it's enabled). 
    2. Internal performance shows this type inference feature, combined with "limit 0" optimization,  shows huge improvement for "limit 0" query, some time with orders of magnitude. 
    3. It has passed the pre-commit test with option enabled/disabled.
    4. It has addressed issues found by MapR QA folks during their extensive testing.
    5. @hsuanyi will add a doc to the JIRA, showing the scope where the type inference feature would or would not be applied.  
    6. As a side note, this PR requires one patch in Drill forked Calcite. The change is needed because of the star column changes specifically made for Drill in forked Calcite.  For now, it's not feasible to push this change to Calcite master, until the rebasing work is done.
    
    @jacques-n , @amansinha100 , @sudheeshkatkam , if you guys have any comment or concern, please let @hsuanyi know. If no concern, I'm going to merge this PR to master branch. 
      


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54761017
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,571 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + * http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.drill.exec.planner.sql;
    +
    +import com.google.common.collect.ImmutableMap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.rel.type.RelDataTypeFactory;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlCharStringLiteral;
    +import org.apache.calcite.sql.SqlDynamicParam;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +
    +import org.apache.drill.common.expression.ExpressionPosition;
    +import org.apache.drill.common.expression.FunctionCall;
    +import org.apache.drill.common.expression.LogicalExpression;
    +import org.apache.drill.common.expression.MajorTypeInLogicalExpression;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.common.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.TypeHelper;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.planner.logical.DrillConstExecutor;
    +import org.apache.drill.exec.resolver.FunctionResolver;
    +import org.apache.drill.exec.resolver.FunctionResolverFactory;
    +
    +import java.util.List;
    +
    +public class TypeInferenceUtils {
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TypeInferenceUtils.class);
    +
    +  public static final TypeProtos.MajorType UNKNOWN_TYPE = TypeProtos.MajorType.getDefaultInstance();
    +  private static ImmutableMap<TypeProtos.MinorType, SqlTypeName> DRILL_TO_CALCITE_TYPE_MAPPING =
    +      ImmutableMap.<TypeProtos.MinorType, SqlTypeName> builder()
    +          .put(TypeProtos.MinorType.INT, SqlTypeName.INTEGER)
    +          .put(TypeProtos.MinorType.BIGINT, SqlTypeName.BIGINT)
    +          .put(TypeProtos.MinorType.FLOAT4, SqlTypeName.FLOAT)
    +          .put(TypeProtos.MinorType.FLOAT8, SqlTypeName.DOUBLE)
    +          .put(TypeProtos.MinorType.VARCHAR, SqlTypeName.VARCHAR)
    +          .put(TypeProtos.MinorType.BIT, SqlTypeName.BOOLEAN)
    +          .put(TypeProtos.MinorType.DATE, SqlTypeName.DATE)
    +          .put(TypeProtos.MinorType.DECIMAL9, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL18, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL28SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.DECIMAL38SPARSE, SqlTypeName.DECIMAL)
    +          .put(TypeProtos.MinorType.TIME, SqlTypeName.TIME)
    +          .put(TypeProtos.MinorType.TIMESTAMP, SqlTypeName.TIMESTAMP)
    +          .put(TypeProtos.MinorType.VARBINARY, SqlTypeName.VARBINARY)
    +          .put(TypeProtos.MinorType.INTERVALYEAR, SqlTypeName.INTERVAL_YEAR_MONTH)
    +          .put(TypeProtos.MinorType.INTERVALDAY, SqlTypeName.INTERVAL_DAY_TIME)
    +          .put(TypeProtos.MinorType.MAP, SqlTypeName.MAP)
    +          .put(TypeProtos.MinorType.LIST, SqlTypeName.ARRAY)
    +          .put(TypeProtos.MinorType.LATE, SqlTypeName.ANY)
    +          // These are defined in the Drill type system but have been turned off for now
    +          // .put(TypeProtos.MinorType.TINYINT, SqlTypeName.TINYINT)
    +          // .put(TypeProtos.MinorType.SMALLINT, SqlTypeName.SMALLINT)
    +          // Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    +      ImmutableMap.<SqlTypeName, TypeProtos.MinorType> builder()
    +          .put(SqlTypeName.INTEGER, TypeProtos.MinorType.INT)
    +          .put(SqlTypeName.BIGINT, TypeProtos.MinorType.BIGINT)
    +          .put(SqlTypeName.FLOAT, TypeProtos.MinorType.FLOAT4)
    +          .put(SqlTypeName.DOUBLE, TypeProtos.MinorType.FLOAT8)
    +          .put(SqlTypeName.VARCHAR, TypeProtos.MinorType.VARCHAR)
    +          .put(SqlTypeName.BOOLEAN, TypeProtos.MinorType.BIT)
    +          .put(SqlTypeName.DATE, TypeProtos.MinorType.DATE)
    +          .put(SqlTypeName.TIME, TypeProtos.MinorType.TIME)
    +          .put(SqlTypeName.TIMESTAMP, TypeProtos.MinorType.TIMESTAMP)
    +          .put(SqlTypeName.VARBINARY, TypeProtos.MinorType.VARBINARY)
    +          .put(SqlTypeName.INTERVAL_YEAR_MONTH, TypeProtos.MinorType.INTERVALYEAR)
    +          .put(SqlTypeName.INTERVAL_DAY_TIME, TypeProtos.MinorType.INTERVALDAY)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +
    +          // The following types are not added due to a variety of reasons:
    +          // (1) Disabling decimal type
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL9)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL18)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL28SPARSE)
    +          //.put(SqlTypeName.DECIMAL, TypeProtos.MinorType.DECIMAL38SPARSE)
    +
    +          // (2) These 2 types are defined in the Drill type system but have been turned off for now
    +          // .put(SqlTypeName.TINYINT, TypeProtos.MinorType.TINYINT)
    +          // .put(SqlTypeName.SMALLINT, TypeProtos.MinorType.SMALLINT)
    +
    +          // (3) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          // .put(SqlTypeName.MAP, TypeProtos.MinorType.MAP)
    +          // .put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .build();
    +
    +  /**
    +   * Given a Drill's TypeProtos.MinorType, return a Calcite's corresponding SqlTypeName
    +   */
    +  public static SqlTypeName getCalciteTypeFromDrillType(final TypeProtos.MinorType type) {
    +    return DRILL_TO_CALCITE_TYPE_MAPPING.get(type);
    +  }
    +
    +  /**
    +   * Given a Calcite's RelDataType, return a Drill's corresponding TypeProtos.MinorType
    +   */
    +  public static TypeProtos.MinorType getDrillTypeFromCalciteType(final RelDataType relDataType) {
    +    final SqlTypeName sqlTypeName = relDataType.getSqlTypeName();
    +    TypeProtos.MinorType minorType = CALCITE_TO_DRILL_MAPPING.get(sqlTypeName);
    +    if(minorType == null) {
    +      minorType = TypeProtos.MinorType.LATE;
    +    }
    +    return minorType;
    +  }
    +
    +  /**
    +   * Give the name and DrillFuncHolder list, return the inference mechanism.
    +   */
    +  public static SqlReturnTypeInference getDrillSqlReturnTypeInference(
    +      final String name,
    +      final List<DrillFuncHolder> functions) {
    +    switch(name.toUpperCase()) {
    +      case "DATE_PART":
    +        return DrillDatePartSqlReturnTypeInference.INSTANCE;
    +
    +      case "SUM":
    +        return new DrillSumSqlReturnTypeInference(functions);
    +
    +      case "COUNT":
    +        return DrillCountSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONCAT":
    +        return DrillConcatSqlReturnTypeInference.INSTANCE;
    +
    +      case "LENGTH":
    +        return DrillLengthSqlReturnTypeInference.INSTANCE;
    +
    +      case "LPAD":
    +      case "RPAD":
    +      case "LTRIM":
    +      case "RTRIM":
    +      case "BTRIM":
    +        return DrillPadTrimSqlReturnTypeInference.INSTANCE;
    +
    +      case "CONVERT_TO":
    +        return DrillConvertToSqlReturnTypeInference.INSTANCE;
    +
    +      case "EXTRACT":
    +        return DrillExtractSqlReturnTypeInference.INSTANCE;
    +
    +      case "CAST":
    +        return DrillCastSqlReturnTypeInference.INSTANCE;
    +
    +      case "FLATTEN":
    +      case "KVGEN":
    +      case "CONVERT_FROM":
    +        return DrillDeferToExecSqlReturnTypeInference.INSTANCE;
    +
    +      default:
    +        return new DrillDefaultSqlReturnTypeInference(functions);
    +    }
    +  }
    +
    +  private static class DrillDefaultSqlReturnTypeInference implements SqlReturnTypeInference {
    +    private final List<DrillFuncHolder> functions;
    +
    +    public DrillDefaultSqlReturnTypeInference(List<DrillFuncHolder> functions) {
    +      this.functions = functions;
    +    }
    +
    +    @Override
    +    public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +      final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +      if (functions.isEmpty()) {
    +        return factory.createTypeWithNullability(
    +            factory.createSqlType(SqlTypeName.ANY),
    +            true);
    +      }
    +
    +      boolean allBooleanOutput = true;
    --- End diff --
    
    Make sense. 
    
    Could you add comment explaining why boolean is handled specially here and those logic has been in Drill for long time  (introduced in DRILL-XXXX)? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by sudheeshkatkam <gi...@git.apache.org>.
Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56426751
  
    --- Diff: contrib/storage-hive/core/src/test/java/org/apache/drill/exec/fn/hive/TestInbuiltHiveUDFs.java ---
    @@ -43,4 +43,32 @@ public void testEncode() throws Exception {
             .baselineValues(new Object[] { null })
             .go();
       }
    +
    +  @Test
    +  public void testReflect() throws Exception {
    +    final String query = "select reflect('java.lang.Math', 'round', cast(2 as float)) as col \n" +
    --- End diff --
    
    How are these queries testing inference?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56458894
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -64,4 +108,47 @@ public boolean isDeterministic() {
       public List<DrillFuncHolder> getFunctions() {
         return functions;
       }
    +
    +  public static class DrillSqlOperatorBuilder {
    +    private String name;
    +    private final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    private int argCountMin = Integer.MAX_VALUE;
    +    private int argCountMax = Integer.MIN_VALUE;
    +    private boolean isDeterministic = true;
    +
    +    public DrillSqlOperatorBuilder setName(final String name) {
    +      this.name = name;
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder addFunctions(Collection<DrillFuncHolder> functions) {
    +      this.functions.addAll(functions);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setArgumentCount(final int argCountMin, final int argCountMax) {
    +      this.argCountMin = Math.min(this.argCountMin, argCountMin);
    +      this.argCountMax = Math.max(this.argCountMax, argCountMax);
    +      return this;
    +    }
    +
    +    public DrillSqlOperatorBuilder setDeterministic(boolean isDeterministic) {
    +      if(this.isDeterministic) {
    --- End diff --
    
    In that case, pls add some comments here. Otherwise, the code is kind of confusing; people would have hard time to understand later on. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r56524249
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java ---
    @@ -364,4 +401,35 @@ private static SchemaPlus rootSchema(SchemaPlus schema) {
         }
       }
     
    +  private static class DrillRexBuilder extends RexBuilder {
    +    private DrillRexBuilder(RelDataTypeFactory typeFactory) {
    +      super(typeFactory);
    +    }
    +
    +    @Override
    +    public RexNode ensureType(
    +        RelDataType type,
    +        RexNode node,
    +        boolean matchNullability) {
    +      RelDataType targetType = type;
    +      if (matchNullability) {
    +        targetType = matchNullability(type, node);
    +      }
    +      if (targetType.getSqlTypeName() == SqlTypeName.ANY) {
    +        return node;
    +      }
    +      if (!node.getType().equals(targetType)) {
    +        if(!targetType.isStruct()) {
    +          final RelDataType anyType = TypeInferenceUtils.createCalciteTypeWithNullability(
    --- End diff --
    
    Cast an expression to "any" type? why do we have to add this? On line 418, in Calcite, the code's intention is to skip CAST to "any", since it does not make sense to cast to "any". Why do you want to explicitly add this here?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request: Drill 4372 review

Posted by jinfengni <gi...@git.apache.org>.
Github user jinfengni commented on a diff in the pull request:

    https://github.com/apache/drill/pull/397#discussion_r54804039
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,43 @@
     
     package org.apache.drill.exec.planner.sql;
     
    -import com.google.common.base.Preconditions;
    -import org.apache.drill.common.types.TypeProtos.MajorType;
    -import org.apache.drill.common.types.TypeProtos.MinorType;
    -import org.apache.calcite.rel.type.RelDataType;
    -import org.apache.calcite.rel.type.RelDataTypeFactory;
    -import org.apache.calcite.sql.SqlCall;
    +import java.util.ArrayList;
    +import java.util.List;
    +
     import org.apache.calcite.sql.SqlFunction;
     import org.apache.calcite.sql.SqlFunctionCategory;
     import org.apache.calcite.sql.SqlIdentifier;
    -import org.apache.calcite.sql.SqlOperatorBinding;
     import org.apache.calcite.sql.parser.SqlParserPos;
    -import org.apache.calcite.sql.type.SqlTypeName;
    -import org.apache.calcite.sql.validate.SqlValidator;
    -import org.apache.calcite.sql.validate.SqlValidatorScope;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    -
    -  private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, new ArrayList<DrillFuncHolder>(), argCount, argCount, isDeterministic);
    --- End diff --
    
    I can see the legacy reason for such code. For now, it's fine to leave it as is. But we need open JIRA as a follow-up work to clean this legacy code.  We should not create DrillSqlOperator using this constructor; in stead, it should go through DrillOperatorTable to find the registered one. Otherwise, there would be two versions of DrillSqlOperators for "castHigh". One is the normally registered, the other one is the one created internally. Such different version would potentially cause problem down the road.  We should clean up those legacy code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---