You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2018/12/06 08:51:48 UTC

[1/3] calcite git commit: [CALCITE-2726] ReduceExpressionRule oversimplifies filter conditions containing nulls

Repository: calcite
Updated Branches:
  refs/heads/master efec74deb -> f3655e15a


[CALCITE-2726] ReduceExpressionRule oversimplifies filter conditions containing nulls

ReduceExpressionsRule might have simplified "(empno=null and mgr=1) is
null" to "false".

Add test case based on [HIVE-20617];
deprecate ExprSimplifier (Julian Hyde)

Close apache/calcite#956


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/4da9c0d9
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/4da9c0d9
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/4da9c0d9

Branch: refs/heads/master
Commit: 4da9c0d94ee0e2a2f4d03845730ffc63a83a7cbd
Parents: efec74d
Author: Zoltan Haindrich <ki...@rxd.hu>
Authored: Wed Dec 5 16:32:50 2018 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 5 16:18:19 2018 -0800

----------------------------------------------------------------------
 .../rel/rules/ReduceExpressionsRule.java        |  9 ++--
 .../org/apache/calcite/rex/RexSimplify.java     |  7 +--
 .../java/org/apache/calcite/rex/RexUtil.java    |  8 ++--
 .../apache/calcite/test/RelOptRulesTest.java    | 15 ++++++
 .../calcite/test/RexImplicationCheckerTest.java | 50 ++++++++++++--------
 .../org/apache/calcite/test/RelOptRulesTest.xml | 23 ++++++++-
 core/src/test/resources/sql/conditions.iq       | 13 +++++
 7 files changed, 93 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 50823f4..48ed3cd 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -54,7 +54,6 @@ import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUnknownAs;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexUtil.ExprSimplifier;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -539,14 +538,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
 
     // Simplify predicates in place
     final RexUnknownAs unknownAs = RexUnknownAs.falseIf(unknownAsFalse);
