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 2016/07/06 05:26:54 UTC

[1/3] calcite git commit: [CALCITE-319] Table aliases should follow case-sensitivity policy

Repository: calcite
Updated Branches:
  refs/heads/master ac9c8c7af -> e30600d63


[CALCITE-319] Table aliases should follow case-sensitivity policy


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

Branch: refs/heads/master
Commit: 54d29dc172283157baf59d06213017fb5278d14c
Parents: ac9c8c7
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jul 5 15:08:06 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jul 5 15:08:06 2016 -0700

----------------------------------------------------------------------
 .../calcite/sql/validate/SqlValidatorImpl.java    | 11 ++++++++++-
 .../main/java/org/apache/calcite/util/Bug.java    |  5 -----
 .../org/apache/calcite/test/SqlValidatorTest.java | 18 +++++++++++++-----
 3 files changed, 23 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/54d29dc1/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 ae4f8e5..2b6b640 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
@@ -2986,7 +2986,16 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final SqlValidatorScope fromScope = getFromScope(select);
     final List<Pair<String, SqlValidatorNamespace>> children =
         ((SelectScope) fromScope).children;
-    int duplicateAliasOrdinal = Util.firstDuplicate(Pair.left(children));
+    List<String> names = Pair.left(children);
+    if (!catalogReader.isCaseSensitive()) {
+      names = Lists.transform(names,
+          new Function<String, String>() {
+            public String apply(String s) {
+              return s.toUpperCase();
+            }
+          });
+    }
+    final int duplicateAliasOrdinal = Util.firstDuplicate(names);
     if (duplicateAliasOrdinal >= 0) {
       final Pair<String, SqlValidatorNamespace> child =
           children.get(duplicateAliasOrdinal);

http://git-wip-us.apache.org/repos/asf/calcite/blob/54d29dc1/core/src/main/java/org/apache/calcite/util/Bug.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Bug.java b/core/src/main/java/org/apache/calcite/util/Bug.java
index f60fbd1..1adc2fe 100644
--- a/core/src/main/java/org/apache/calcite/util/Bug.java
+++ b/core/src/main/java/org/apache/calcite/util/Bug.java
@@ -151,11 +151,6 @@ public abstract class Bug {
    * Array items in MongoDB adapter</a> is fixed. */
   public static final boolean CALCITE_194_FIXED = false;
 
-  /** Whether
-   * <a href="https://issues.apache.org/jira/browse/CALCITE-319">[CALCITE-319]
-   * Table aliases should follow case-sensitivity policy</a> is fixed. */
-  public static final boolean CALCITE_319_FIXED = false;
-
   /** Whether the remaining issues raised in
    * <a href="https://issues.apache.org/jira/browse/CALCITE-461">[CALCITE-461]
    * Convert more planner rules to handle grouping sets</a> are fixed.

http://git-wip-us.apache.org/repos/asf/calcite/blob/54d29dc1/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 1c44a0b..5fcd47e 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7325,18 +7325,26 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     checkFails("select count(1), ^empno^ from emp",
         "Expression 'EMPNO' is not being grouped");
+  }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-319">[CALCITE-319]
+   * Table aliases should follow case-sensitivity policy</a>. */
+  @Test public void testCaseInsensitiveTableAlias() {
+    final SqlTester tester1 = tester
+        .withCaseSensitive(false)
+        .withQuoting(Quoting.BRACKET);
+    final SqlTester tester2 = tester.withQuoting(Quoting.BRACKET);
     // Table aliases should follow case-sensitivity preference.
     //
     // In MySQL, table aliases are case-insensitive:
     // mysql> select `D`.day from DAYS as `d`, DAYS as `D`;
     // ERROR 1066 (42000): Not unique table/alias: 'D'
+    tester1.checkQueryFails("select count(*) from dept as [D], ^dept as [d]^",
+        "Duplicate relation name 'd' in FROM clause");
     tester2.checkQuery("select count(*) from dept as [D], dept as [d]");
-    if (!Bug.CALCITE_319_FIXED) {
-      return;
-    }
-    tester1.checkQueryFails("select count(*) from dept as [D], dept as [d]",
-        "xxx");
+    tester2.checkQueryFails("select count(*) from dept as [D], ^dept as [D]^",
+        "Duplicate relation name 'D' in FROM clause");
   }
 
   /** Tests matching of built-in operator names. */


[3/3] calcite git commit: Cosmetic changes

Posted by jh...@apache.org.
Cosmetic changes


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

Branch: refs/heads/master
Commit: e30600d63b5468cc8566977bcc1695ce17db91df
Parents: 787a9a1
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jul 5 17:46:21 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jul 5 17:46:21 2016 -0700

----------------------------------------------------------------------
 .../standalone-server/src/main/shaded-resources/LICENSE   | 10 +++++-----
 avatica/tck/src/main/resources/META-INF/LICENSE           | 10 +++++-----
 core/src/main/codegen/templates/Parser.jj                 |  5 ++---
 .../apache/calcite/rel/rules/LoptOptimizeJoinRule.java    |  3 +--
 .../src/main/java/org/apache/calcite/sql/SqlFunction.java |  3 +--
 .../apache/calcite/rel/rel2sql/RelToSqlConverterTest.java |  6 ++----
 .../test/java/org/apache/calcite/sql/test/SqlTests.java   |  7 ++-----
 .../java/org/apache/calcite/test/MockCatalogReader.java   |  3 +--
 .../test/java/org/apache/calcite/tools/PlannerTest.java   |  6 ++----
 9 files changed, 21 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/avatica/standalone-server/src/main/shaded-resources/LICENSE
----------------------------------------------------------------------
diff --git a/avatica/standalone-server/src/main/shaded-resources/LICENSE b/avatica/standalone-server/src/main/shaded-resources/LICENSE
index 0720b21..877a48a 100644
--- a/avatica/standalone-server/src/main/shaded-resources/LICENSE
+++ b/avatica/standalone-server/src/main/shaded-resources/LICENSE
@@ -223,21 +223,21 @@ under the following "3-clause BSD" license:
 
     Copyright (c) 2001-2016, The HSQL Development Group
     All rights reserved.
-    
+
     Redistribution and use in source and binary forms, with or without
     modification, are permitted provided that the following conditions are met:
-    
+
     Redistributions of source code must retain the above copyright notice, this
     list of conditions and the following disclaimer.
-    
+
     Redistributions in binary form must reproduce the above copyright notice,
     this list of conditions and the following disclaimer in the documentation
     and/or other materials provided with the distribution.
-    
+
     Neither the name of the HSQL Development Group nor the names of its
     contributors may be used to endorse or promote products derived from this
     software without specific prior written permission.
-    
+
     THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
     AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
     IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/avatica/tck/src/main/resources/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/avatica/tck/src/main/resources/META-INF/LICENSE b/avatica/tck/src/main/resources/META-INF/LICENSE
index 0720b21..877a48a 100644
--- a/avatica/tck/src/main/resources/META-INF/LICENSE
+++ b/avatica/tck/src/main/resources/META-INF/LICENSE
@@ -223,21 +223,21 @@ under the following "3-clause BSD" license:
 
     Copyright (c) 2001-2016, The HSQL Development Group
     All rights reserved.
-    
+
     Redistribution and use in source and binary forms, with or without
     modification, are permitted provided that the following conditions are met:
-    
+
     Redistributions of source code must retain the above copyright notice, this
     list of conditions and the following disclaimer.
-    
+
     Redistributions in binary form must reproduce the above copyright notice,
     this list of conditions and the following disclaimer in the documentation
     and/or other materials provided with the distribution.
-    
+
     Neither the name of the HSQL Development Group nor the names of its
     contributors may be used to endorse or promote products derived from this
     software without specific prior written permission.
-    
+
     THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS"
     AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE
     IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/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 d7415cc..9a8ab59 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -229,9 +229,8 @@ PARSER_END(${parser.class})
 
 /* For Debug */
 JAVACODE
-void debug_message1()
-{
-    LOGGER.info("{} , {}", getToken( 0 ).image, getToken( 1 ).image );
+void debug_message1() {
+    LOGGER.info("{} , {}", getToken(0).image, getToken(1).image);
 }
 
 JAVACODE String unquotedIdentifier() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
index baa4139..2d17096 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/LoptOptimizeJoinRule.java
@@ -1627,8 +1627,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       projects.add(
           Pair.of(
               (RexNode) rexBuilder.makeInputRef(currFields.get(i).getType(), i),
-              currFields.get(i).getName())
-      );
+              currFields.get(i).getName()));
     }
     for (int i = 0; i < nNewFields; i++) {
       RexNode projExpr;

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
index 7f9b0f3..e485422 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlFunction.java
@@ -243,8 +243,7 @@ public class SqlFunction extends SqlOperator {
 
     final SqlFunction function =
         (SqlFunction) SqlUtil.lookupRoutine(validator.getOperatorTable(),
-            getNameAsId(), argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION, getKind()
-        );
+            getNameAsId(), argTypes, argNames, getFunctionType(), SqlSyntax.FUNCTION, getKind());
     try {
       // if we have a match on function name and parameter count, but
       // couldn't find a function with  a COLUMN_LIST type, retry, but

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index 8b19898..1d70131 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -187,8 +187,7 @@ public class RelToSqlConverterTest {
         query,
         "SELECT \"product_class_id\", SUM(\"net_weight\"), MIN(\"low_fat\"), COUNT(*)\n"
             + "FROM \"foodmart\".\"product\"\n"
-            + "GROUP BY \"product_class_id\""
-    );
+            + "GROUP BY \"product_class_id\"");
   }
 
   @Test
