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/17 01:31:48 UTC

[GitHub] drill pull request: Drill 4372 for review

GitHub user hsuanyi opened a pull request:

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

    Drill 4372 for review

    @jinfengni, @amansinha100  
    I just updated and rebased the initial patch. Can you start review it?

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

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

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

    https://github.com/apache/drill/pull/377.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 #377
    
----
commit 9f6fdca69ef3c652bce0961dd76ad8bbdf1cfaa9
Author: Sudheesh Katkam <sk...@maprtech.com>
Date:   2015-12-22T04:38:59Z

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

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

    DRILL-4372: Expose the functions return type to 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 for 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/377#discussion_r53537111
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - 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.MAP, TypeProtos.MinorType.MAP)
    +          //.put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          // (2) Avoid late binding
    +          .put(SqlTypeName.ANY, TypeProtos.MinorType.LATE)
    +          // (3) 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)
    +          // (4) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
       // This is a list of all types that cannot be folded at planning time for various reasons, most of the types are
       // currently not supported at all. The reasons for the others can be found in the evaluation code in the reduce method
       public static final List<Object> NON_REDUCIBLE_TYPES = ImmutableList.builder().add(
           // cannot represent this as a literal according to calcite
    -      TypeProtos.MinorType.INTERVAL,
    +      // TypeProtos.MinorType.INTERVAL,
    --- End diff --
    
    Why 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 for 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/377#discussion_r53537084
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -75,7 +82,7 @@
         NULL_IF_NULL;
       }
     
    -  public static enum FunctionScope {
    +  enum FunctionScope {
    --- End diff --
    
    Can you remove "public static" on NullHandling enum 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 for 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/377#discussion_r53538950
  
    --- Diff: logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java ---
    @@ -43,30 +43,61 @@
     import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
     import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
     
    +/**
    + * {@link LogicalExpression Logical expression} visitor.
    + * @param <T> Return value type
    + * @param <VAL> Contextual value type
    + * @param <EXCEP> Exception type
    + */
     public interface ExprVisitor<T, VAL, EXCEP extends Exception> {
    --- End diff --
    
    Can you undo this change (leave/ improve the doc)?


---
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 for 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/377#discussion_r53755712
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -109,6 +116,16 @@ public int getValue() {
         public static FunctionCostCategory getDefault() {
           return SIMPLE;
         }
    +  }
     
    +  /**
    +   * The number of argument(s) for a function
    +   *
    +   * This annotation element tells if the number of argument(s) is fixed or arbitrary. Most functions are supposed to
    +   * have fixed number of argument(s); One of the exceptions is String Concatenation function (i.e., concat(...)), which
    +   * can take as many arguments as users would like.
    +   */
    +  enum FunctionArgumentNumber {
    +    FIXED, VARIABLE
    --- 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 for 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/377#discussion_r53538957
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    --- End diff --
    
    guava's ImmutableSet?


---
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 for 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/377#discussion_r53561411
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,17 +63,30 @@
     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.logical.DrillParseContext;
    +import org.apache.drill.exec.planner.physical.PrelUtil;
     
    +import java.util.Arrays;
     import java.util.List;
    +import java.util.Map;
     
    -public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    +import static org.apache.calcite.util.Static.RESOURCE;
    --- 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 for 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/377#discussion_r53811472
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - 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.MAP, TypeProtos.MinorType.MAP)
    --- 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 for review

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

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


---
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 for 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/377#discussion_r53540918
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertFrom implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    +  @Param VarCharHolder in2;
       @Output VarBinaryHolder out;
    --- End diff --
    
    To honest, I was struggling at this point a lot. Since this is a dummy function whose real implementation won't be determined until DrillOptiq. Thus, the return type can be just chosen arbitrarily. 
    
    How about letting me add the above as comment in 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 for 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/377#discussion_r53541208
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,46 @@ 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<Integer, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = false;
           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.getFunctionArgumentNumber() == FunctionArgumentNumber.VARIABLE ? -1 : func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          functions.put(paramCount, func);
    +        }
    +        // 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;
    --- End diff --
    
    Sure, I can fix this. Basically, as function registration, we should just separate them into two groups.


---
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 for 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/377#discussion_r53533236
  
    --- Diff: exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java ---
    @@ -31,6 +31,48 @@
     
     public class ${className} {
     
    +/* Dummy function template to allow Optiq to validate this function call.
    + * At DrillOptiq time we rewrite all date_part() functions to extract functions,
    + * since they are essentially the same
    + */
    +@FunctionTemplate(name = "DATE_PART", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
    --- End diff --
    
    This is one of the dummy functions we were discussing above. I think this dummy might look less meaningful if it is checked-in alone.


---
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 for 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/377#discussion_r53537609
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertFrom implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    +  @Param VarCharHolder in2;
       @Output VarBinaryHolder out;
    --- End diff --
    
    I think it's not right to put VarBinary as output type for convert_from() function. 


---
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 for 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/377#discussion_r53522848
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java ---
    @@ -216,6 +217,20 @@ public void eval() {
       }
     
       @FunctionTemplate(names = {"char_length", "character_length", "length"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
    +  public static class DummyCharLength implements DrillSimpleFunc {
    --- End diff --
    
    Why do we add this new "char_length" implementation with empty body? 


---
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 for 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/377#discussion_r53538945
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/store/sys/FunctionIterator.java ---
    @@ -0,0 +1,77 @@
    +/**
    + * 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.store.sys;
    +
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.annotations.FunctionTemplate;
    +import org.apache.drill.exec.expr.fn.DrillFuncHolder;
    +import org.apache.drill.exec.ops.FragmentContext;
    +
    +import java.util.Arrays;
    +import java.util.Iterator;
    +
    +/**
    + *
    + */
    +public class FunctionIterator implements Iterator<Object> {
    --- End diff --
    
    This system table is quite useful, but this implementation is not the best way to present them.
    
    Again, I am responsible for this; it was WIP.


---
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 for 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/377#discussion_r53537811
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_to", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertTo implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    --- End diff --
    
    convert_to should take arbitrary type, and convert to binary type. Why does it only accept for VarChar? 


---
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 for 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/377#discussion_r53538435
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillExtractConvertlet.java ---
    @@ -51,17 +53,38 @@ public RexNode convertCall(SqlRexContext cx, SqlCall call) {
         final List<RexNode> exprs = new LinkedList<>();
     
         RelDataTypeFactory typeFactory = cx.getTypeFactory();
    -
    -    //RelDataType nullableReturnType =
    -
         for (SqlNode node: operands) {
            exprs.add(cx.convertExpression(node));
         }
    +    TimeUnit timeUnit = ((SqlIntervalQualifier) call.getOperandList().get(0)).getStartUnit();
    +    boolean isNullable = exprs.get(1).getType().isNullable();
    +    RelDataType returnType = inferReturnType(typeFactory, timeUnit, isNullable);
    +    return rexBuilder.makeCall(returnType, call.getOperator(), exprs);
    +  }
     
    -    // Determine NULL-able using 2nd argument's Null-able.
    -    RelDataType returnType = typeFactory.createTypeWithNullability(typeFactory.createSqlType(SqlTypeName.BIGINT), exprs.get(1).getType().isNullable());
    +  public static RelDataType inferReturnType(RelDataTypeFactory factory, TimeUnit timeUnit, boolean isNullable) {
    --- End diff --
    
    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 for 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/377#discussion_r53537030
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertFrom implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    --- End diff --
    
    convert_from should take varbinary 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 for 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/377#discussion_r53756495
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    +        return factory.createTypeWithNullability(type, true);
    +      } else {
    +        return type;
    +      }
    +    }
    +
    +    boolean allBooleanOutput = true;
    --- End diff --
    
    In general, that is not safe. Specifically, what if one of the ANY column turns out to be a type which a function does not support. 
    
    For boolean, we have not choice since if this function is in WHERE-CLAUSE, Calcite forbidden ANY type appearing over there.


---
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 for 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/377#discussion_r53529398
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,42 @@
      */
     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 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.expr.annotations.FunctionTemplate.FunctionArgumentNumber;
     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 ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  /* 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<>();
    --- End diff --
    
    why remove "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 for 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/377#discussion_r53539261
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    --- End diff --
    
    Q1: why do u only check the first operand type?
    
    Q2: Can you put such logic in SqlReturntypeInference for CONCACT only, in stead of mixing them with other functions/operator?
    



---
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 for 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/377#discussion_r53540001
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    +        return factory.createTypeWithNullability(type, true);
    +      } else {
    +        return type;
    +      }
    +    }
    +
    +    boolean allBooleanOutput = true;
    +    for(DrillFuncHolder function : functions) {
    +      if(function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +        allBooleanOutput = false;
    +        break;
    +      }
    +    }
    +    if(allBooleanOutput) {
    +      return opBinding.getTypeFactory().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) {
    +        return opBinding.getTypeFactory()
    +            .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +      }
    +    }
    +
    +    final List<LogicalExpression> args = Lists.newArrayList();
    +    for(final RelDataType type : opBinding.collectOperandTypes()) {
    +      final TypeProtos.MajorType majorType = getMajorType(type);
    +      args.add(new DumbLogicalExpression(majorType));
    +    }
    +    final FunctionCall functionCall = new FunctionCall(opBinding.getOperator().getName(), args, ExpressionPosition.UNKNOWN);
    +    final FunctionResolver functionResolver = FunctionResolverFactory.getResolver();
    +    final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
    --- End diff --
    
    Is it guarantee that there is a DrillFuncHolder to be found from this call? Can it possible return NULL?
    
    Can you explain whether / where the implicit cast happen in this type infer logic?



---
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 for 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/377#discussion_r53544612
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java ---
    @@ -20,11 +20,24 @@
     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;
     
     class Checker implements SqlOperandTypeChecker {
    --- End diff --
    
    Looks like we are creating instances of this class unnecessarily (one per UDF).
    
    Can we have static instances for some counts (0,1,2,3, 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 for 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/377#discussion_r53561382
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillExtractConvertlet.java ---
    @@ -51,17 +53,38 @@ public RexNode convertCall(SqlRexContext cx, SqlCall call) {
         final List<RexNode> exprs = new LinkedList<>();
     
         RelDataTypeFactory typeFactory = cx.getTypeFactory();
    -
    -    //RelDataType nullableReturnType =
    -
         for (SqlNode node: operands) {
            exprs.add(cx.convertExpression(node));
         }
    +    TimeUnit timeUnit = ((SqlIntervalQualifier) call.getOperandList().get(0)).getStartUnit();
    +    boolean isNullable = exprs.get(1).getType().isNullable();
    +    RelDataType returnType = inferReturnType(typeFactory, timeUnit, isNullable);
    +    return rexBuilder.makeCall(returnType, call.getOperator(), exprs);
    +  }
     
    -    // Determine NULL-able using 2nd argument's Null-able.
    -    RelDataType returnType = typeFactory.createTypeWithNullability(typeFactory.createSqlType(SqlTypeName.BIGINT), exprs.get(1).getType().isNullable());
    +  public static RelDataType inferReturnType(RelDataTypeFactory factory, TimeUnit timeUnit, boolean isNullable) {
    --- End diff --
    
    This static method is used in another place too. But I think it makes more sense to put it to TypeInferenceUtils. Thanks for bringing my attention to this one.
    
    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 for 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/377#discussion_r53755632
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - 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.MAP, TypeProtos.MinorType.MAP)
    +          //.put(SqlTypeName.ARRAY, TypeProtos.MinorType.LIST)
    +          .put(SqlTypeName.CHAR, TypeProtos.MinorType.VARCHAR)
    +          // (2) Avoid late binding
    +          .put(SqlTypeName.ANY, TypeProtos.MinorType.LATE)
    +          // (3) 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)
    +          // (4) Calcite types currently not supported by Drill, nor defined in the Drill type list:
    +          //      - SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
    +          .build();
    +
       // This is a list of all types that cannot be folded at planning time for various reasons, most of the types are
       // currently not supported at all. The reasons for the others can be found in the evaluation code in the reduce method
       public static final List<Object> NON_REDUCIBLE_TYPES = ImmutableList.builder().add(
           // cannot represent this as a literal according to calcite
    -      TypeProtos.MinorType.INTERVAL,
    +      // TypeProtos.MinorType.INTERVAL,
    --- 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 for 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/377#discussion_r53543104
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -55,12 +55,19 @@
     
       FunctionScope scope();
       NullHandling nulls() default NullHandling.INTERNAL;
    +
    +  /**
    +   * This method tells if the number of argument(s) for a function is FIXED or VARIABLE. Since most functions expect
    +   * receiving fixed number of argument(s), the default value is FIXED.
    +   */
    +  FunctionArgumentNumber argNumber() default FunctionArgumentNumber.FIXED;
    --- End diff --
    
    As a user, I do not see how setting argNumber to VARIABLE is conveying any meaning to Drill, since we do not support specifying variable number of arguments using Param annotation.
    
    If the problem is with _concat_, maybe we should have special handling for certain functions, while registering in DrillFunctionRegistry, rather than through ScanResult?


---
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 for 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/377#discussion_r53561348
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java ---
    @@ -32,6 +34,7 @@
     @FunctionTemplate(name = "flatten", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyFlatten implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in;
    --- End diff --
    
    Along the same argument as the above:
    https://github.com/hsuanyi/incubator-drill/blob/DRILL-4372_For_Review/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java#L86
    
    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 for 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/377#discussion_r53756583
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,22 +265,43 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    +    final List<MajorType> argumentTypes = Lists.newArrayList();
    +    for (LogicalExpression expression : expressions) {
    +      argumentTypes.add(expression.getMajorType());
    +    }
    +    return argumentTypes;
    +  }
    +
    +  public MajorType getReturnType(final List<LogicalExpression> logicalExpressions) {
    +    final List<MajorType> types = Lists.newArrayList();
    +    for(final LogicalExpression logicalExpression : logicalExpressions) {
    +      types.add(logicalExpression.getMajorType());
    +    }
    +    return getReturnTypeMajorType(types);
    +  }
    +
    +  public MajorType getReturnTypeMajorType(final List<MajorType> types) {
    --- 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 for 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/377#discussion_r53537856
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyFlatten.java ---
    @@ -32,6 +34,7 @@
     @FunctionTemplate(name = "flatten", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyFlatten implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in;
    --- End diff --
    
    Why does flatten() take VarChar as input?



---
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 for 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/377#discussion_r53527480
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,7 +265,24 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    +    final List<MajorType> argumentTypes = Lists.newArrayList();
    +    for (LogicalExpression expression : expressions) {
    +      argumentTypes.add(expression.getMajorType());
    +    }
    +    return argumentTypes;
    +  }
    +
    +  public final MajorType getReturnTypeMajorType(final List<MajorType> types) {
    +    final List<LogicalExpression> expressions = Lists.newArrayList();
    +    for(final MajorType type : types) {
    +      final LogicalExpression dumbExpr = new DumbLogicalExpression(type);
    --- End diff --
    
    In stead of creating a DumbLogicalExpression augmented with type, it's better to do the opposite way.  Slightly modify the original one such that it accepts a list of MajorTypes. It only requires to know the argument type, to decide the return type. 
    
    MajorType getReturnTypeMajorType(final List<MajorType> types) : This one uses most part of original implementation.
    
    MajorType getReturnType(List<LogicalExpression> expressions) : this one will pass in the list of argument types and then call the first 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 for 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/377#discussion_r53538428
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.sql.SqlCall;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperandCountRange;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class DrillCalciteSqlOperatorWrapper extends SqlOperator implements DrillCalciteSqlWrapper {
    +  public final SqlOperator operator;
    +  private SqlOperandTypeChecker operandTypeChecker = new Checker();
    +
    +  public DrillCalciteSqlOperatorWrapper(SqlOperator operator) {
    +    super(
    +        operator.getName(),
    +        operator.getKind(),
    +        operator.getLeftPrec(),
    +        operator.getRightPrec(),
    +        operator.getReturnTypeInference(),
    +        operator.getOperandTypeInference(),
    +        operator.getOperandTypeChecker());
    +    this.operator = operator;
    +  }
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  @Override
    +  public SqlOperandTypeChecker getOperandTypeChecker() {
    +    return operandTypeChecker;
    +  }
    +
    +  @Override
    +  public SqlOperandCountRange getOperandCountRange() {
    +    return operandTypeChecker.getOperandCountRange();
    +  }
    +
    +  @Override
    +  public SqlSyntax getSyntax() {
    +    return operator.getSyntax();
    +  }
    +
    +  @Override
    +  public SqlCall createCall(
    +      SqlLiteral functionQualifier,
    +      SqlParserPos pos,
    +      SqlNode... operands) {
    +    return operator.createCall(functionQualifier, pos, operands);
    +  }
    +
    +  @Override
    +  public SqlNode rewriteCall(SqlValidator validator, SqlCall call) {
    +    return operator.rewriteCall(validator, call);
    +  }
    +
    +  @Override
    +  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +    return operator.inferReturnType(opBinding);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(
    +      SqlValidator validator,
    +      SqlValidatorScope scope,
    +      SqlCall call) {
    +    return operator.deriveType(validator,
    +        scope,
    +        call);
    +  }
    +
    +  @Override
    +  public boolean checkOperandTypes(
    +      SqlCallBinding callBinding,
    +      boolean throwOnFailure) {
    +    return true;
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53538425
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java ---
    @@ -0,0 +1,212 @@
    +/**
    + * 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.ArrayListMultimap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlFunction;
    +import org.apache.calcite.sql.SqlOperandCountRange;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +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.List;
    +
    +public class DrillCalciteSqlFunctionWrapper extends SqlFunction implements DrillCalciteSqlWrapper  {
    +  private final SqlFunction operator;
    +  private final SqlOperandTypeChecker operandTypeChecker = new Checker();
    +  private final ArrayListMultimap<String, SqlOperator> opMap;
    +
    +  public DrillCalciteSqlFunctionWrapper(
    +      final SqlFunction wrappedFunction,
    +      final ArrayListMultimap<String, SqlOperator> opMap) {
    +    super(wrappedFunction.getName(),
    +        wrappedFunction.getSqlIdentifier(),
    +        wrappedFunction.getKind(),
    +        wrappedFunction.getReturnTypeInference(),
    +        wrappedFunction.getOperandTypeInference(),
    +        wrappedFunction.getOperandTypeChecker(),
    +        wrappedFunction.getParamTypes(),
    +        wrappedFunction.getFunctionType());
    +    this.operator = wrappedFunction;
    +    this.opMap = opMap;
    +  }
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  @Override
    +  public boolean validRexOperands(int count, boolean fail) {
    +    return true;
    +  }
    +
    +  @Override
    +  public String getAllowedSignatures(String opNameToUse) {
    +    return operator.getAllowedSignatures(opNameToUse);
    +  }
    +
    +  @Override
    +  public SqlOperandTypeInference getOperandTypeInference() {
    +    return operator.getOperandTypeInference();
    +  }
    +
    +  @Override
    +  public boolean isAggregator() {
    +    return operator.isAggregator();
    +  }
    +
    +  @Override
    +  public boolean requiresOrder() {
    +    return operator.requiresOrder();
    +  }
    +
    +  @Override
    +  public boolean allowsFraming() {
    +    return operator.allowsFraming();
    +  }
    +
    +  @Override
    +  public SqlReturnTypeInference getReturnTypeInference() {
    +    return operator.getReturnTypeInference();
    +  }
    +
    +  @Override
    +  public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
    +    return operator.getMonotonicity(call);
    +  }
    +
    +  @Override
    +  public boolean isDeterministic() {
    +    return operator.isDeterministic();
    +  }
    +
    +  @Override
    +  public boolean isDynamicFunction() {
    +    return operator.isDynamicFunction();
    +  }
    +
    +  @Override
    +  public boolean requiresDecimalExpansion() {
    +    return operator.requiresDecimalExpansion();
    +  }
    +
    +  @Override
    +  public boolean argumentMustBeScalar(int ordinal) {
    +    return operator.argumentMustBeScalar(ordinal);
    +  }
    +
    +  @Override
    +  public SqlOperandTypeChecker getOperandTypeChecker() {
    +    return operandTypeChecker;
    +  }
    +
    +  @Override
    +  public SqlOperandCountRange getOperandCountRange() {
    +    return operandTypeChecker.getOperandCountRange();
    +  }
    +
    +  @Override
    +  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +    if(operator == SqlStdOperatorTable.CAST) {
    +      return operator.inferReturnType(opBinding);
    +    } else if(operator == SqlStdOperatorTable.EXTRACT) {
    +      final RelDataType returnType = DrillExtractConvertlet.inferReturnType(
    +          opBinding.getTypeFactory(),
    +          opBinding.getOperandType(0).getIntervalQualifier().getStartUnit(),
    +          opBinding.getOperandType(1).isNullable());
    +      return returnType;
    +    }
    +
    +    final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    for(SqlOperator sqlOperator : opMap.get(getOperator().getName().toLowerCase())) {
    +      if(sqlOperator instanceof DrillSqlOperator
    +          && ((DrillSqlOperator) sqlOperator).getFunctions() != null) {
    +        functions.addAll(((DrillSqlOperator) sqlOperator).getFunctions());
    +      }
    +    }
    +
    +    return TypeInferenceUtils.inferReturnType(opBinding, functions);
    +  }
    +
    +  @Override
    +  public boolean checkOperandTypes(
    +      SqlCallBinding callBinding,
    +      boolean throwOnFailure) {
    +    return true;
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53537107
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - 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.MAP, TypeProtos.MinorType.MAP)
    --- End diff --
    
    Some of these types are commented out. Can you add/ correct comments explaining why?
    
    Also, some of the comments are incorrect.


---
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 for 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/377#discussion_r53499889
  
    --- Diff: pom.xml ---
    @@ -1278,7 +1278,7 @@
               <dependency>
                 <groupId>org.apache.calcite</groupId>
                 <artifactId>calcite-core</artifactId>
    -            <version>1.4.0-drill-r10</version>
    +            <version>1.4.0-drill-test-r14</version>
    --- End diff --
    
    could you pls share the link to the branch for this forked 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 for 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/377#discussion_r53533767
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,42 @@
      */
     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 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.expr.annotations.FunctionTemplate.FunctionArgumentNumber;
     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 ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  /* 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<>();
    --- End diff --
    
    The entire functionSignatureMap is removed, not just the access modifier. 
    
    Or did you mean something else ?


---
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 for 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/377#discussion_r53537102
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
           .build();
     
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    --- End diff --
    
    Can you move this and the above type mapping to another class? Maybe a utility 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 for 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/377#discussion_r53538429
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlOperatorWrapper.java ---
    @@ -0,0 +1,187 @@
    +/**
    + * 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.sql.SqlCall;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlLiteral;
    +import org.apache.calcite.sql.SqlNode;
    +import org.apache.calcite.sql.SqlOperandCountRange;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +import org.apache.calcite.sql.validate.SqlValidator;
    +import org.apache.calcite.sql.validate.SqlValidatorScope;
    +
    +public class DrillCalciteSqlOperatorWrapper extends SqlOperator implements DrillCalciteSqlWrapper {
    +  public final SqlOperator operator;
    +  private SqlOperandTypeChecker operandTypeChecker = new Checker();
    +
    +  public DrillCalciteSqlOperatorWrapper(SqlOperator operator) {
    +    super(
    +        operator.getName(),
    +        operator.getKind(),
    +        operator.getLeftPrec(),
    +        operator.getRightPrec(),
    +        operator.getReturnTypeInference(),
    +        operator.getOperandTypeInference(),
    +        operator.getOperandTypeChecker());
    +    this.operator = operator;
    +  }
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  @Override
    +  public SqlOperandTypeChecker getOperandTypeChecker() {
    +    return operandTypeChecker;
    +  }
    +
    +  @Override
    +  public SqlOperandCountRange getOperandCountRange() {
    +    return operandTypeChecker.getOperandCountRange();
    +  }
    +
    +  @Override
    +  public SqlSyntax getSyntax() {
    +    return operator.getSyntax();
    +  }
    +
    +  @Override
    +  public SqlCall createCall(
    +      SqlLiteral functionQualifier,
    +      SqlParserPos pos,
    +      SqlNode... operands) {
    +    return operator.createCall(functionQualifier, pos, operands);
    +  }
    +
    +  @Override
    +  public SqlNode rewriteCall(SqlValidator validator, SqlCall call) {
    +    return operator.rewriteCall(validator, call);
    +  }
    +
    +  @Override
    +  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +    return operator.inferReturnType(opBinding);
    +  }
    +
    +  @Override
    +  public RelDataType deriveType(
    +      SqlValidator validator,
    +      SqlValidatorScope scope,
    +      SqlCall call) {
    +    return operator.deriveType(validator,
    +        scope,
    +        call);
    +  }
    +
    +  @Override
    +  public boolean checkOperandTypes(
    +      SqlCallBinding callBinding,
    +      boolean throwOnFailure) {
    +    return true;
    +  }
    +
    +  @Override
    +  public boolean validRexOperands(int count, boolean fail) {
    +    return true;
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53539694
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    +        return factory.createTypeWithNullability(type, true);
    +      } else {
    +        return type;
    +      }
    +    }
    +
    +    boolean allBooleanOutput = true;
    --- End diff --
    
    Why is boolean type special here?
    
    Why can this logic apply to other type? If all the functions' return type is int, can we return int 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 for 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/377#discussion_r53543109
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,22 +265,43 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    --- End diff --
    
    Does this method belong 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 for 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/377#discussion_r53538446
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -45,17 +95,27 @@ public DrillOperatorTable(FunctionImplementationRegistry registry) {
     
       public void add(String name, SqlOperator op) {
         operators.add(op);
    -    opMap.put(name, op);
    +    opMap.put(name.toLowerCase(), op);
       }
     
       @Override
    -  public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax, List<SqlOperator> operatorList) {
    -    inner.lookupOperatorOverloads(opName, category, syntax, operatorList);
    +  public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category,
    +      SqlSyntax syntax, List<SqlOperator> operatorList) {
     
    -    if (operatorList.isEmpty() && syntax == SqlSyntax.FUNCTION && opName.isSimple()) {
    -      List<SqlOperator> drillOps = opMap.get(opName.getSimple().toLowerCase());
    -      if (drillOps != null) {
    -        operatorList.addAll(drillOps);
    +    final List<SqlOperator> calciteOperatorList = Lists.newArrayList();
    +    SqlStdOperatorTable.instance().lookupOperatorOverloads(opName, category, syntax, calciteOperatorList);
    --- End diff --
    
    _inner_.lookup...?


---
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 for 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/377#discussion_r53536574
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,42 @@
      */
     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 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.expr.annotations.FunctionTemplate.FunctionArgumentNumber;
     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 ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  /* 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<>();
    --- End diff --
    
    Line 55:  +    final Map<String, String> functionSignatureMap = new HashMap<>();


---
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 for 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/377#discussion_r53537087
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,46 @@ 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<Integer, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = false;
           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.getFunctionArgumentNumber() == FunctionArgumentNumber.VARIABLE ? -1 : func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          functions.put(paramCount, func);
    +        }
    +        // 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;
    --- End diff --
    
    This changes the current behavior. isDeterministic is set by the last DrillFuncHolder, which is incorrect if one of the other implementations had a non reducible type.
    
    I am responsible for this change, but can you fix 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 for 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/377#discussion_r53528079
  
    --- Diff: exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/Extract.java ---
    @@ -31,6 +31,48 @@
     
     public class ${className} {
     
    +/* Dummy function template to allow Optiq to validate this function call.
    + * At DrillOptiq time we rewrite all date_part() functions to extract functions,
    + * since they are essentially the same
    + */
    +@FunctionTemplate(name = "DATE_PART", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
    --- End diff --
    
    I feel these new function implementations could be put into a separate patch, under a different JIRA, since they are different issue (although it's probably exposed by the function type change you made in this PR). 


---
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 for 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/377#discussion_r53538419
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java ---
    @@ -0,0 +1,205 @@
    +/**
    + * 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.ArrayListMultimap;
    +import com.google.common.collect.Lists;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.sql.SqlAggFunction;
    +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.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +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.List;
    +
    +public class DrillCalciteSqlAggFunctionWrapper extends SqlAggFunction implements DrillCalciteSqlWrapper {
    +  private final SqlAggFunction operator;
    +  private final ArrayListMultimap<String, SqlOperator> opMap;
    +  private final RelDataType relDataType;
    +  private final SqlOperandTypeChecker operandTypeChecker = new Checker();
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  private DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      ArrayListMultimap<String, SqlOperator> opMap,
    +      RelDataType relDataType) {
    +    super(sqlAggFunction.getName(),
    +        sqlAggFunction.getSqlIdentifier(),
    +        sqlAggFunction.getKind(),
    +        sqlAggFunction.getReturnTypeInference(),
    +        sqlAggFunction.getOperandTypeInference(),
    +        sqlAggFunction.getOperandTypeChecker(),
    +        sqlAggFunction.getFunctionType(),
    +        sqlAggFunction.requiresOrder(),
    +        sqlAggFunction.requiresOver());
    +    this.operator = sqlAggFunction;
    +    this.opMap = opMap;
    +    this.relDataType = relDataType;
    +  }
    +
    +  public DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      ArrayListMultimap<String, SqlOperator> opMap) {
    +    this(sqlAggFunction, opMap, null);
    +  }
    +
    +  public DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      RelDataType relDataType) {
    +    this(sqlAggFunction, null, relDataType);
    +  }
    +
    +  @Override
    +  public boolean validRexOperands(int count, boolean fail) {
    +    return true;
    +  }
    +
    +    @Override
    +    public String getAllowedSignatures(String opNameToUse) {
    +        return operator.getAllowedSignatures(opNameToUse);
    +    }
    +
    +    @Override
    +    public SqlOperandTypeInference getOperandTypeInference() {
    +        return operator.getOperandTypeInference();
    +    }
    +
    +    @Override
    +    public boolean isAggregator() {
    +        return operator.isAggregator();
    +    }
    +
    +    @Override
    +    public boolean allowsFraming() {
    +        return operator.allowsFraming();
    +    }
    +
    +    @Override
    +    public SqlReturnTypeInference getReturnTypeInference() {
    +        return operator.getReturnTypeInference();
    +    }
    +
    +    @Override
    +    public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
    +        return operator.getMonotonicity(call);
    +    }
    +
    +    @Override
    +    public boolean isDeterministic() {
    +        return operator.isDeterministic();
    +    }
    +
    +    @Override
    +    public boolean isDynamicFunction() {
    +        return operator.isDynamicFunction();
    +    }
    +
    +    @Override
    +    public boolean requiresDecimalExpansion() {
    +        return operator.requiresDecimalExpansion();
    +    }
    +
    +    @Override
    +    public boolean argumentMustBeScalar(int ordinal) {
    +        return operator.argumentMustBeScalar(ordinal);
    +    }
    +
    +    @Override
    +    public SqlOperandTypeChecker getOperandTypeChecker() {
    +        return operandTypeChecker;
    +    }
    +
    +    @Override
    +    public SqlOperandCountRange getOperandCountRange() {
    +        return operandTypeChecker.getOperandCountRange();
    +    }
    +
    +  @Override
    +  public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
    +    if(relDataType != null) {
    +      return relDataType;
    +    }
    +
    +    final List<DrillFuncHolder> functions = Lists.newArrayList();
    +    for(SqlOperator sqlOperator : opMap.get(getOperator().getName().toLowerCase())) {
    +      if(sqlOperator instanceof DrillSqlAggOperator
    +          && ((DrillSqlAggOperator) sqlOperator).getFunctions() != null) {
    +        functions.addAll(((DrillSqlAggOperator) sqlOperator).getFunctions());
    +      }
    +    }
    +
    +    return TypeInferenceUtils.inferReturnType(opBinding, functions);
    +  }
    +
    +    @Override
    +    public boolean checkOperandTypes(
    +            SqlCallBinding callBinding,
    +            boolean throwOnFailure) {
    +        return true;
    --- End diff --
    
    doc


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

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

    https://github.com/apache/drill/pull/377#issuecomment-186424169
  
    As a general reminder, we should use a more descriptive commit message. It is useful to understand what this patch is about just by looking at the commit message.


---
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 for 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/377#discussion_r53756051
  
    --- Diff: logical/src/main/java/org/apache/drill/common/expression/visitors/ExprVisitor.java ---
    @@ -43,30 +43,61 @@
     import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
     import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
     
    +/**
    + * {@link LogicalExpression Logical expression} visitor.
    + * @param <T> Return value type
    + * @param <VAL> Contextual value type
    + * @param <EXCEP> Exception type
    + */
     public interface ExprVisitor<T, VAL, EXCEP extends Exception> {
    --- 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 for 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/377#discussion_r53815227
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    +        return factory.createTypeWithNullability(type, true);
    +      } else {
    +        return type;
    +      }
    +    }
    +
    +    boolean allBooleanOutput = true;
    +    for(DrillFuncHolder function : functions) {
    +      if(function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +        allBooleanOutput = false;
    +        break;
    +      }
    +    }
    +    if(allBooleanOutput) {
    +      return opBinding.getTypeFactory().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) {
    +        return opBinding.getTypeFactory()
    +            .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +      }
    +    }
    +
    +    final List<LogicalExpression> args = Lists.newArrayList();
    +    for(final RelDataType type : opBinding.collectOperandTypes()) {
    +      final TypeProtos.MajorType majorType = getMajorType(type);
    +      args.add(new DumbLogicalExpression(majorType));
    +    }
    +    final FunctionCall functionCall = new FunctionCall(opBinding.getOperator().getName(), args, ExpressionPosition.UNKNOWN);
    +    final FunctionResolver functionResolver = FunctionResolverFactory.getResolver();
    +    final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
    --- End diff --
    
    If it is Null, a user exception should be thrown. I just enhanced the error message in the new patch.
    
    Regarding implicit casting, it happens in DefaultFunctionResolver. For example, say, a DrillFuncHolder Fn which accepts BIGINT, but we have INT in the operands. 
    
    DefaultFunctionResolver knows how to deal with this operands with implicit casting, as opposed to just failing out due to type mismatch.


---
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 for 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/377#discussion_r53536850
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -17,40 +17,42 @@
      */
     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 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.expr.annotations.FunctionTemplate.FunctionArgumentNumber;
     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 ArrayListMultimap<String, DrillFuncHolder> methods = ArrayListMultimap.create();
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillFunctionRegistry.class);
     
    -  /* 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<>();
    --- End diff --
    
    This functionSignatureMap is moved into the body of the a method (the constructor), where access modified cannot be added. 


---
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 for 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/377#discussion_r53543354
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -55,12 +55,19 @@
     
       FunctionScope scope();
       NullHandling nulls() default NullHandling.INTERNAL;
    +
    +  /**
    +   * This method tells if the number of argument(s) for a function is FIXED or VARIABLE. Since most functions expect
    +   * receiving fixed number of argument(s), the default value is FIXED.
    +   */
    +  FunctionArgumentNumber argNumber() default FunctionArgumentNumber.FIXED;
    --- End diff --
    
    There would be no need for dummy functions 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 for 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/377#discussion_r53561344
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_to", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertTo implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    --- 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 for 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/377#discussion_r53541240
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertFrom.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_from", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertFrom implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    --- 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 for 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/377#discussion_r53756134
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    --- End diff --
    
    I removed this set and used a new approach. You can take a look at this class again.


---
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 for 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/377#discussion_r53539514
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/resolver/FunctionResolver.java ---
    @@ -23,8 +23,11 @@
     import org.apache.drill.common.expression.FunctionCall;
     import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
    +/**
    + *
    --- 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 for 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/377#discussion_r53543118
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,22 +265,43 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    +    final List<MajorType> argumentTypes = Lists.newArrayList();
    +    for (LogicalExpression expression : expressions) {
    +      argumentTypes.add(expression.getMajorType());
    +    }
    +    return argumentTypes;
    +  }
    +
    +  public MajorType getReturnType(final List<LogicalExpression> logicalExpressions) {
    +    final List<MajorType> types = Lists.newArrayList();
    +    for(final LogicalExpression logicalExpression : logicalExpressions) {
    +      types.add(logicalExpression.getMajorType());
    +    }
    +    return getReturnTypeMajorType(types);
    +  }
    +
    +  public MajorType getReturnTypeMajorType(final List<MajorType> types) {
    --- End diff --
    
    please rename 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 for 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/377#discussion_r53537367
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,7 +265,24 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    +    final List<MajorType> argumentTypes = Lists.newArrayList();
    +    for (LogicalExpression expression : expressions) {
    +      argumentTypes.add(expression.getMajorType());
    +    }
    +    return argumentTypes;
    +  }
    +
    +  public final MajorType getReturnTypeMajorType(final List<MajorType> types) {
    +    final List<LogicalExpression> expressions = Lists.newArrayList();
    +    for(final MajorType type : types) {
    +      final LogicalExpression dumbExpr = new DumbLogicalExpression(type);
    --- End diff --
    
    @jinfengni 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 for 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/377#discussion_r53756645
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,22 +265,43 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    --- 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 for 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/377#discussion_r53538415
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlAggFunctionWrapper.java ---
    @@ -0,0 +1,205 @@
    +/**
    + * 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.ArrayListMultimap;
    +import com.google.common.collect.Lists;
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.sql.SqlAggFunction;
    +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.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +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.List;
    +
    +public class DrillCalciteSqlAggFunctionWrapper extends SqlAggFunction implements DrillCalciteSqlWrapper {
    +  private final SqlAggFunction operator;
    +  private final ArrayListMultimap<String, SqlOperator> opMap;
    +  private final RelDataType relDataType;
    +  private final SqlOperandTypeChecker operandTypeChecker = new Checker();
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  private DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      ArrayListMultimap<String, SqlOperator> opMap,
    +      RelDataType relDataType) {
    +    super(sqlAggFunction.getName(),
    +        sqlAggFunction.getSqlIdentifier(),
    +        sqlAggFunction.getKind(),
    +        sqlAggFunction.getReturnTypeInference(),
    +        sqlAggFunction.getOperandTypeInference(),
    +        sqlAggFunction.getOperandTypeChecker(),
    +        sqlAggFunction.getFunctionType(),
    +        sqlAggFunction.requiresOrder(),
    +        sqlAggFunction.requiresOver());
    +    this.operator = sqlAggFunction;
    +    this.opMap = opMap;
    +    this.relDataType = relDataType;
    +  }
    +
    +  public DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      ArrayListMultimap<String, SqlOperator> opMap) {
    +    this(sqlAggFunction, opMap, null);
    +  }
    +
    +  public DrillCalciteSqlAggFunctionWrapper(
    +      SqlAggFunction sqlAggFunction,
    +      RelDataType relDataType) {
    +    this(sqlAggFunction, null, relDataType);
    +  }
    +
    +  @Override
    +  public boolean validRexOperands(int count, boolean fail) {
    +    return true;
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53675500
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java ---
    @@ -38,6 +39,9 @@
     import com.google.common.base.Stopwatch;
     import com.google.common.collect.Lists;
     
    +/**
    + *
    --- 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 for 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/377#discussion_r53538421
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlFunctionWrapper.java ---
    @@ -0,0 +1,212 @@
    +/**
    + * 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.ArrayListMultimap;
    +import com.google.common.collect.Lists;
    +
    +import org.apache.calcite.rel.type.RelDataType;
    +import org.apache.calcite.sql.SqlCall;
    +import org.apache.calcite.sql.SqlCallBinding;
    +import org.apache.calcite.sql.SqlFunction;
    +import org.apache.calcite.sql.SqlOperandCountRange;
    +import org.apache.calcite.sql.SqlOperator;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.SqlSyntax;
    +import org.apache.calcite.sql.SqlWriter;
    +import org.apache.calcite.sql.fun.SqlStdOperatorTable;
    +import org.apache.calcite.sql.type.SqlOperandTypeChecker;
    +import org.apache.calcite.sql.type.SqlOperandTypeInference;
    +import org.apache.calcite.sql.type.SqlReturnTypeInference;
    +import org.apache.calcite.sql.validate.SqlMonotonicity;
    +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.List;
    +
    +public class DrillCalciteSqlFunctionWrapper extends SqlFunction implements DrillCalciteSqlWrapper  {
    +  private final SqlFunction operator;
    +  private final SqlOperandTypeChecker operandTypeChecker = new Checker();
    +  private final ArrayListMultimap<String, SqlOperator> opMap;
    +
    +  public DrillCalciteSqlFunctionWrapper(
    +      final SqlFunction wrappedFunction,
    +      final ArrayListMultimap<String, SqlOperator> opMap) {
    +    super(wrappedFunction.getName(),
    +        wrappedFunction.getSqlIdentifier(),
    +        wrappedFunction.getKind(),
    +        wrappedFunction.getReturnTypeInference(),
    +        wrappedFunction.getOperandTypeInference(),
    +        wrappedFunction.getOperandTypeChecker(),
    +        wrappedFunction.getParamTypes(),
    +        wrappedFunction.getFunctionType());
    +    this.operator = wrappedFunction;
    +    this.opMap = opMap;
    +  }
    +
    +  @Override
    +  public SqlOperator getOperator() {
    +    return operator;
    +  }
    +
    +  @Override
    +  public boolean validRexOperands(int count, boolean fail) {
    +    return true;
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53541919
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillConstExecutor.java ---
    @@ -98,11 +98,37 @@
           //      - CHAR, SYMBOL, MULTISET, DISTINCT, STRUCTURED, ROW, OTHER, CURSOR, COLUMN_LIST
           .build();
     
    +  private static ImmutableMap<SqlTypeName, TypeProtos.MinorType> CALCITE_TO_DRILL_MAPPING =
    --- End diff --
    
    Addressed. I moved both CALCITE_TO_DRILL_MAPPING and another to TypeInferenceUtils.java


---
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 for 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/377#discussion_r53538706
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/resolver/FunctionResolver.java ---
    @@ -23,8 +23,11 @@
     import org.apache.drill.common.expression.FunctionCall;
     import org.apache.drill.exec.expr.fn.DrillFuncHolder;
     
    +/**
    + *
    --- End diff --
    
    ?


---
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 for 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/377#discussion_r53541152
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -75,7 +82,7 @@
         NULL_IF_NULL;
       }
     
    -  public static enum FunctionScope {
    +  enum FunctionScope {
    --- 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 for 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/377#discussion_r53756634
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,22 +265,43 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    --- End diff --
    
    moved it to DefaultFunctionResolver as a helper


---
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 for 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/377#discussion_r53809552
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFunctionRegistry.java ---
    @@ -92,38 +94,46 @@ 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<Integer, DrillFuncHolder> functions = ArrayListMultimap.create();
    +      final ArrayListMultimap<Integer, DrillFuncHolder> aggregateFunctions = ArrayListMultimap.create();
    +      final String name = function.getKey().toUpperCase();
    +      boolean isDeterministic = false;
           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.getFunctionArgumentNumber() == FunctionArgumentNumber.VARIABLE ? -1 : func.getParamCount();
    +        if(func.isAggregating()) {
    +          aggregateFunctions.put(paramCount, func);
    +        } else {
    +          functions.put(paramCount, func);
    +        }
    +        // 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;
    --- End diff --
    
    Addressed in the new patch. Moreover, I will add the following comment into the code.
    
    In order to prevent Drill from folding constant functions with types that cannot be materialized
    into literals, the deterministic property of the DrillSqlOperator which has functions with the NON_REDUCIBLE_TYPES will be set to false.
    
    However,  partition-pruning will be initiated "only" for the deterministic DrillSqlOperator. Thus, an additional logic is added to PruneScanRule to help decide if partition-pruning can be taken.


---
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 for 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/377#discussion_r53540420
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    +        return factory.createTypeWithNullability(type, true);
    +      } else {
    +        return type;
    +      }
    +    }
    +
    +    boolean allBooleanOutput = true;
    +    for(DrillFuncHolder function : functions) {
    +      if(function.getReturnType().getMinorType() != TypeProtos.MinorType.BIT) {
    +        allBooleanOutput = false;
    +        break;
    +      }
    +    }
    +    if(allBooleanOutput) {
    +      return opBinding.getTypeFactory().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) {
    +        return opBinding.getTypeFactory()
    +            .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +      }
    +    }
    +
    +    final List<LogicalExpression> args = Lists.newArrayList();
    +    for(final RelDataType type : opBinding.collectOperandTypes()) {
    +      final TypeProtos.MajorType majorType = getMajorType(type);
    +      args.add(new DumbLogicalExpression(majorType));
    +    }
    +    final FunctionCall functionCall = new FunctionCall(opBinding.getOperator().getName(), args, ExpressionPosition.UNKNOWN);
    +    final FunctionResolver functionResolver = FunctionResolverFactory.getResolver();
    +    final DrillFuncHolder func = functionResolver.getBestMatch(functions, functionCall);
    +
    +    // If the return type is VarChar,
    +    // set the precision as the maximum
    +    RelDataType returnType = getReturnType(opBinding, func);
    +      if(returnType.getSqlTypeName() == SqlTypeName.VARCHAR) {
    +      final boolean isNullable = returnType.isNullable();
    +      returnType = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +
    +      if(isNullable) {
    +        returnType = factory.createTypeWithNullability(returnType, true);
    +      }
    +    }
    +
    +    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 anyType = factory.createSqlType(SqlTypeName.ANY);
    +    final RelDataType nullableAnyType = factory.createTypeWithNullability(anyType, true);
    +
    +    final TypeProtos.MajorType returnType = func.getReturnType();
    +    if (NONE.equals(returnType)) {
    +      return nullableAnyType;
    +    }
    +
    +    final TypeProtos.MinorType minorType = returnType.getMinorType();
    +    final SqlTypeName sqlTypeName = DrillConstExecutor.DRILL_TO_CALCITE_TYPE_MAPPING.get(minorType);
    +    if (sqlTypeName == null) {
    +      return factory.createTypeWithNullability(nullableAnyType, true);
    +    }
    +
    +    final RelDataType relReturnType;
    +    switch (sqlTypeName) {
    +      case INTERVAL_DAY_TIME:
    +        relReturnType = factory.createSqlIntervalType(
    +            new SqlIntervalQualifier(
    +                TimeUnit.DAY,
    +                    TimeUnit.MINUTE,
    --- End diff --
    
    Why only minutes, not seconds for INTERVAL_DAY_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 for 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/377#discussion_r53538444
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -26,17 +63,30 @@
     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.logical.DrillParseContext;
    +import org.apache.drill.exec.planner.physical.PrelUtil;
     
    +import java.util.Arrays;
     import java.util.List;
    +import java.util.Map;
     
    -public class DrillOperatorTable extends SqlStdOperatorTable {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
    +import static org.apache.calcite.util.Static.RESOURCE;
    --- 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 for 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/377#discussion_r53544668
  
    --- Diff: logical/src/main/java/org/apache/drill/common/expression/DumbLogicalExpression.java ---
    @@ -0,0 +1,60 @@
    +/**
    + * 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.common.expression;
    +
    +import java.util.Iterator;
    +
    +import org.apache.drill.common.expression.visitors.ExprVisitor;
    +import org.apache.drill.common.types.TypeProtos;
    +
    +public class DumbLogicalExpression implements LogicalExpression {
    --- End diff --
    
    doc and 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 for 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/377#discussion_r53538246
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -109,6 +116,16 @@ public int getValue() {
         public static FunctionCostCategory getDefault() {
           return SIMPLE;
         }
    +  }
     
    +  /**
    +   * The number of argument(s) for a function
    +   *
    +   * This annotation element tells if the number of argument(s) is fixed or arbitrary. Most functions are supposed to
    +   * have fixed number of argument(s); One of the exceptions is String Concatenation function (i.e., concat(...)), which
    +   * can take as many arguments as users would like.
    +   */
    +  enum FunctionArgumentNumber {
    +    FIXED, VARIABLE
    --- End diff --
    
    Even under VARIABLE category, there could be function which takes >=0, >=1, etc. For instance, I assume concat will take >=2 arguments.  0 or 1 input should not be valid. Do you have logic to differentiate difference 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 for 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/377#discussion_r53536497
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,81 @@
     
     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 com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.sql.SqlIntervalQualifier;
     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.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.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos.MajorType;
    +import org.apache.drill.common.types.TypeProtos.MinorType;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
    +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;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
     
       private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
    +  @Deprecated
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, null, 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, int argCount, MajorType returnType, boolean isDeterministic) {
    +    super(new SqlIdentifier(name, SqlParserPos.ZERO),
    +        null,
    --- End diff --
    
    Why do you pass null as SqlReturnTypeInference? Why do you put the logic for returnTypeInference in a static method in TypeInferenceUtils? To sense, it makes more sense to implement SqlReturnTypeInference for different DrillSqlOperator.  There could be a Drill general TypeInference for most of drill functions, in addition to specific one ("concast", "decimal").  Will it better to keep such logic in the TyepInferenceUtil?
    



---
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 for 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/377#discussion_r53540182
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    --- End diff --
    
    This "NONE" is actually referring to "any - late, nullable".  Can we use a more meaningful word? 


---
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 for 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/377#discussion_r53812052
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    --- End diff --
    
    UNKOWN_TYPE
    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 for 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/377#discussion_r53561342
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/conv/DummyConvertTo.java ---
    @@ -32,6 +34,8 @@
     @FunctionTemplate(name = "convert_to", scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
     public class DummyConvertTo implements DrillSimpleFunc {
     
    +  @Param VarCharHolder in1;
    --- End diff --
    
    Initially, I needed to add "@Param". Otherwise, validation will fail due to failure of finding a function with the correct number of argument.
    
    Now, with this 
    https://github.com/hsuanyi/incubator-drill/blob/DRILL-4372_For_Review/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java#L103
    we can just remove this line.


---
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 for 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/377#discussion_r53811953
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java ---
    @@ -0,0 +1,196 @@
    +/**
    + * 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.Lists;
    +import com.google.common.collect.Sets;
    +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.SqlIntervalQualifier;
    +import org.apache.calcite.sql.SqlOperatorBinding;
    +import org.apache.calcite.sql.parser.SqlParserPos;
    +import org.apache.calcite.sql.type.SqlTypeName;
    +import org.apache.drill.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos;
    +import org.apache.drill.common.types.Types;
    +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;
    +import java.util.Set;
    +
    +public class TypeInferenceUtils {
    +  public static final TypeProtos.MajorType NONE = TypeProtos.MajorType.getDefaultInstance();
    +  public static final int MAX_VARCHAR_LENGTH = 65535;
    +
    +  private static final Set<String> setFnWithDynamicTypes = Sets.newHashSet();
    +  static {
    +    setFnWithDynamicTypes.add("FALTTEN");
    +    setFnWithDynamicTypes.add("KVGEN");
    +    setFnWithDynamicTypes.add("CONVERT_FROM");
    +  }
    +
    +  public static RelDataType inferReturnType(final SqlOperatorBinding opBinding, final List<DrillFuncHolder> functions) {
    +    if(functions == null
    +        || functions.isEmpty()
    +            || setFnWithDynamicTypes.contains(opBinding.getOperator().getName().toUpperCase())) {
    +      return opBinding.getTypeFactory()
    +          .createTypeWithNullability(opBinding.getTypeFactory().createSqlType(SqlTypeName.ANY), true);
    +    }
    +
    +    final RelDataTypeFactory factory = opBinding.getTypeFactory();
    +    final String name = opBinding.getOperator().getName().toUpperCase();
    +    if(name.equals("CONCAT")) {
    +    final RelDataType type = factory.createSqlType(SqlTypeName.VARCHAR, MAX_VARCHAR_LENGTH);
    +      if(opBinding.getOperandType(0).isNullable()) {
    --- End diff --
    
    Now I added one SqlReturntypeInference for Concat specifically. 
    See TypeInferenceUtils#DrillConcatSqlReturnTypeInference
    
    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 for 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/377#discussion_r53561412
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java ---
    @@ -45,17 +95,27 @@ public DrillOperatorTable(FunctionImplementationRegistry registry) {
     
       public void add(String name, SqlOperator op) {
         operators.add(op);
    -    opMap.put(name, op);
    +    opMap.put(name.toLowerCase(), op);
       }
     
       @Override
    -  public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax, List<SqlOperator> operatorList) {
    -    inner.lookupOperatorOverloads(opName, category, syntax, operatorList);
    +  public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category,
    +      SqlSyntax syntax, List<SqlOperator> operatorList) {
     
    -    if (operatorList.isEmpty() && syntax == SqlSyntax.FUNCTION && opName.isSimple()) {
    -      List<SqlOperator> drillOps = opMap.get(opName.getSimple().toLowerCase());
    -      if (drillOps != null) {
    -        operatorList.addAll(drillOps);
    +    final List<SqlOperator> calciteOperatorList = Lists.newArrayList();
    +    SqlStdOperatorTable.instance().lookupOperatorOverloads(opName, category, syntax, calciteOperatorList);
    --- 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 for 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/377#discussion_r53756698
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/annotations/FunctionTemplate.java ---
    @@ -55,12 +55,19 @@
     
       FunctionScope scope();
       NullHandling nulls() default NullHandling.INTERNAL;
    +
    +  /**
    +   * This method tells if the number of argument(s) for a function is FIXED or VARIABLE. Since most functions expect
    +   * receiving fixed number of argument(s), the default value is FIXED.
    +   */
    +  FunctionArgumentNumber argNumber() default FunctionArgumentNumber.FIXED;
    --- End diff --
    
    Like above, this no longer exists. 
    
    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 for 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/377#discussion_r53533306
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java ---
    @@ -264,7 +265,24 @@ public boolean isFieldReader(int i) {
         return this.parameters[i].isFieldReader;
       }
     
    -  public MajorType getReturnType(List<LogicalExpression> args) {
    +  public static List<MajorType> getResultType(List<LogicalExpression> expressions) {
    +    final List<MajorType> argumentTypes = Lists.newArrayList();
    +    for (LogicalExpression expression : expressions) {
    +      argumentTypes.add(expression.getMajorType());
    +    }
    +    return argumentTypes;
    +  }
    +
    +  public final MajorType getReturnTypeMajorType(final List<MajorType> types) {
    +    final List<LogicalExpression> expressions = Lists.newArrayList();
    +    for(final MajorType type : types) {
    +      final LogicalExpression dumbExpr = new DumbLogicalExpression(type);
    --- End diff --
    
    @hnfgns done! take a look please.
    
    @jinfengni make sense. working on 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 for 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/377#discussion_r53538433
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillCalciteSqlWrapper.java ---
    @@ -0,0 +1,24 @@
    +/**
    + * 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;
    +
    +public interface DrillCalciteSqlWrapper {
    --- End diff --
    
    doc


---
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 for 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/377#discussion_r53500132
  
    --- Diff: pom.xml ---
    @@ -1278,7 +1278,7 @@
               <dependency>
                 <groupId>org.apache.calcite</groupId>
                 <artifactId>calcite-core</artifactId>
    -            <version>1.4.0-drill-r10</version>
    +            <version>1.4.0-drill-test-r14</version>
    --- End diff --
    
    https://github.com/hsuanyi/incubator-calcite/tree/DrillCalcite1.4.0TypeExpo


---
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 for 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/377#discussion_r53561067
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java ---
    @@ -18,69 +18,81 @@
     
     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 com.google.common.collect.Lists;
    +import com.google.common.collect.Sets;
    +
    +import java.util.List;
    +import java.util.Set;
    +
    +import org.apache.calcite.avatica.util.TimeUnit;
    +import org.apache.calcite.sql.SqlIntervalQualifier;
     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.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.common.expression.DumbLogicalExpression;
    +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.types.TypeProtos.MajorType;
    +import org.apache.drill.common.types.TypeProtos.MinorType;
    +import org.apache.drill.common.types.Types;
    +import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
    +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;
     
     public class DrillSqlOperator extends SqlFunction {
    -  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
    +  // static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
     
       private static final MajorType NONE = MajorType.getDefaultInstance();
    -  private final MajorType returnType;
       private final boolean isDeterministic;
    +  private final List<DrillFuncHolder> functions;
     
    +  @Deprecated
       public DrillSqlOperator(String name, int argCount, boolean isDeterministic) {
    -    this(name, argCount, MajorType.getDefaultInstance(), isDeterministic);
    +    this(name, null, 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, int argCount, MajorType returnType, boolean isDeterministic) {
    +    super(new SqlIdentifier(name, SqlParserPos.ZERO),
    +        null,
    --- End diff --
    
    1. Why do you pass null as SqlReturnTypeInference? 
    Since Calcite's inference mechanism is not going to be used in Drill at all, those three fields do not have practical meaning. 
    
    2. Why do you put the logic for returnTypeInference in a static method in TypeInferenceUtils?
    Does make sense. I addressed it in the new patch.



---
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 for 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/377#discussion_r53537095
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionImplementationRegistry.java ---
    @@ -38,6 +39,9 @@
     import com.google.common.base.Stopwatch;
     import com.google.common.collect.Lists;
     
    +/**
    + *
    --- End diff --
    
    Empty?


---
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 for 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/377#discussion_r53533100
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/StringFunctions.java ---
    @@ -216,6 +217,20 @@ public void eval() {
       }
     
       @FunctionTemplate(names = {"char_length", "character_length", "length"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
    +  public static class DummyCharLength implements DrillSimpleFunc {
    --- End diff --
    
    As you can see, there are a bunch of dummy functions added in. 
    As here:
    https://github.com/hsuanyi/incubator-drill/blob/DRILL-4372_For_Review/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/TypeInferenceUtils.java#L101
    
    If there is no a dummy, this line cannot find a DrillFuncHolder which matches the given SqlOperatorBinding. 
    
    Their bodies are empty and it is fine because later in DrillOptiq, the functions which will actually be used will be chosen.
    
    
    
    
    
    



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