-    boolean reduced = reduceExpressionsInternal(rel, simplify, unknownAs,
+    final boolean reduced = reduceExpressionsInternal(rel, simplify, unknownAs,
         expList, predicates);
 
-    final ExprSimplifier simplifier =
-        new ExprSimplifier(simplify, unknownAs, matchNullability);
     boolean simplified = false;
     for (int i = 0; i < expList.size(); i++) {
-      RexNode expr2 = simplifier.apply(expList.get(i));
+      final RexNode expr2 =
+          simplify.simplifyPreservingType(expList.get(i), unknownAs,
+              matchNullability);
       if (!expr2.equals(expList.get(i))) {
         expList.set(i, expr2);
         simplified = true;

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
index f78a90c..c7b6dac 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSimplify.java
@@ -164,15 +164,16 @@ public class RexSimplify {
    * <p>This is useful if you are simplifying expressions in a
    * {@link Project}. */
   public RexNode simplifyPreservingType(RexNode e) {
-    return simplifyPreservingType(e, defaultUnknownAs);
+    return simplifyPreservingType(e, defaultUnknownAs, true);
   }
 
-  private RexNode simplifyPreservingType(RexNode e, RexUnknownAs unknownAs) {
+  public RexNode simplifyPreservingType(RexNode e, RexUnknownAs unknownAs,
+      boolean matchNullability) {
     final RexNode e2 = simplifyUnknownAs(e, unknownAs);
     if (e2.getType() == e.getType()) {
       return e2;
     }
-    final RexNode e3 = rexBuilder.makeCast(e.getType(), e2, true);
+    final RexNode e3 = rexBuilder.makeCast(e.getType(), e2, matchNullability);
     if (e3.equals(e)) {
       return e;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index 56eabb6..053d950 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -2607,7 +2607,11 @@ public class RexUtil {
     }
   }
 
-  /** Deep expressions simplifier. */
+  /** Deep expressions simplifier.
+   *
+   * <p>This class is broken because it does not change the value of
+   * {@link RexUnknownAs} as it recurses into an expression. Do not use. */
+  @Deprecated // to be removed before 2.0
   public static class ExprSimplifier extends RexShuttle {
     private final RexSimplify simplify;
     private final Map<RexNode, RexUnknownAs> unknownAsMap =
@@ -2615,12 +2619,10 @@ public class RexUtil {
     private final RexUnknownAs unknownAs;
     private final boolean matchNullability;
 
-    @Deprecated // to be removed before 2.0
     public ExprSimplifier(RexSimplify simplify) {
       this(simplify, RexUnknownAs.UNKNOWN, true);
     }
 
-    @Deprecated // to be removed before 2.0
     public ExprSimplifier(RexSimplify simplify, boolean matchNullability) {
       this(simplify, RexUnknownAs.UNKNOWN, matchNullability);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 3db0ccc..ae82438 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -4208,6 +4208,21 @@ public class RelOptRulesTest extends RelOptTestBase {
         + "case when MGR > 0 then deptno / MGR else null end > 1";
     checkPlanning(program, sql);
   }
+
+  /** Test case for
+  * <a href="https://issues.apache.org/jira/browse/CALCITE-2726">[CALCITE-2726]
+  * ReduceExpressionRule may oversimplify filter conditions containing nulls</a>.
+  */
+  @Test public void testNoOversimplificationBelowIsNull() {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .build();
+
+    String sql =
+        "select * from emp where ( (empno=1 and mgr=1) or (empno=null and mgr=1) ) is null";
+    checkPlanning(program, sql);
+  }
+
 }
 
 // End RelOptRulesTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
index 207a819..2b9b57f 100644
--- a/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexImplicationCheckerTest.java
@@ -34,7 +34,6 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUnknownAs;
-import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.server.CalciteServerStatement;
@@ -345,14 +344,11 @@ public class RexImplicationCheckerTest {
   /** Test case for
    * <a href="https://issues.apache.org/jira/browse/CALCITE-2041">[CALCITE-2041]
    * When simplifying a nullable expression, allow the result to change type to
-   * NOT NULL</a> and
-   * {@link org.apache.calcite.rex.RexUtil.ExprSimplifier#matchNullability}. */
+   * NOT NULL</a> and match nullability.
+   *
+   * @see RexSimplify#simplifyPreservingType(RexNode, RexUnknownAs, boolean) */
   @Test public void testSimplifyCastMatchNullability() {
     final Fixture f = new Fixture();
-    final RexUtil.ExprSimplifier defaultSimplifier =
-        new RexUtil.ExprSimplifier(f.simplify, RexUnknownAs.UNKNOWN, true);
-    final RexUtil.ExprSimplifier nonMatchingNullabilitySimplifier =
-        new RexUtil.ExprSimplifier(f.simplify, RexUnknownAs.UNKNOWN, false);
 
     // The cast is nullable, while the literal is not nullable. When we simplify
     // it, we end up with the literal. If defaultSimplifier is used, a CAST is
@@ -361,9 +357,13 @@ public class RexImplicationCheckerTest {
     // nonMatchingNullabilitySimplifier is used, the CAST is not added and the
     // simplified expression only consists of the literal.
     final RexNode e = f.cast(f.intRelDataType, f.literal(2014));
-    assertThat(defaultSimplifier.apply(e).toString(),
+    assertThat(
+        f.simplify.simplifyPreservingType(e, RexUnknownAs.UNKNOWN, true)
+            .toString(),
         is("CAST(2014):JavaType(class java.lang.Integer)"));
-    assertThat(nonMatchingNullabilitySimplifier.apply(e).toString(),
+    assertThat(
+        f.simplify.simplifyPreservingType(e, RexUnknownAs.UNKNOWN, false)
+            .toString(),
         is("2014"));
 
     // In this case, the cast is not nullable. Thus, in both cases, the
@@ -371,9 +371,13 @@ public class RexImplicationCheckerTest {
     RelDataType notNullIntRelDataType = f.typeFactory.createJavaType(int.class);
     final RexNode e2 = f.cast(notNullIntRelDataType,
         f.cast(notNullIntRelDataType, f.literal(2014)));
-    assertThat(defaultSimplifier.apply(e2).toString(),
+    assertThat(
+        f.simplify.simplifyPreservingType(e2, RexUnknownAs.UNKNOWN, true)
+            .toString(),
         is("2014"));
-    assertThat(nonMatchingNullabilitySimplifier.apply(e2).toString(),
+    assertThat(
+        f.simplify.simplifyPreservingType(e2, RexUnknownAs.UNKNOWN, false)
+            .toString(),
         is("2014"));
   }
 
@@ -385,8 +389,6 @@ public class RexImplicationCheckerTest {
     final ImmutableList<TimeUnitRange> timeUnitRanges =
         ImmutableList.of(TimeUnitRange.YEAR, TimeUnitRange.MONTH);
     final Fixture f = new Fixture();
-    final RexUtil.ExprSimplifier defaultSimplifier =
-        new RexUtil.ExprSimplifier(f.simplify, RexUnknownAs.UNKNOWN, true);
 
     final RexNode literalTs =
         f.timestampLiteral(new TimestampString("2010-10-10 00:00:00"));
@@ -404,12 +406,18 @@ public class RexImplicationCheckerTest {
         final RexNode outerCeilCall = f.rexBuilder.makeCall(
             SqlStdOperatorTable.CEIL, innerCeilCall,
             f.rexBuilder.makeFlag(timeUnitRanges.get(j)));
-        final RexCall floorSimplifiedExpr = (RexCall) defaultSimplifier.apply(outerFloorCall);
+        final RexCall floorSimplifiedExpr =
+            (RexCall) f.simplify.simplifyPreservingType(outerFloorCall,
+                RexUnknownAs.UNKNOWN, true);
         assertThat(floorSimplifiedExpr.getKind(), is(SqlKind.FLOOR));
-        assertThat(((RexLiteral) floorSimplifiedExpr.getOperands().get(1)).getValue().toString(),
+        assertThat(((RexLiteral) floorSimplifiedExpr.getOperands().get(1))
+                .getValue().toString(),
             is(timeUnitRanges.get(j).toString()));
-        assertThat(floorSimplifiedExpr.getOperands().get(0).toString(), is(literalTs.toString()));
-        final RexCall ceilSimplifiedExpr = (RexCall) defaultSimplifier.apply(outerCeilCall);
+        assertThat(floorSimplifiedExpr.getOperands().get(0).toString(),
+            is(literalTs.toString()));
+        final RexCall ceilSimplifiedExpr =
+            (RexCall) f.simplify.simplifyPreservingType(outerCeilCall,
+                RexUnknownAs.UNKNOWN, true);
         assertThat(ceilSimplifiedExpr.getKind(), is(SqlKind.CEIL));
         assertThat(((RexLiteral) ceilSimplifiedExpr.getOperands().get(1)).getValue().toString(),
             is(timeUnitRanges.get(j).toString()));
@@ -432,9 +440,13 @@ public class RexImplicationCheckerTest {
         final RexNode outerCeilCall = f.rexBuilder.makeCall(
             SqlStdOperatorTable.CEIL, innerCeilCall,
             f.rexBuilder.makeFlag(timeUnitRanges.get(j)));
-        final RexCall floorSimplifiedExpr = (RexCall) defaultSimplifier.apply(outerFloorCall);
+        final RexCall floorSimplifiedExpr =
+            (RexCall) f.simplify.simplifyPreservingType(outerFloorCall,
+                RexUnknownAs.UNKNOWN, true);
         assertThat(floorSimplifiedExpr.toString(), is(outerFloorCall.toString()));
-        final RexCall ceilSimplifiedExpr = (RexCall) defaultSimplifier.apply(outerCeilCall);
+        final RexCall ceilSimplifiedExpr =
+            (RexCall) f.simplify.simplifyPreservingType(outerCeilCall,
+                RexUnknownAs.UNKNOWN, true);
         assertThat(ceilSimplifiedExpr.toString(), is(outerCeilCall.toString()));
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index dd775ff..192e573 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -3114,6 +3114,25 @@ LogicalProject(EXPR$0=[CAST(/($2, $3)):INTEGER NOT NULL])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testNoOversimplificationBelowIsNull">
+        <Resource name="sql">
+            <![CDATA[select * from emp where ( (empno=1 and mgr=1) or (empno=null and mgr=1) ) is null]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NULL(OR(AND(=($0, 1), =($3, 1)), AND(=($0, null), =($3, 1))))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+        <Resource name="planAfter">
+            <![CDATA[
+LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
+  LogicalFilter(condition=[IS NULL(OR(AND(=($0, 1), =($3, 1)), AND(null, =($3, 1))))])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testOrAlwaysTrue">
         <Resource name="sql">
             <![CDATA[select * from EMPNULLABLES_20
@@ -6855,7 +6874,7 @@ LogicalProject(NEWCOL=[+($0, CASE(=('a', 'a'), 1, null))])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-LogicalProject(NEWCOL=[+($0, 1)])
+LogicalProject(NEWCOL=[+($0, CAST(1):INTEGER)])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -8705,7 +8724,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-  LogicalFilter(condition=[AND(>($3, 0), CASE(>($3, 0), >(/($7, $3), 1), false))])
+  LogicalFilter(condition=[AND(>($3, 0), CASE(>($3, 0), >(/($7, $3), 1), null))])
     LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>

http://git-wip-us.apache.org/repos/asf/calcite/blob/4da9c0d9/core/src/test/resources/sql/conditions.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/conditions.iq b/core/src/test/resources/sql/conditions.iq
index bc565af..70f7b76 100644
--- a/core/src/test/resources/sql/conditions.iq
+++ b/core/src/test/resources/sql/conditions.iq
@@ -258,4 +258,17 @@ select "value" from "nullables" a
 
 !ok
 
+# Test case for [CALCITE-2726] based on [HIVE-20617]
+with ax(s, t) as (values ('a','a'),('a','a '),('b','bb'))
+select 'expected 1' as e,count(*) as c
+from ax where ((s,t) in (('a','a'),(null, 'bb'))) is null;
++------------+---+
+| E          | C |
++------------+---+
+| expected 1 | 1 |
++------------+---+
+(1 row)
+
+!ok
+
 # End conditions.iq


[3/3] calcite git commit: [CALCITE-2670] Combine similar JSON aggregate functions in operator table

Posted by jh...@apache.org.
[CALCITE-2670] Combine similar JSON aggregate functions in operator table

Convert parameters from Enum to more specific types;
include varying flag values in JSON aggregate functions.

Some cosmetic stuff.

Close apache/calcite#916


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/f3655e15
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/f3655e15
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/f3655e15

Branch: refs/heads/master
Commit: f3655e15a11a9fb266af290cb390e690d4301c09
Parents: 40d12b7
Author: hongzezhang <ho...@tencent.com>
Authored: Wed Nov 14 19:48:49 2018 +0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 5 16:59:02 2018 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  26 ++---
 .../calcite/adapter/enumerable/RexImpTable.java |  34 +++---
 .../calcite/jdbc/JavaTypeFactoryImpl.java       |   2 +
 .../apache/calcite/rel/core/AggregateCall.java  |   2 +-
 .../apache/calcite/runtime/SqlFunctions.java    |  18 +---
 .../sql/SqlJsonConstructorNullClause.java       |   9 +-
 .../apache/calcite/sql/SqlJsonEmptyOrError.java |   5 +-
 .../calcite/sql/SqlJsonExistsErrorBehavior.java |   5 +-
 .../sql/SqlJsonQueryEmptyOrErrorBehavior.java   |   2 +-
 .../sql/SqlJsonValueEmptyOrErrorBehavior.java   |   2 +-
 .../calcite/sql/dialect/MssqlSqlDialect.java    |   4 +-
 .../sql/fun/SqlJsonArrayAggAggFunction.java     |  51 ++++++---
 .../calcite/sql/fun/SqlJsonArrayFunction.java   |   2 +-
 .../sql/fun/SqlJsonObjectAggAggFunction.java    |  47 +++++---
 .../calcite/sql/fun/SqlJsonObjectFunction.java  |   2 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  16 +--
 .../org/apache/calcite/util/BuiltInMethod.java  |  21 ++--
 .../calcite/test/SqlToRelConverterTest.java     |  58 ++++++++++
 .../calcite/test/SqlToRelConverterTest.xml      | 108 +++++++++++++++++++
 core/src/test/resources/sql/misc.iq             |   6 ++
 .../java/org/apache/calcite/test/CsvTest.java   |   8 +-
 21 files changed, 304 insertions(+), 124 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index cf6b3f2..6cecc56 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -5267,14 +5267,8 @@ SqlCall JsonObjectAggFunctionCall() :
         }
     ]
     <RPAREN> {
-        switch (nullClause) {
-        case ABSENT_ON_NULL:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_ABSENT_ON_NULL.createCall(span.end(this), args);
-        case NULL_ON_NULL:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL.createCall(span.end(this), args);
-        default:
-          return SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL.createCall(span.end(this), args);
-        }
+        return SqlStdOperatorTable.JSON_OBJECTAGG.with(nullClause)
+            .createCall(span.end(this), args);
     }
 }
 
@@ -5313,7 +5307,7 @@ SqlCall JsonArrayFunctionCall() :
 
 SqlCall JsonArrayAggFunctionCall() :
 {
-    final SqlNode arg;
+    final SqlNode[] args = new SqlNode[1];
     List<SqlNode> list;
     final Span span;
     SqlJsonConstructorNullClause nullClause =
@@ -5322,21 +5316,17 @@ SqlCall JsonArrayAggFunctionCall() :
 }
 {
     <JSON_ARRAYAGG> { span = span(); }
-    <LPAREN> arg = JsonValueExpression(false)
+    <LPAREN> e = JsonValueExpression(false) {
+        args[0] = e;
+    }
     [
         e = JsonConstructorNullClause() {
             nullClause = (SqlJsonConstructorNullClause) ((SqlLiteral) e).getValue();
         }
     ]
     <RPAREN> {
-        switch (nullClause) {
-        case ABSENT_ON_NULL:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL.createCall(span.end(this), arg);
-        case NULL_ON_NULL:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_NULL_ON_NULL.createCall(span.end(this), arg);
-        default:
-          return SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL.createCall(span.end(this), arg);
-        }
+        return SqlStdOperatorTable.JSON_ARRAYAGG.with(nullClause)
+            .createCall(span.end(this), args);
     }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 7776ac9..dd117ee 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -45,6 +45,8 @@ import org.apache.calcite.schema.impl.AggregateFunctionImpl;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.fun.SqlJsonArrayAggAggFunction;
+import org.apache.calcite.sql.fun.SqlJsonObjectAggAggFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.type.SqlTypeName;
@@ -157,12 +159,10 @@ import static org.apache.calcite.sql.fun.SqlStdOperatorTable.IS_TRUE;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.ITEM;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_API_COMMON_SYNTAX;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAY;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG_ABSENT_ON_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG_NULL_ON_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_ARRAYAGG;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_EXISTS;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECT;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG_ABSENT_ON_NULL;
-import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG_NULL_ON_NULL;
+import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_OBJECTAGG;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_QUERY;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_STRUCTURED_VALUE_EXPRESSION;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.JSON_VALUE_ANY;
@@ -448,19 +448,13 @@ public class RexImpTable {
     defineMethod(JSON_VALUE_ANY, BuiltInMethod.JSON_VALUE_ANY.method, NullPolicy.NONE);
     defineMethod(JSON_QUERY, BuiltInMethod.JSON_QUERY.method, NullPolicy.NONE);
     defineMethod(JSON_OBJECT, BuiltInMethod.JSON_OBJECT.method, NullPolicy.NONE);
-    aggMap.put(JSON_OBJECTAGG_NULL_ON_NULL,
+    aggMap.put(JSON_OBJECTAGG,
         JsonObjectAggImplementor
-            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD_NULL_ON_NULL.method));
-    aggMap.put(JSON_OBJECTAGG_ABSENT_ON_NULL,
-        JsonObjectAggImplementor
-            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD_ABSENT_ON_NULL.method));
+            .supplierFor(BuiltInMethod.JSON_OBJECTAGG_ADD.method));
     defineMethod(JSON_ARRAY, BuiltInMethod.JSON_ARRAY.method, NullPolicy.NONE);
-    aggMap.put(JSON_ARRAYAGG_NULL_ON_NULL,
-        JsonArrayAggImplementor
-            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD_NULL_ON_NULL.method));
-    aggMap.put(JSON_ARRAYAGG_ABSENT_ON_NULL,
+    aggMap.put(JSON_ARRAYAGG,
         JsonArrayAggImplementor
-            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD_ABSENT_ON_NULL.method));
+            .supplierFor(BuiltInMethod.JSON_ARRAYAGG_ADD.method));
     defineImplementor(IS_JSON_VALUE, NullPolicy.NONE,
             new MethodImplementor(BuiltInMethod.IS_JSON_VALUE.method), false);
     defineImplementor(IS_JSON_OBJECT, NullPolicy.NONE,
@@ -1813,12 +1807,16 @@ public class RexImpTable {
     }
 
     @Override public void implementAdd(AggContext info, AggAddContext add) {
+      final SqlJsonObjectAggAggFunction function =
+          (SqlJsonObjectAggAggFunction) info.aggregation();
       add.currentBlock().add(
           Expressions.statement(
               Expressions.call(m,
                   Iterables.concat(
                       Collections.singletonList(add.accumulator().get(0)),
-                      add.arguments()))));
+                      add.arguments(),
+                      Collections.singletonList(
+                          Expressions.constant(function.getNullClause()))))));
     }
 
     @Override public Expression implementResult(AggContext info,
@@ -1854,12 +1852,16 @@ public class RexImpTable {
 
     @Override public void implementAdd(AggContext info,
         AggAddContext add) {
+      final SqlJsonArrayAggAggFunction function =
+          (SqlJsonArrayAggAggFunction) info.aggregation();
       add.currentBlock().add(
           Expressions.statement(
               Expressions.call(m,
                   Iterables.concat(
                       Collections.singletonList(add.accumulator().get(0)),
-                      add.arguments()))));
+                      add.arguments(),
+                      Collections.singletonList(
+                          Expressions.constant(function.getNullClause()))))));
     }
 
     @Override public Expression implementResult(AggContext info,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index 59805fe..08a7aa8 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -213,6 +213,8 @@ public class JavaTypeFactoryImpl
         return ByteString.class;
       case GEOMETRY:
         return GeoFunctions.Geom.class;
+      case SYMBOL:
+        return Enum.class;
       case ANY:
         return Object.class;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
index fb32cbc..d612ea4 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
@@ -254,7 +254,7 @@ public class AggregateCall {
   }
 
   public String toString() {
-    StringBuilder buf = new StringBuilder(aggFunction.getName());
+    StringBuilder buf = new StringBuilder(aggFunction.toString());
     buf.append("(");
     if (distinct) {
       buf.append((argList.size() == 0) ? "DISTINCT" : "DISTINCT ");

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index af008fb..7644d0a 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -2409,7 +2409,7 @@ public class SqlFunctions {
       if (!matcher.matches()) {
         throw RESOURCE.illegalJsonPathSpec(pathSpec).ex();
       }
-      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ENGLISH));
+      PathMode mode = PathMode.valueOf(matcher.group(1).toUpperCase(Locale.ROOT));
       String pathWff = matcher.group(2);
       DocumentContext ctx;
       switch (mode) {
@@ -2636,14 +2636,6 @@ public class SqlFunctions {
     }
   }
 
-  public static void jsonObjectAggAddNullOnNull(Map map, String k, Object v) {
-    jsonObjectAggAdd(map, k, v, SqlJsonConstructorNullClause.NULL_ON_NULL);
-  }
-
-  public static void jsonObjectAggAddAbsentOnNull(Map map, String k, Object v) {
-    jsonObjectAggAdd(map, k, v, SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-  }
-
   public static String jsonArray(SqlJsonConstructorNullClause nullClause,
       Object... elements) {
     List<Object> list = new ArrayList<>();
@@ -2670,14 +2662,6 @@ public class SqlFunctions {
     }
   }
 
-  public static void jsonArrayAggAddNullOnNull(List list, Object element) {
-    jsonArrayAggAdd(list, element, SqlJsonConstructorNullClause.NULL_ON_NULL);
-  }
-
-  public static void jsonArrayAggAddAbsentOnNull(List list, Object element) {
-    jsonArrayAggAdd(list, element, SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-  }
-
   public static boolean isJsonValue(String input) {
     try {
       dejsonize(input);

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
index a4a94b8..ef660fa 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonConstructorNullClause.java
@@ -20,7 +20,14 @@ package org.apache.calcite.sql;
  * Indicating that how do Json constructors handle null
  */
 public enum SqlJsonConstructorNullClause {
-  NULL_ON_NULL, ABSENT_ON_NULL
+  NULL_ON_NULL("NULL ON NULL"),
+  ABSENT_ON_NULL("ABSENT ON NULL");
+
+  public final String sql;
+
+  SqlJsonConstructorNullClause(String sql) {
+    this.sql = sql;
+  }
 }
 
 // End SqlJsonConstructorNullClause.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
index 82491cd..07d847a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonEmptyOrError.java
@@ -23,10 +23,11 @@ import java.util.Locale;
  * EmptyOrErrorBehavior is invoked.
  */
 public enum SqlJsonEmptyOrError {
-  EMPTY, ERROR;
+  EMPTY,
+  ERROR;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH, "SqlJsonEmptyOrError[%s]", name());
+    return String.format(Locale.ROOT, "SqlJsonEmptyOrError[%s]", name());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
index 4da8a4f..1f9e9d8 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonExistsErrorBehavior.java
@@ -20,7 +20,10 @@ package org.apache.calcite.sql;
  * Categorizing Json exists error behaviors.
  */
 public enum SqlJsonExistsErrorBehavior {
-  TRUE, FALSE, UNKNOWN, ERROR
+  TRUE,
+  FALSE,
+  UNKNOWN,
+  ERROR
 }
 
 // End SqlJsonExistsErrorBehavior.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
index 2dd2e34..1893552 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonQueryEmptyOrErrorBehavior.java
@@ -28,7 +28,7 @@ public enum SqlJsonQueryEmptyOrErrorBehavior {
   EMPTY_OBJECT;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH,
+    return String.format(Locale.ROOT,
         "SqlJsonQueryEmptyOrErrorBehavior[%s]", name());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java b/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
index cb39279..0650fb7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlJsonValueEmptyOrErrorBehavior.java
@@ -27,7 +27,7 @@ public enum SqlJsonValueEmptyOrErrorBehavior {
   DEFAULT;
 
   @Override public String toString() {
-    return String.format(Locale.ENGLISH,
+    return String.format(Locale.ROOT,
         "SqlJsonValueEmptyOrErrorBehavior[%s]", name());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
index 644a39b..89de617 100644
--- a/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/MssqlSqlDialect.java
@@ -180,8 +180,8 @@ public class MssqlSqlDialect extends SqlDialect {
 
   private void unparseSqlIntervalLiteralMssql(
       SqlWriter writer, SqlIntervalLiteral literal, int sign) {
-    SqlIntervalLiteral.IntervalValue interval
-        = (SqlIntervalLiteral.IntervalValue) literal.getValue();
+    final SqlIntervalLiteral.IntervalValue interval =
+        (SqlIntervalLiteral.IntervalValue) literal.getValue();
     unparseSqlIntervalQualifier(writer, interval.getIntervalQualifier(),
         RelDataTypeSystem.DEFAULT);
     writer.sep(",", true);

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
index 4791e67..a5d1520 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayAggAggFunction.java
@@ -16,20 +16,27 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Optionality;
 
+import java.util.Locale;
+import java.util.Objects;
+
 /**
- * The <code>JSON_OBJECTAGG</code> aggregation function.
+ * The <code>JSON_OBJECTAGG</code> aggregate function.
  */
 public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
   private final SqlJsonConstructorNullClause nullClause;
@@ -37,9 +44,9 @@ public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
   public SqlJsonArrayAggAggFunction(String name,
       SqlJsonConstructorNullClause nullClause) {
     super(name, null, SqlKind.JSON_ARRAYAGG, ReturnTypes.VARCHAR_2000, null,
-        OperandTypes.ANY, SqlFunctionCategory.SYSTEM, false, false,
-        Optionality.FORBIDDEN);
-    this.nullClause = nullClause;
+        OperandTypes.family(SqlTypeFamily.ANY), SqlFunctionCategory.SYSTEM,
+        false, false, Optionality.FORBIDDEN);
+    this.nullClause = Objects.requireNonNull(nullClause);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
@@ -47,21 +54,31 @@ public class SqlJsonArrayAggAggFunction extends SqlAggFunction {
     assert call.operandCount() == 1;
     final SqlWriter.Frame frame = writer.startFunCall("JSON_ARRAYAGG");
     call.operand(0).unparse(writer, leftPrec, rightPrec);
-    switch (nullClause) {
-    case ABSENT_ON_NULL:
-      writer.keyword("ABSENT ON NULL");
-      break;
-    case NULL_ON_NULL:
-      writer.keyword("NULL ON NULL");
-      break;
-    default:
-      throw new IllegalStateException("unreachable code");
-    }
+    writer.keyword(nullClause.sql);
     writer.endFunCall(frame);
   }
 
-  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
-    return (E) ((SqlLiteral) operand).getValue();
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    // To prevent operator rewriting by SqlFunction#deriveType.
+    for (SqlNode operand : call.getOperandList()) {
+      RelDataType nodeType = validator.deriveType(scope, operand);
+      ((SqlValidatorImpl) validator).setValidatedNodeType(operand, nodeType);
+    }
+    return validateOperands(validator, scope, call);
+  }
+
+  @Override public String toString() {
+    return getName() + String.format(Locale.ROOT, "<%s>", nullClause);
+  }
+
+  public SqlJsonArrayAggAggFunction with(SqlJsonConstructorNullClause nullClause) {
+    return this.nullClause == nullClause ? this
+        : new SqlJsonArrayAggAggFunction(getName(), nullClause);
+  }
+
+  public SqlJsonConstructorNullClause getNullClause() {
+    return nullClause;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
index 0306654..29ae0f8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonArrayFunction.java
@@ -67,7 +67,7 @@ public class SqlJsonArrayFunction extends SqlFunction {
     final StringBuilder sb = new StringBuilder();
     sb.append("{0}(");
     for (int i = 1; i < operandsCount; i++) {
-      sb.append(String.format(Locale.ENGLISH, "{%d} ", i + 1));
+      sb.append(String.format(Locale.ROOT, "{%d} ", i + 1));
     }
     sb.append("{1})");
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
index 4b50c5c..d8b85c7 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectAggAggFunction.java
@@ -16,31 +16,38 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlJsonConstructorNullClause;
 import org.apache.calcite.sql.SqlKind;
-import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.sql.validate.SqlValidatorImpl;
+import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Optionality;
 
+import java.util.Locale;
+import java.util.Objects;
+
 /**
- * The <code>JSON_OBJECTAGG</code> aggregation function.
+ * The <code>JSON_OBJECTAGG</code> aggregate function.
  */
 public class SqlJsonObjectAggAggFunction extends SqlAggFunction {
   private final SqlJsonConstructorNullClause nullClause;
 
+  /** Creates a SqlJsonObjectAggAggFunction. */
   public SqlJsonObjectAggAggFunction(String name,
       SqlJsonConstructorNullClause nullClause) {
     super(name, null, SqlKind.JSON_OBJECTAGG, ReturnTypes.VARCHAR_2000, null,
         OperandTypes.family(SqlTypeFamily.CHARACTER, SqlTypeFamily.ANY),
         SqlFunctionCategory.SYSTEM, false, false, Optionality.FORBIDDEN);
-    this.nullClause = nullClause;
+    this.nullClause = Objects.requireNonNull(nullClause);
   }
 
   @Override public void unparse(SqlWriter writer, SqlCall call, int leftPrec,
@@ -51,21 +58,31 @@ public class SqlJsonObjectAggAggFunction extends SqlAggFunction {
     call.operand(0).unparse(writer, leftPrec, rightPrec);
     writer.keyword("VALUE");
     call.operand(1).unparse(writer, leftPrec, rightPrec);
-    switch (nullClause) {
-    case ABSENT_ON_NULL:
-      writer.keyword("ABSENT ON NULL");
-      break;
-    case NULL_ON_NULL:
-      writer.keyword("NULL ON NULL");
-      break;
-    default:
-      throw new IllegalStateException("unreachable code");
-    }
+    writer.keyword(nullClause.sql);
     writer.endFunCall(frame);
   }
 
-  private <E extends Enum<E>> E getEnumValue(SqlNode operand) {
-    return (E) ((SqlLiteral) operand).getValue();
+  @Override public RelDataType deriveType(SqlValidator validator,
+      SqlValidatorScope scope, SqlCall call) {
+    // To prevent operator rewriting by SqlFunction#deriveType.
+    for (SqlNode operand : call.getOperandList()) {
+      RelDataType nodeType = validator.deriveType(scope, operand);
+      ((SqlValidatorImpl) validator).setValidatedNodeType(operand, nodeType);
+    }
+    return validateOperands(validator, scope, call);
+  }
+
+  @Override public String toString() {
+    return getName() + String.format(Locale.ROOT, "<%s>", nullClause);
+  }
+
+  public SqlJsonObjectAggAggFunction with(SqlJsonConstructorNullClause nullClause) {
+    return this.nullClause == nullClause ? this
+        : new SqlJsonObjectAggAggFunction(getName(), nullClause);
+  }
+
+  public SqlJsonConstructorNullClause getNullClause() {
+    return nullClause;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
index 771c823..0240532 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlJsonObjectFunction.java
@@ -93,7 +93,7 @@ public class SqlJsonObjectFunction extends SqlFunction {
     StringBuilder sb = new StringBuilder();
     sb.append("{0}(");
     for (int i = 1; i < operandsCount; i++) {
-      sb.append(String.format(Locale.ENGLISH, "{%d} ", i + 1));
+      sb.append(String.format(Locale.ROOT, "{%d} ", i + 1));
     }
     sb.append("{1})");
     return sb.toString();

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index 03d4336..6c3c596 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -1291,24 +1291,16 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   public static final SqlFunction JSON_OBJECT = new SqlJsonObjectFunction();
 
-  public static final SqlAggFunction JSON_OBJECTAGG_NULL_ON_NULL =
-      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG_NULL_ON_NULL",
+  public static final SqlJsonObjectAggAggFunction JSON_OBJECTAGG =
+      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG",
           SqlJsonConstructorNullClause.NULL_ON_NULL);
 
-  public static final SqlAggFunction JSON_OBJECTAGG_ABSENT_ON_NULL =
-      new SqlJsonObjectAggAggFunction("JSON_OBJECTAGG_ABSENT_ON_NULL",
-          SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-
   public static final SqlFunction JSON_ARRAY = new SqlJsonArrayFunction();
 
-  public static final SqlAggFunction JSON_ARRAYAGG_NULL_ON_NULL =
-      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG_NULL_ON_NULL",
+  public static final SqlJsonArrayAggAggFunction JSON_ARRAYAGG =
+      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG",
           SqlJsonConstructorNullClause.NULL_ON_NULL);
 
-  public static final SqlAggFunction JSON_ARRAYAGG_ABSENT_ON_NULL =
-      new SqlJsonArrayAggAggFunction("JSON_ARRAYAGG_ABSENT_ON_NULL",
-          SqlJsonConstructorNullClause.ABSENT_ON_NULL);
-
   public static final SqlBetweenOperator BETWEEN =
       new SqlBetweenOperator(
           SqlBetweenOperator.Flag.ASYMMETRIC,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 8160ead..f23a931 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -272,27 +272,20 @@ public enum BuiltInMethod {
       Object.class, String.class),
   JSON_EXISTS(SqlFunctions.class, "jsonExists", Object.class),
   JSON_VALUE_ANY(SqlFunctions.class, "jsonValueAny", Object.class,
-      SqlJsonValueEmptyOrErrorBehavior.class,
-      Object.class,
-      SqlJsonValueEmptyOrErrorBehavior.class,
-      Object.class),
-  JSON_QUERY(SqlFunctions.class, "jsonQuery",
-      Object.class,
+      SqlJsonValueEmptyOrErrorBehavior.class, Object.class,
+      SqlJsonValueEmptyOrErrorBehavior.class, Object.class),
+  JSON_QUERY(SqlFunctions.class, "jsonQuery", Object.class,
       SqlJsonQueryWrapperBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class,
       SqlJsonQueryEmptyOrErrorBehavior.class),
   JSON_OBJECT(SqlFunctions.class, "jsonObject",
       SqlJsonConstructorNullClause.class),
-  JSON_OBJECTAGG_ADD_NULL_ON_NULL(SqlFunctions.class,
-      "jsonObjectAggAddNullOnNull", Map.class, String.class, Object.class),
-  JSON_OBJECTAGG_ADD_ABSENT_ON_NULL(SqlFunctions.class,
-      "jsonObjectAggAddAbsentOnNull", Map.class, String.class, Object.class),
+  JSON_OBJECTAGG_ADD(SqlFunctions.class, "jsonObjectAggAdd", Map.class,
+      String.class, Object.class, SqlJsonConstructorNullClause.class),
   JSON_ARRAY(SqlFunctions.class, "jsonArray",
       SqlJsonConstructorNullClause.class),
-  JSON_ARRAYAGG_ADD_NULL_ON_NULL(SqlFunctions.class,
-      "jsonArrayAggAddNullOnNull", List.class, Object.class),
-  JSON_ARRAYAGG_ADD_ABSENT_ON_NULL(SqlFunctions.class,
-      "jsonArrayAggAddAbsentOnNull", List.class, Object.class),
+  JSON_ARRAYAGG_ADD(SqlFunctions.class, "jsonArrayAggAdd",
+      List.class, Object.class, SqlJsonConstructorNullClause.class),
   IS_JSON_VALUE(SqlFunctions.class, "isJsonValue", String.class),
   IS_JSON_OBJECT(SqlFunctions.class, "isJsonObject", String.class),
   IS_JSON_ARRAY(SqlFunctions.class, "isJsonArray", String.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 1207020..f13894c 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -2882,6 +2882,64 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).with(tester).ok();
   }
 
+  @Test public void testJsonExists() {
+    final String sql = "select json_exists(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonValue() {
+    final String sql = "select json_value(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonQuery() {
+    final String sql = "select json_query(ename, 'lax $')\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonArray() {
+    final String sql = "select json_array(ename, ename)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonArrayAgg() {
+    final String sql = "select json_arrayagg(ename)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonObject() {
+    final String sql = "select json_object(ename: deptno, ename: deptno)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonObjectAgg() {
+    final String sql = "select json_objectagg(ename: deptno)\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
+  @Test public void testJsonPredicate() {
+    final String sql = "select\n"
+        + "ename is json,\n"
+        + "ename is json value,\n"
+        + "ename is json object,\n"
+        + "ename is json array,\n"
+        + "ename is json scalar,\n"
+        + "ename is not json,\n"
+        + "ename is not json value,\n"
+        + "ename is not json object,\n"
+        + "ename is not json array,\n"
+        + "ename is not json scalar\n"
+        + "from emp";
+    sql(sql).ok();
+  }
+
   @Test public void testWithinGroup1() {
     final String sql = "select deptno,\n"
         + " collect(empno) within group (order by deptno, hiredate desc)\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 28d2e3f..d8539c7 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -5316,6 +5316,114 @@ LogicalProject(ANYEMPNO=[$1])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testJsonExists">
+        <Resource name="sql">
+            <![CDATA[select json_exists(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_EXISTS(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonValue">
+        <Resource name="sql">
+            <![CDATA[select json_value(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(JSON_VALUE_ANY(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'), FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null, FLAG(SqlJsonValueEmptyOrErrorBehavior[NULL]), null)):VARCHAR(2000) CHARACTER SET "ISO-8859-1" COLLATE "ISO-8859-1$en_US$primary"])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonQuery">
+        <Resource name="sql">
+            <![CDATA[select json_query(ename, 'lax $')
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_QUERY(JSON_API_COMMON_SYNTAX(JSON_VALUE_EXPRESSION($1), 'lax $'), FLAG(WITHOUT_ARRAY), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]), FLAG(SqlJsonQueryEmptyOrErrorBehavior[NULL]))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonArray">
+        <Resource name="sql">
+            <![CDATA[select json_array(ename, ename)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_ARRAY(FLAG(ABSENT_ON_NULL), JSON_STRUCTURED_VALUE_EXPRESSION($1), JSON_STRUCTURED_VALUE_EXPRESSION($1))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonArrayAgg">
+        <Resource name="sql">
+            <![CDATA[select json_arrayagg(ename)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[JSON_ARRAYAGG<ABSENT_ON_NULL>($0)])
+  LogicalProject($f0=[JSON_STRUCTURED_VALUE_EXPRESSION($1)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonObject">
+        <Resource name="sql">
+            <![CDATA[select select json_object(ename: deptno, ename: deptno)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[JSON_OBJECT(FLAG(NULL_ON_NULL), $1, JSON_STRUCTURED_VALUE_EXPRESSION($7), $1, JSON_STRUCTURED_VALUE_EXPRESSION($7))])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonObjectAgg">
+        <Resource name="sql">
+            <![CDATA[select json_objectagg(ename: deptno)
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalAggregate(group=[{}], EXPR$0=[JSON_OBJECTAGG<NULL_ON_NULL>($0, $1)])
+  LogicalProject(ENAME=[$1], $f1=[JSON_STRUCTURED_VALUE_EXPRESSION($7)])
+    LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
+    <TestCase name="testJsonPredicate">
+        <Resource name="sql">
+            <![CDATA[select
+ename is json,
+ename is json value,
+ename is json object,
+ename is json array,
+ename is json scalar,
+ename is not json,
+ename is not json value,
+ename is not json object,
+ename is not json array,
+ename is not json scalar
+from emp]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(EXPR$0=[IS JSON VALUE($1)], EXPR$1=[IS JSON VALUE($1)], EXPR$2=[IS JSON OBJECT($1)], EXPR$3=[IS JSON ARRAY($1)], EXPR$4=[IS JSON SCALAR($1)], EXPR$5=[IS NOT JSON VALUE($1)], EXPR$6=[IS NOT JSON VALUE($1)], EXPR$7=[IS NOT JSON OBJECT($1)], EXPR$8=[IS NOT JSON ARRAY($1)], EXPR$9=[IS NOT JSON SCALAR($1)])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testWithinGroup1">
         <Resource name="sql">
             <![CDATA[select deptno,

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 0862a54..8b771ff 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -2213,4 +2213,10 @@ X
 true
 !ok
 
+# JSON
+values json_exists('{"foo":"bar"}', 'strict $.foo' false on error);
+EXPR$0
+true
+!ok
+
 # End misc.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/f3655e15/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index 703e813..5789d41 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -512,8 +512,8 @@ public class CsvTest {
     Properties info = new Properties();
     info.put("model", jsonPath("bug"));
 
-    try (Connection connection
-        = DriverManager.getConnection("jdbc:calcite:", info)) {
+    try (Connection connection =
+        DriverManager.getConnection("jdbc:calcite:", info)) {
       ResultSet res = connection.getMetaData().getColumns(null, null,
           "DATE", "JOINEDAT");
       res.next();
@@ -560,8 +560,8 @@ public class CsvTest {
     Properties info = new Properties();
     info.put("model", jsonPath("bug"));
 
-    try (Connection connection
-        = DriverManager.getConnection("jdbc:calcite:", info)) {
+    try (Connection connection =
+        DriverManager.getConnection("jdbc:calcite:", info)) {
       Statement statement = connection.createStatement();
       final String sql = "select * from \"DATE\"\n"
           + "where EMPNO >= 140 and EMPNO < 200";


[2/3] calcite git commit: [CALCITE-2468] Validator throws IndexOutOfBoundsException when trying to infer operand type from STRUCT return type (Rong Rong)

Posted by jh...@apache.org.
[CALCITE-2468] Validator throws IndexOutOfBoundsException when trying to infer operand type from STRUCT return type (Rong Rong)

Add example in server module's type.iq test.
Fix type inference when operator is AS with a complex row of just 1 element.

Close apache/calcite#932


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/40d12b74
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/40d12b74
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/40d12b74

Branch: refs/heads/master
Commit: 40d12b7419bd544bd829011a18ac37f6bc529ce7
Parents: 4da9c0d
Author: Rong Rong <wa...@hotmail.com>
Authored: Mon Aug 20 17:16:19 2018 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Dec 5 16:19:32 2018 -0800

----------------------------------------------------------------------
 .../calcite/sql/validate/SqlValidatorImpl.java  | 31 +++++++++++++-------
 .../calcite/test/SqlToRelConverterTest.java     | 10 +++++++
 .../calcite/test/SqlToRelConverterTest.xml      | 12 ++++++++
 server/src/test/resources/sql/type.iq           | 22 ++++++++++++++
 4 files changed, 65 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/40d12b74/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 626399f..41fdab5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -131,6 +131,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.function.Supplier;
+import javax.annotation.Nonnull;
 
 import static org.apache.calcite.sql.SqlUtil.stripAs;
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -454,7 +455,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     selectItems.add(expanded);
     aliases.add(alias);
 
-    inferUnknownTypes(targetType, scope, expanded);
+    if (expanded != null) {
+      inferUnknownTypes(targetType, scope, expanded);
+    }
     final RelDataType type = deriveType(selectScope, expanded);
     setValidatedNodeType(expanded, type);
     fields.add(Pair.of(alias, type));
@@ -1720,9 +1723,12 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   protected void inferUnknownTypes(
-      RelDataType inferredType,
-      SqlValidatorScope scope,
-      SqlNode node) {
+      @Nonnull RelDataType inferredType,
+      @Nonnull SqlValidatorScope scope,
+      @Nonnull SqlNode node) {
+    Objects.requireNonNull(inferredType);
+    Objects.requireNonNull(scope);
+    Objects.requireNonNull(node);
     final SqlValidatorScope newScope = scopes.get(node);
     if (newScope != null) {
       scope = newScope;
@@ -1790,6 +1796,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       } else {
         setValidatedNodeType(caseCall.getElseOperand(), returnType);
       }
+    } else if (node.getKind()  == SqlKind.AS) {
+      // For AS operator, only infer the operand not the alias
+      inferUnknownTypes(inferredType, scope, ((SqlCall) node).operand(0));
     } else if (node instanceof SqlCall) {
       final SqlCall call = (SqlCall) node;
       final SqlOperandTypeInference operandTypeInference =
@@ -1797,18 +1806,20 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final SqlCallBinding callBinding = new SqlCallBinding(this, scope, call);
       final List<SqlNode> operands = callBinding.operands();
       final RelDataType[] operandTypes = new RelDataType[operands.size()];
-      if (operandTypeInference == null) {
-        // TODO:  eventually should assert(operandTypeInference != null)
-        // instead; for now just eat it
-        Arrays.fill(operandTypes, unknownType);
-      } else {
+      Arrays.fill(operandTypes, unknownType);
+      // TODO:  eventually should assert(operandTypeInference != null)
+      // instead; for now just eat it
+      if (operandTypeInference != null) {
         operandTypeInference.inferOperandTypes(
             callBinding,
             inferredType,
             operandTypes);
       }
       for (int i = 0; i < operands.size(); ++i) {
-        inferUnknownTypes(operandTypes[i], scope, operands.get(i));
+        final SqlNode operand = operands.get(i);
+        if (operand != null) {
+          inferUnknownTypes(operandTypes[i], scope, operand);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/40d12b74/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
index 77fa01c..1207020 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelConverterTest.java
@@ -123,6 +123,16 @@ public class SqlToRelConverterTest extends SqlToRelTestBase {
     sql(sql).ok();
   }
 
+  /** Test case for:
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2468">[CALCITE-2468]
+   * struct type alias should not cause IOOBE.</a>.
+   */
+  @Test public void testStructTypeAlias() {
+    final String sql = "select t.r AS myRow \n"
+        + "from (select row(row(1)) r from dept) t";
+    sql(sql).ok();
+  }
+
   @Test
   public void testJoinUsingDynamicTable() {
     final String sql = "select * from SALES.NATION t1\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/40d12b74/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 9128946..28d2e3f 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -1459,6 +1459,18 @@ LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testStructTypeAlias">
+        <Resource name="sql">
+            <![CDATA[select t.r AS myRow
+from (select row(row(1)) r from dept) t]]>
+        </Resource>
+        <Resource name="plan">
+            <![CDATA[
+LogicalProject(MYROW$$0$$0=[1])
+  LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testJoinWithUnion">
         <Resource name="sql">
             <![CDATA[select grade

http://git-wip-us.apache.org/repos/asf/calcite/blob/40d12b74/server/src/test/resources/sql/type.iq
----------------------------------------------------------------------
diff --git a/server/src/test/resources/sql/type.iq b/server/src/test/resources/sql/type.iq
index 64314e9..7103be4 100644
--- a/server/src/test/resources/sql/type.iq
+++ b/server/src/test/resources/sql/type.iq
@@ -49,7 +49,29 @@ select * from t;
 
 !ok
 
+
+# Create a table with complex structure type
+# This is to test struct type inference in
+# <a href="https://issues.apache.org/jira/browse/CALCITE-1222">[CALCITE-2468]
+
+create type mytype1 as (ii int not null);
+(0 rows modified)
+
+!update
+
+# Create a complex table
+create table v (i int not null, j mytype1 not null);
+(0 rows modified)
+
+!update
+
+select i AS myInt, j AS myStruct from v;
+MYINT INTEGER(10) NOT NULL
+MYSTRUCT STRUCT NOT NULL
+!type
+
 drop table t;
+drop table v;
 (0 rows modified)
 
 !update