@@ -368,8 +367,7 @@ public class RelToSqlConverterTest {
             + "INNER JOIN \"foodmart\".\"product_class\" "
             + "ON \"product\".\"product_class_id\" = \"product_class\".\"product_class_id\"\n"
             + "WHERE \"customer\".\"city\" = 'San Francisco' AND "
-            + "\"product_class\".\"product_department\" = 'Snacks'"
-    );
+            + "\"product_class\".\"product_department\" = 'Snacks'");
   }
 
   @Test public void testSimpleIn() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
index ea4091b..0349c35 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTests.java
@@ -262,11 +262,8 @@ public abstract class SqlTests {
       fail("Query returned 2 or more rows, expected 1");
     }
     if ((actual < (expected - delta)) || (actual > (expected + delta))) {
-      fail(
-          "Query returned " + actual + ", expected " + expected + ((delta == 0)
-              ? ""
-              : ("+/-" + delta))
-      );
+      fail("Query returned " + actual + ", expected " + expected
+          + ((delta == 0) ? "" : ("+/-" + delta)));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index 0ccedc2..e56dd96 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -385,8 +385,7 @@ public class MockCatalogReader implements Prepare.CatalogReader {
                         @Override public int size() {
                           return mapping.size();
                         }
-                      }
-                  );
+                      });
                 }
               });
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/e30600d6/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
index 265a504..4aa2795 100644
--- a/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
+++ b/core/src/test/java/org/apache/calcite/tools/PlannerTest.java
@@ -125,8 +125,7 @@ public class PlannerTest {
   public void testParseIdentiferMaxLengthWithDefault() throws Exception {
     Planner planner = getPlanner(null, SqlParser.configBuilder().build());
     planner.parse("select name as "
-        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\""
-    );
+        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\"");
   }
 
   @Test
@@ -134,8 +133,7 @@ public class PlannerTest {
     Planner planner = getPlanner(null,
         SqlParser.configBuilder().setIdentifierMaxLength(512).build());
     planner.parse("select name as "
-        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\""
-    );
+        + "aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa from \"emps\"");
   }
 
   /** Unit test that parses, validates and converts the query using


[2/3] calcite git commit: [CALCITE-1305] Case-insensitive table aliases and GROUP BY

Posted by jh...@apache.org.
[CALCITE-1305] Case-insensitive table aliases and GROUP BY


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

Branch: refs/heads/master
Commit: 787a9a1b19da84021d2a9090c6ff5972e999c6b3
Parents: 54d29dc
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jul 5 12:59:48 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jul 5 17:38:38 2016 -0700

----------------------------------------------------------------------
 .../sql/validate/AggregatingSelectScope.java     |  1 -
 .../calcite/sql/validate/DelegatingScope.java    |  8 ++++++++
 .../apache/calcite/sql/validate/WithScope.java   |  3 ++-
 .../apache/calcite/test/SqlValidatorTest.java    | 19 +++++++++++++++++++
 .../main/java/org/apache/calcite/piglet/Ast.java |  2 +-
 5 files changed, 30 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/787a9a1b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
index cc884ee..29514ac 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggregatingSelectScope.java
@@ -103,7 +103,6 @@ public class AggregatingSelectScope
       // modifies it and makes it no longer equivalent. While copying,
       // we fully qualify all identifiers.
       final SqlNodeList groupList = select.getGroup();
-//          SqlValidatorUtil.DeepCopier.copy(parent, select.getGroup());
       for (SqlNode groupExpr : groupList) {
         SqlValidatorUtil.analyzeGroupItem(this, temporaryGroupExprList,
             groupExprProjection, builder, groupExpr);

http://git-wip-us.apache.org/repos/asf/calcite/blob/787a9a1b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
index 383fd12..d48bb50 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingScope.java
@@ -182,6 +182,14 @@ public abstract class DelegatingScope implements SqlValidatorScope {
         final SqlIdentifier prefix = identifier.getComponent(0, i);
         fromNs = resolve(prefix.names, null, null);
         if (fromNs != null) {
+          if (fromNs.getEnclosingNode() != null) {
+            String alias =
+                SqlValidatorUtil.getAlias(fromNs.getEnclosingNode(), -1);
+            if (alias != null
+                && !alias.equals(identifier.names.get(i - 1))) {
+              identifier = identifier.setName(i - 1, alias);
+            }
+          }
           break;
         }
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/787a9a1b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
index b650402..9141f83 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
@@ -56,7 +56,8 @@ class WithScope extends ListScope {
   @Override public SqlValidatorNamespace resolve(List<String> names,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
-    if (names.equals(withItem.name.getSimple())) {
+    if (names.size() == 1
+        && names.get(0).equals(withItem.name.getSimple())) {
       return validator.getNamespace(withItem);
     }
     return super.resolve(names, ancestorOut, offsetOut);

http://git-wip-us.apache.org/repos/asf/calcite/blob/787a9a1b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 5fcd47e..02e29c1 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -7347,6 +7347,25 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "Duplicate relation name 'D' in FROM clause");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1305">[CALCITE-1305]
+   * Case-insensitive table aliases and GROUP BY</a>. */
+  @Test public void testCaseInsensitiveTableAliasInGroupBy() {
+    final SqlTester tester1 = tester
+        .withCaseSensitive(false)
+        .withUnquotedCasing(Casing.UNCHANGED);
+    tester1.checkQuery("select deptno, count(*) from EMP AS emp\n"
+        + "group by eMp.deptno");
+    tester1.checkQuery("select deptno, count(*) from EMP AS EMP\n"
+        + "group by eMp.deptno");
+    tester1.checkQuery("select deptno, count(*) from EMP\n"
+        + "group by eMp.deptno");
+    tester1.checkQuery("select * from EMP where exists (\n"
+        + "  select 1 from dept\n"
+        + "  group by eMp.deptno)");
+    tester1.checkQuery("select deptno, count(*) from EMP group by DEPTNO");
+  }
+
   /** Tests matching of built-in operator names. */
   @Test public void testUnquotedBuiltInFunctionNames() {
     final SqlTester mysql = tester

http://git-wip-us.apache.org/repos/asf/calcite/blob/787a9a1b/piglet/src/main/java/org/apache/calcite/piglet/Ast.java
----------------------------------------------------------------------
diff --git a/piglet/src/main/java/org/apache/calcite/piglet/Ast.java b/piglet/src/main/java/org/apache/calcite/piglet/Ast.java
index 6c70606..78d0c13 100644
--- a/piglet/src/main/java/org/apache/calcite/piglet/Ast.java
+++ b/piglet/src/main/java/org/apache/calcite/piglet/Ast.java
@@ -310,7 +310,7 @@ public class Ast {
    * <p>Syntax:
    * <blockquote>
    *   <code>alias = GROUP alias
-   *   ( ALL | BY ( exp | '(' exp [, exp]... ')' ) );</code>
+   *   ( ALL | BY ( exp | '(' exp [, exp]... ')' ) )</code>
    * </blockquote>
    */
   public static class GroupStmt extends Assignment1 {