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 2015/09/10 02:48:51 UTC

[1/2] incubator-calcite git commit: Replace Stack with ArrayDeque

Repository: incubator-calcite
Updated Branches:
  refs/heads/master f003187cf -> 16512edc1


Replace Stack with ArrayDeque


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

Branch: refs/heads/master
Commit: 600e1caf083f4dc30becb386f984eebda8c48a9c
Parents: f003187
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Sep 9 15:33:49 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Sep 9 15:33:49 2015 -0700

----------------------------------------------------------------------
 .../calcite/sql/pretty/SqlPrettyWriter.java     |  5 +-
 .../calcite/sql/validate/SqlScopedShuttle.java  |  6 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  | 55 ++++++-----
 .../calcite/sql2rel/SqlToRelConverter.java      |  5 +-
 .../concurrent/ConcurrentTestCommandScript.java | 98 ++++++++------------
 5 files changed, 75 insertions(+), 94 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/600e1caf/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index 4098d52..5637f01 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -29,12 +29,13 @@ import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
+import java.util.ArrayDeque;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Properties;
 import java.util.Set;
-import java.util.Stack;
 import java.util.logging.Logger;
 
 /**
@@ -132,7 +133,7 @@ public class SqlPrettyWriter implements SqlWriter {
   private final SqlDialect dialect;
   private final StringWriter sw = new StringWriter();
   protected final PrintWriter pw;
-  private final Stack<FrameImpl> listStack = new Stack<FrameImpl>();
+  private final Deque<FrameImpl> listStack = new ArrayDeque<>();
   protected FrameImpl frame;
   private boolean needWhitespace;
   protected String nextWhitespace;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/600e1caf/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
index eca367b..66ee43d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlScopedShuttle.java
@@ -21,7 +21,8 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.util.SqlVisitor;
 
-import java.util.Stack;
+import java.util.ArrayDeque;
+import java.util.Deque;
 
 /**
  * Refinement to {@link SqlShuttle} which maintains a stack of scopes.
@@ -32,8 +33,7 @@ import java.util.Stack;
 public abstract class SqlScopedShuttle extends SqlShuttle {
   //~ Instance fields --------------------------------------------------------
 
-  private final Stack<SqlValidatorScope> scopes =
-      new Stack<SqlValidatorScope>();
+  private final Deque<SqlValidatorScope> scopes = new ArrayDeque<>();
 
   //~ Constructors -----------------------------------------------------------
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/600e1caf/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 90fa0e2..3c94bf1 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
@@ -89,11 +89,13 @@ import com.google.common.collect.Sets;
 import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.util.AbstractList;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.GregorianCalendar;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -102,7 +104,6 @@ import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.Stack;
 import java.util.logging.Level;
 import java.util.logging.Logger;
 
@@ -171,42 +172,41 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * Maps ParsePosition strings to the {@link SqlIdentifier} identifier
    * objects at these positions
    */
-  protected final Map<String, IdInfo> idPositions =
-      new HashMap<String, IdInfo>();
+  protected final Map<String, IdInfo> idPositions = new HashMap<>();
 
   /**
    * Maps {@link SqlNode query node} objects to the {@link SqlValidatorScope}
    * scope created from them}.
    */
   protected final Map<SqlNode, SqlValidatorScope> scopes =
-      new IdentityHashMap<SqlNode, SqlValidatorScope>();
+      new IdentityHashMap<>();
 
   /**
    * Maps a {@link SqlSelect} node to the scope used by its WHERE and HAVING
    * clauses.
    */
   private final Map<SqlSelect, SqlValidatorScope> whereScopes =
-      new IdentityHashMap<SqlSelect, SqlValidatorScope>();
+      new IdentityHashMap<>();
 
   /**
    * Maps a {@link SqlSelect} node to the scope used by its SELECT and HAVING
    * clauses.
    */
   private final Map<SqlSelect, SqlValidatorScope> selectScopes =
-      new IdentityHashMap<SqlSelect, SqlValidatorScope>();
+      new IdentityHashMap<>();
 
   /**
    * Maps a {@link SqlSelect} node to the scope used by its ORDER BY clause.
    */
   private final Map<SqlSelect, SqlValidatorScope> orderScopes =
-      new IdentityHashMap<SqlSelect, SqlValidatorScope>();
+      new IdentityHashMap<>();
 
   /**
    * Maps a {@link SqlSelect} node that is the argument to a CURSOR
    * constructor to the scope of the result of that select node
    */
   private final Map<SqlSelect, SqlValidatorScope> cursorScopes =
-      new IdentityHashMap<SqlSelect, SqlValidatorScope>();
+      new IdentityHashMap<>();
 
   /**
    * Maps a {@link SqlNode node} to the
@@ -214,7 +214,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * contain.
    */
   protected final Map<SqlNode, SqlValidatorNamespace> namespaces =
-      new IdentityHashMap<SqlNode, SqlValidatorNamespace>();
+      new IdentityHashMap<>();
 
   /**
    * Set of select expressions used as cursor definitions. In standard SQL,
@@ -228,8 +228,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * is needed to handle nested function calls. The function call currently
    * being validated is at the top of the stack.
    */
-  protected final Stack<FunctionParamInfo> functionCallStack =
-      new Stack<FunctionParamInfo>();
+  protected final Deque<FunctionParamInfo> functionCallStack =
+      new ArrayDeque<>();
 
   private int nextGeneratedId;
   protected final RelDataTypeFactory typeFactory;
@@ -242,12 +242,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    * instance.
    */
   private final Map<SqlNode, RelDataType> nodeToTypeMap =
-      new IdentityHashMap<SqlNode, RelDataType>();
+      new IdentityHashMap<>();
   private final AggFinder aggFinder;
   private final AggFinder aggOrOverFinder;
   private final SqlConformance conformance;
-  private final Map<SqlNode, SqlNode> originalExprs =
-      new HashMap<SqlNode, SqlNode>();
+  private final Map<SqlNode, SqlNode> originalExprs = new HashMap<>();
 
   // REVIEW jvs 30-June-2006: subclasses may override shouldExpandIdentifiers
   // in a way that ignores this; we should probably get rid of the protected
@@ -320,9 +319,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNodeList selectList,
       SqlSelect select,
       boolean includeSystemVars) {
-    List<SqlNode> list = new ArrayList<SqlNode>();
-    List<Map.Entry<String, RelDataType>> types =
-        new ArrayList<Map.Entry<String, RelDataType>>();
+    final List<SqlNode> list = new ArrayList<>();
+    final List<Map.Entry<String, RelDataType>> types = new ArrayList<>();
     for (int i = 0; i < selectList.size(); i++) {
       final SqlNode selectItem = selectList.get(i);
       expandSelectItem(
@@ -601,7 +599,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       for (int i = 0; i < id.names.size(); i++) {
         if (pos.toString().equals(
             id.getComponent(i).getParserPosition().toString())) {
-          List<SqlMoniker> objNames = new ArrayList<SqlMoniker>();
+          final List<SqlMoniker> objNames = new ArrayList<>();
           SqlValidatorUtil.getSchemaObjectMonikers(
               getCatalogReader(),
               id.names.subList(0, i + 1),
@@ -725,7 +723,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       List<String> names,
       SqlValidator validator,
       Collection<SqlMoniker> result) {
-    List<SqlMoniker> objNames = new ArrayList<SqlMoniker>();
+    final List<SqlMoniker> objNames = new ArrayList<>();
     SqlValidatorUtil.getSchemaObjectMonikers(
         validator.getCatalogReader(),
         names,
@@ -1356,15 +1354,15 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlValidatorScope scope) {
     final List<SqlNode> rows = values.getOperandList();
     assert rows.size() >= 1;
-    List<RelDataType> rowTypes = new ArrayList<RelDataType>();
+    final List<RelDataType> rowTypes = new ArrayList<>();
     for (final SqlNode row : rows) {
       assert row.getKind() == SqlKind.ROW;
       SqlCall rowConstructor = (SqlCall) row;
 
       // REVIEW jvs 10-Sept-2003: Once we support single-row queries as
       // rows, need to infer aliases from there.
-      final List<String> aliasList = new ArrayList<String>();
-      final List<RelDataType> typeList = new ArrayList<RelDataType>();
+      final List<String> aliasList = new ArrayList<>();
+      final List<RelDataType> typeList = new ArrayList<>();
       for (Ord<SqlNode> column : Ord.zip(rowConstructor.getOperandList())) {
         final String alias = deriveAlias(column.e, column.i);
         aliasList.add(alias);
@@ -3483,8 +3481,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return baseRowType;
     }
     List<RelDataTypeField> targetFields = baseRowType.getFieldList();
-    final List<Map.Entry<String, RelDataType>> types =
-        new ArrayList<Map.Entry<String, RelDataType>>();
+    final List<Map.Entry<String, RelDataType>> types = new ArrayList<>();
     if (append) {
       for (RelDataTypeField targetField : targetFields) {
         types.add(
@@ -3492,7 +3489,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
                 targetField.getType()));
       }
     }
-    Set<Integer> assignedFields = new HashSet<Integer>();
+    final Set<Integer> assignedFields = new HashSet<>();
     for (SqlNode node : targetColumnList) {
       SqlIdentifier id = (SqlIdentifier) node;
       String name = id.getSimple();
@@ -4022,7 +4019,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     if (!sqlQuery.isA(SqlKind.QUERY)) {
       return Collections.nCopies(fieldCount, null);
     }
-    final ArrayList<List<String>> list = new ArrayList<List<String>>();
+    final List<List<String>> list = new ArrayList<>();
     for (int i = 0; i < fieldCount; i++) {
       List<String> origin = getFieldOrigin(sqlQuery, i);
 //            assert origin == null || origin.size() >= 4 : origin;
@@ -4063,7 +4060,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   public RelDataType getParameterRowType(SqlNode sqlQuery) {
     // NOTE: We assume that bind variables occur in depth-first tree
     // traversal in the same order that they occurred in the SQL text.
-    final List<RelDataType> types = new ArrayList<RelDataType>();
+    final List<RelDataType> types = new ArrayList<>();
     sqlQuery.accept(
         new SqlShuttle() {
           @Override public SqlNode visit(SqlDynamicParam param) {
@@ -4468,8 +4465,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     public final Map<String, String> columnListParamToParentCursorMap;
 
     public FunctionParamInfo() {
-      cursorPosToSelectMap = new HashMap<Integer, SqlSelect>();
-      columnListParamToParentCursorMap = new HashMap<String, String>();
+      cursorPosToSelectMap = new HashMap<>();
+      columnListParamToParentCursorMap = new HashMap<>();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/600e1caf/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index b3169c5..7f73a08 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -162,15 +162,16 @@ import com.google.common.collect.Sets;
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
 import java.util.AbstractList;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Deque;
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.Stack;
 import java.util.TreeSet;
 import java.util.logging.Level;
 import java.util.logging.Logger;
@@ -231,7 +232,7 @@ public class SqlToRelConverter {
    * Stack of names of datasets requested by the <code>
    * TABLE(SAMPLE(&lt;datasetName&gt;, &lt;query&gt;))</code> construct.
    */
-  private final Stack<String> datasetStack = new Stack<>();
+  private final Deque<String> datasetStack = new ArrayDeque<>();
 
   /**
    * Mapping of non-correlated subqueries that have been converted to their

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/600e1caf/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
index 8c82119..6c53bc7 100644
--- a/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
+++ b/core/src/test/java/org/apache/calcite/test/concurrent/ConcurrentTestCommandScript.java
@@ -36,13 +36,14 @@ import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.sql.Timestamp;
+import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Deque;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.Stack;
 import java.util.StringTokenizer;
 import java.util.TreeMap;
 import java.util.TreeSet;
@@ -237,21 +238,19 @@ public class ConcurrentTestCommandScript
   private File scriptDirectory;
   private long scriptStartTime = 0;
 
-  private final List<ConcurrentTestPlugin> plugins =
-      new ArrayList<ConcurrentTestPlugin>();
+  private final List<ConcurrentTestPlugin> plugins = new ArrayList<>();
   private final Map<String, ConcurrentTestPlugin> pluginForCommand =
-      new HashMap<String, ConcurrentTestPlugin>();
+      new HashMap<>();
   private final Map<String, ConcurrentTestPlugin> preSetupPluginForCommand =
-      new HashMap<String, ConcurrentTestPlugin>();
-  private List<String> setupCommands = new ArrayList<String>();
-  private List<String> cleanupCommands = new ArrayList<String>();
+      new HashMap<>();
+  private final List<String> setupCommands = new ArrayList<>();
+  private final List<String> cleanupCommands = new ArrayList<>();
 
-  private Map<Integer, BufferedWriter> threadBufferedWriters =
-      new HashMap<Integer, BufferedWriter>();
-  private Map<Integer, StringWriter> threadStringWriters =
-      new HashMap<Integer, StringWriter>();
-  private Map<Integer, ResultsReader> threadResultsReaders =
-      new HashMap<Integer, ResultsReader>();
+  private final Map<Integer, BufferedWriter> threadBufferedWriters =
+      new HashMap<>();
+  private final Map<Integer, StringWriter> threadStringWriters = new HashMap<>();
+  private final Map<Integer, ResultsReader> threadResultsReaders =
+      new HashMap<>();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -328,11 +327,7 @@ public class ConcurrentTestCommandScript
 
 
   public boolean useLockstep() {
-    if (lockstep == null) {
-      return false;
-    }
-
-    return lockstep.booleanValue();
+    return lockstep != null && lockstep;
   }
 
   public boolean isDisabled() {
@@ -342,11 +337,7 @@ public class ConcurrentTestCommandScript
       }
     }
 
-    if (disabled == null) {
-      return false;
-    }
-
-    return disabled.booleanValue();
+    return disabled != null && disabled;
   }
 
   public void executeSetup() throws Exception {
@@ -396,24 +387,20 @@ public class ConcurrentTestCommandScript
         }
 
         if (isSelect(sql)) {
-          Statement stmt = connection.createStatement();
-          try {
+          try (Statement stmt = connection.createStatement()) {
             ResultSet rset = stmt.executeQuery(sql);
             storeResults(threadId, rset, -1);
-          } finally {
-            stmt.close();
           }
         } else if (sql.equalsIgnoreCase("commit")) {
           connection.commit();
         } else if (sql.equalsIgnoreCase("rollback")) {
           connection.rollback();
         } else {
-          Statement stmt = connection.createStatement();
-          try {
+          try (Statement stmt = connection.createStatement()) {
             int rows = stmt.executeUpdate(sql);
             if (rows != 1) {
-              storeMessage(threadId,
-                  String.valueOf(rows) + " rows affected.");
+              storeMessage(
+                  threadId, String.valueOf(rows) + " rows affected.");
             } else {
               storeMessage(threadId, "1 row affected.");
             }
@@ -423,8 +410,6 @@ public class ConcurrentTestCommandScript
             } else {
               throw ex;
             }
-          } finally {
-            stmt.close();
           }
         }
       }
@@ -496,10 +481,10 @@ public class ConcurrentTestCommandScript
    * @return the map.
    */
   private Map<Integer, String[]> collectResults() {
-    TreeMap<Integer, String[]> results = new TreeMap<Integer, String[]>();
+    final TreeMap<Integer, String[]> results = new TreeMap<>();
 
     // get all normal threads
-    TreeSet<Integer> threadIds = new TreeSet<Integer>(getThreadIds());
+    final TreeSet<Integer> threadIds = new TreeSet<>(getThreadIds());
     // add the "special threads"
     threadIds.add(SETUP_THREAD_ID);
     threadIds.add(CLEANUP_THREAD_ID);
@@ -694,7 +679,7 @@ public class ConcurrentTestCommandScript
         Pattern.compile("\\$((\\$)|([A-Za-z]\\w*)|\\{([A-Za-z]\\w*)\\})");
 
     public VariableTable() {
-      map = new HashMap<String, String>();
+      map = new HashMap<>();
     }
 
     /** Exception. */
@@ -740,13 +725,13 @@ public class ConcurrentTestCommandScript
 
     public String expand(String in) {
       if (in.contains("$")) {
-        StringBuffer out = new StringBuffer();
+        StringBuilder out = new StringBuilder();
         Matcher matcher = symbolPattern.matcher(in);
         int lastEnd = 0;
         while (matcher.find()) {
           int start = matcher.start();
           int end = matcher.end();
-          String val = null;
+          String val;
           if (null != matcher.group(2)) {
             val = "$";          // matched $$
           } else {
@@ -790,7 +775,7 @@ public class ConcurrentTestCommandScript
     private int order;
     private int repeatCount;
     private boolean scriptHasVars;
-    private Stack<File> currentDirectory = new Stack<File>();
+    private final Deque<File> currentDirectory = new ArrayDeque<>();
 
     /** Binding of a value to a variable. */
     private class Binding {
@@ -813,7 +798,7 @@ public class ConcurrentTestCommandScript
 
     // A list of Bindings that must be applied immediately after parsing
     // last @var.
-    private List<Binding> deferredBindings = new ArrayList<Binding>();
+    private final List<Binding> deferredBindings = new ArrayList<>();
 
     public CommandParser() {
       state = PRE_SETUP_STATE;
@@ -862,13 +847,12 @@ public class ConcurrentTestCommandScript
     private void load(String scriptFileName) throws IOException {
       File scriptFile = new File(currentDirectory.peek(), scriptFileName);
       currentDirectory.push(scriptDirectory = scriptFile.getParentFile());
-      BufferedReader in = new BufferedReader(new FileReader(scriptFile));
-      try {
+      try (BufferedReader in = new BufferedReader(new FileReader(scriptFile))) {
         String line;
         while ((line = in.readLine()) != null) {
           line = line.trim();
           Map<String, String> commandStateMap = lookupState(state);
-          String command = null;
+          final String command;
           boolean isSql = false;
           if (line.equals("") || line.startsWith("--")) {
             continue;
@@ -910,8 +894,6 @@ public class ConcurrentTestCommandScript
                 "Premature end of file in '" + state + "' state");
           }
         }
-      } finally {
-        in.close();
       }
     }
 
@@ -1242,8 +1224,8 @@ public class ConcurrentTestCommandScript
       for (int i = 0, n = STATE_TABLE.length; i < n; i++) {
         if (state.equals(STATE_TABLE[i].state)) {
           StateDatum[] stateData = STATE_TABLE[i].stateData;
-          ArrayList<StateDatum> stateDataList =
-              new ArrayList<StateDatum>(Arrays.asList(stateData));
+          final List<StateDatum> stateDataList =
+              new ArrayList<>(Arrays.asList(stateData));
           for (String cmd : commands) {
             stateDataList.add(new StateDatum(cmd, state));
           }
@@ -1263,13 +1245,13 @@ public class ConcurrentTestCommandScript
     private Map<String, String> lookupState(String state) {
       assert state != null;
 
-      for (int i = 0, n = STATE_TABLE.length; i < n; i++) {
-        if (state.equals(STATE_TABLE[i].state)) {
-          StateDatum[] stateData = STATE_TABLE[i].stateData;
+      for (StateAction a : STATE_TABLE) {
+        if (state.equals(a.state)) {
+          StateDatum[] stateData = a.stateData;
 
           Map<String, String> result = new HashMap<String, String>();
-          for (int j = 0, m = stateData.length; j < m; j++) {
-            result.put(stateData[j].x, stateData[j].y);
+          for (StateDatum datum : stateData) {
+            result.put(datum.x, datum.y);
           }
           return result;
         }
@@ -1306,7 +1288,7 @@ public class ConcurrentTestCommandScript
       boolean more = line.endsWith("\\");
       if (more) {
         line = line.substring(0, line.lastIndexOf('\\')); // snip
-        StringBuffer buf = new StringBuffer(line);        // save
+        StringBuilder buf = new StringBuilder(line);        // save
         while (more) {
           line = in.readLine();
           if (line == null) {
@@ -1337,7 +1319,7 @@ public class ConcurrentTestCommandScript
     private String readSql(String startOfSql, BufferedReader in)
         throws IOException {
       // REVIEW mb StringBuffer not always needed
-      StringBuffer sql = new StringBuffer(startOfSql);
+      StringBuilder sql = new StringBuilder(startOfSql);
       sql.append('\n');
 
       String line;
@@ -1484,7 +1466,7 @@ public class ConcurrentTestCommandScript
       this.command = command;
       boolean needShell = hasWildcard(command);
       if (needShell) {
-        argv = new ArrayList<String>();
+        argv = new ArrayList<>();
         argv.add("/bin/sh");
         argv.add("-c");
         argv.add(command);
@@ -1498,7 +1480,7 @@ public class ConcurrentTestCommandScript
     }
 
     private List<String> tokenize(String s) {
-      List<String> result = new ArrayList<String>();
+      List<String> result = new ArrayList<>();
       StringTokenizer tokenizer = new StringTokenizer(s);
       while (tokenizer.hasMoreTokens()) {
         result.add(tokenizer.nextToken());
@@ -2013,8 +1995,8 @@ public class ConcurrentTestCommandScript
     List<String> files;             // FILE
 
     public Tool() {
-      bindings = new ArrayList<String>();
-      files = new ArrayList<String>();
+      bindings = new ArrayList<>();
+      files = new ArrayList<>();
     }
 
     // returns 0 on success, 1 on error, 2 on bad invocation.


[2/2] incubator-calcite git commit: [CALCITE-881] Allow schema.table.column references in GROUP BY

Posted by jh...@apache.org.
[CALCITE-881] Allow schema.table.column references in GROUP BY

To keep the horse in front of the cart, move a few fields and methods from
AggregatingSelectScope to a new class Resolved, which is created after the
scope tree is built, i.e. during validation.


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

Branch: refs/heads/master
Commit: 16512edc1308b6643bbdba901996a4b0a5c014e9
Parents: 600e1ca
Author: Julian Hyde <jh...@apache.org>
Authored: Wed Sep 9 16:57:12 2015 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Sep 9 17:00:16 2015 -0700

----------------------------------------------------------------------
 .../sql/fun/SqlAbstractGroupFunction.java       |   3 +-
 .../sql/validate/AggregatingSelectScope.java    | 106 ++++++++++++-------
 .../calcite/sql/validate/DelegatingScope.java   |  10 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |  32 +++---
 .../apache/calcite/test/SqlValidatorTest.java   |  23 ++--
 core/src/test/resources/sql/misc.oq             |  31 ++++++
 6 files changed, 142 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
index 0d9d2ea..443bd4f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlAbstractGroupFunction.java
@@ -63,6 +63,7 @@ public class SqlAbstractGroupFunction extends SqlFunction {
     super.validateCall(call, validator, scope, operandScope);
     final SelectScope selectScope =
         SqlValidatorUtil.getEnclosingSelectScope(scope);
+    assert selectScope != null;
     final SqlSelect select = selectScope.getNode();
     if (!validator.isAggregate(select)) {
       throw validator.newValidationError(call,
@@ -81,7 +82,7 @@ public class SqlAbstractGroupFunction extends SqlFunction {
       } else {
         operand = validator.expand(operand, scope);
       }
-      if (!aggregatingSelectScope.isGroupingExpr(operand)) {
+      if (!aggregatingSelectScope.resolved.get().isGroupingExpr(operand)) {
         throw validator.newValidationError(operand,
             Static.RESOURCE.groupingArgument(getName()));
       }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/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 58fda81..580e71c 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
@@ -25,12 +25,14 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.util.ImmutableBitSet;
 
+import com.google.common.base.Supplier;
+import com.google.common.base.Suppliers;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
 import com.google.common.collect.Sets;
 
+import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -52,15 +54,21 @@ public class AggregatingSelectScope
   private final boolean distinct;
 
   /** Use while under construction. */
-  private final List<SqlNode> temporaryGroupExprList = Lists.newArrayList();
+  private List<SqlNode> temporaryGroupExprList;
 
-  /** Use after construction is complete. Assigned from
-   * {@link #temporaryGroupExprList} towards the end of the constructor. */
-  public final ImmutableList<SqlNode> groupExprList;
-  public final ImmutableBitSet groupSet;
-  public final ImmutableList<ImmutableBitSet> groupSets;
-  public final boolean indicator;
-  public final Map<Integer, Integer> groupExprProjection;
+  public final Supplier<Resolved> resolved =
+      Suppliers.memoize(
+          new Supplier<Resolved>() {
+            public Resolved get() {
+              assert temporaryGroupExprList == null;
+              temporaryGroupExprList = new ArrayList<>();
+              try {
+                return resolve();
+              } finally {
+                temporaryGroupExprList = null;
+              }
+            }
+          });
 
   //~ Constructors -----------------------------------------------------------
 
@@ -81,7 +89,12 @@ public class AggregatingSelectScope
     super(selectScope);
     this.select = select;
     this.distinct = distinct;
-    final Map<Integer, Integer> groupExprProjection = Maps.newHashMap();
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  private Resolved resolve() {
+    final Map<Integer, Integer> groupExprProjection = new HashMap<>();
     final ImmutableList.Builder<ImmutableList<ImmutableBitSet>> builder =
         ImmutableList.builder();
     if (select.getGroup() != null) {
@@ -95,8 +108,6 @@ public class AggregatingSelectScope
             groupExprProjection, builder, groupExpr);
       }
     }
-    this.groupExprList = ImmutableList.copyOf(temporaryGroupExprList);
-    this.groupExprProjection = ImmutableMap.copyOf(groupExprProjection);
 
     final Set<ImmutableBitSet> flatGroupSets =
         Sets.newTreeSet(ImmutableBitSet.COMPARATOR);
@@ -109,13 +120,10 @@ public class AggregatingSelectScope
       flatGroupSets.add(ImmutableBitSet.of());
     }
 
-    this.groupSet = ImmutableBitSet.range(groupExprList.size());
-    this.groupSets = ImmutableList.copyOf(flatGroupSets);
-    this.indicator = !groupSets.equals(ImmutableList.of(groupSet));
+    return new Resolved(temporaryGroupExprList, flatGroupSets,
+        groupExprProjection);
   }
 
-  //~ Methods ----------------------------------------------------------------
-
   /**
    * Returns the expressions that are in the GROUP BY clause (or the SELECT
    * DISTINCT clause, if distinct) and that can therefore be referenced
@@ -141,10 +149,11 @@ public class AggregatingSelectScope
       }
       return groupExprs.build();
     } else if (select.getGroup() != null) {
-      if (groupExprList != null) {
-        return groupExprList;
-      } else {
+      if (temporaryGroupExprList != null) {
+        // we are in the middle of resolving
         return ImmutableList.copyOf(temporaryGroupExprList);
+      } else {
+        return resolved.get().groupExprList;
       }
     } else {
       return ImmutableList.of();
@@ -155,11 +164,6 @@ public class AggregatingSelectScope
     return select;
   }
 
-  /** Returns whether a field should be nullable due to grouping sets. */
-  public boolean isNullable(int i) {
-    return i < groupExprList.size() && !allContain(groupSets, i);
-  }
-
   private static boolean allContain(List<ImmutableBitSet> bitSets, int bit) {
     for (ImmutableBitSet bitSet : bitSets) {
       if (!bitSet.get(bit)) {
@@ -170,9 +174,10 @@ public class AggregatingSelectScope
   }
 
   @Override public RelDataType nullifyType(SqlNode node, RelDataType type) {
-    for (Ord<SqlNode> groupExpr : Ord.zip(groupExprList)) {
+    final Resolved r = this.resolved.get();
+    for (Ord<SqlNode> groupExpr : Ord.zip(r.groupExprList)) {
       if (groupExpr.e.equalsDeep(node, false)) {
-        if (isNullable(groupExpr.i)) {
+        if (r.isNullable(groupExpr.i)) {
           return validator.getTypeFactory().createTypeWithNullability(type,
               true);
         }
@@ -235,19 +240,44 @@ public class AggregatingSelectScope
     checkAggregateExpr(expr, true);
   }
 
-  /** Returns whether a given expression is equal to one of the grouping
-   * expressions. Determines whether it is valid as an operand to GROUPING. */
-  public boolean isGroupingExpr(SqlNode operand) {
-    return lookupGroupingExpr(operand) >= 0;
-  }
+  /** Information about an aggregating scope that can only be determined
+   * after validation has occurred. Therefore it cannot be populated when
+   * the scope is created. */
+  public class Resolved {
+    public final ImmutableList<SqlNode> groupExprList;
+    public final ImmutableBitSet groupSet;
+    public final ImmutableList<ImmutableBitSet> groupSets;
+    public final boolean indicator;
+    public final Map<Integer, Integer> groupExprProjection;
 
-  public int lookupGroupingExpr(SqlNode operand) {
-    for (Ord<SqlNode> groupExpr : Ord.zip(groupExprList)) {
-      if (operand.equalsDeep(groupExpr.e, false)) {
-        return groupExpr.i;
+    Resolved(List<SqlNode> groupExprList, Iterable<ImmutableBitSet> groupSets,
+        Map<Integer, Integer> groupExprProjection) {
+      this.groupExprList = ImmutableList.copyOf(groupExprList);
+      this.groupSet = ImmutableBitSet.range(groupExprList.size());
+      this.groupSets = ImmutableList.copyOf(groupSets);
+      this.indicator = !this.groupSets.equals(ImmutableList.of(groupSet));
+      this.groupExprProjection = ImmutableMap.copyOf(groupExprProjection);
+    }
+
+    /** Returns whether a field should be nullable due to grouping sets. */
+    public boolean isNullable(int i) {
+      return i < groupExprList.size() && !allContain(groupSets, i);
+    }
+
+    /** Returns whether a given expression is equal to one of the grouping
+     * expressions. Determines whether it is valid as an operand to GROUPING. */
+    public boolean isGroupingExpr(SqlNode operand) {
+      return lookupGroupingExpr(operand) >= 0;
+    }
+
+    public int lookupGroupingExpr(SqlNode operand) {
+      for (Ord<SqlNode> groupExpr : Ord.zip(groupExprList)) {
+        if (operand.equalsDeep(groupExpr.e, false)) {
+          return groupExpr.i;
+        }
       }
+      return -1;
     }
-    return -1;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/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 dc50668..d5049e7 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
@@ -198,6 +198,16 @@ public abstract class DelegatingScope implements SqlValidatorScope {
         identifier = identifier.setName(j, field.getName());
         fromRowType = field.getType();
       }
+      if (i > 1) {
+        // Simplify overqualified identifiers.
+        // For example, schema.emp.deptno becomes emp.deptno.
+        //
+        // It is safe to convert schema.emp or database.schema.emp to emp
+        // because it would not have resolved if the FROM item had an alias. The
+        // following query is invalid:
+        //   SELECT schema.emp.deptno FROM schema.emp AS e
+        identifier = identifier.getComponent(i - 1, identifier.names.size());
+      }
       return SqlQualified.create(this, i, fromNs, identifier);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 7f73a08..b51a0dc 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -2461,11 +2461,12 @@ public class SqlToRelConverter {
     // build a map to remember the projections from the top scope to the
     // output of the current root.
     //
-    // Currently farrago allows expressions, not just column references in
-    // group by list. This is not SQL 2003 compliant.
+    // Calcite allows expressions, not just column references in
+    // group by list. This is not SQL 2003 compliant, but hey.
 
     final AggregatingSelectScope scope = aggConverter.aggregatingSelectScope;
-    for (SqlNode groupExpr : scope.groupExprList) {
+    final AggregatingSelectScope.Resolved r = scope.resolved.get();
+    for (SqlNode groupExpr : r.groupExprList) {
       aggConverter.addGroupExpr(groupExpr);
     }
 
@@ -2496,8 +2497,8 @@ public class SqlToRelConverter {
         // at all.  The rest of the system doesn't like 0-tuples, so we
         // select a dummy constant here.
         preExprs =
-            Collections.singletonList(
-                (RexNode) rexBuilder.makeExactLiteral(BigDecimal.ZERO));
+            ImmutableList.<RexNode>of(
+                rexBuilder.makeExactLiteral(BigDecimal.ZERO));
         preNames = Collections.singletonList(null);
       }
 
@@ -2511,7 +2512,7 @@ public class SqlToRelConverter {
               preNames,
               true),
           false);
-      bb.mapRootRelToFieldProjection.put(bb.root, scope.groupExprProjection);
+      bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection);
 
       // REVIEW jvs 31-Oct-2007:  doesn't the declaration of
       // monotonicity here assume sort-based aggregation at
@@ -2526,8 +2527,8 @@ public class SqlToRelConverter {
 
       // Add the aggregator
       bb.setRoot(
-          createAggregate(bb, aggConverter.aggregatingSelectScope.indicator,
-              scope.groupSet, scope.groupSets, aggConverter.getAggCalls()),
+          createAggregate(bb, r.indicator, r.groupSet, r.groupSets,
+              aggConverter.getAggCalls()),
           false);
 
       // Generate NULL values for rolled-up not-null fields.
@@ -2540,7 +2541,7 @@ public class SqlToRelConverter {
         for (RelDataTypeField field : aggregate.getRowType().getFieldList()) {
           final int i = field.getIndex();
           final RexNode rex;
-          if (i < groupCount && scope.isNullable(i)) {
+          if (i < groupCount && r.isNullable(i)) {
             ++converted;
 
             rex = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
@@ -2562,7 +2563,7 @@ public class SqlToRelConverter {
         }
       }
 
-      bb.mapRootRelToFieldProjection.put(bb.root, scope.groupExprProjection);
+      bb.mapRootRelToFieldProjection.put(bb.root, r.groupExprProjection);
 
       // Replace subqueries in having here and modify having to use
       // the replaced expressions
@@ -4498,11 +4499,13 @@ public class SqlToRelConverter {
               filterArg,
               type,
               nameMap.get(outerCall.toString()));
+      final AggregatingSelectScope.Resolved r =
+          aggregatingSelectScope.resolved.get();
       RexNode rex =
           rexBuilder.addAggCall(
               aggCall,
               groupExprs.size(),
-              aggregatingSelectScope.indicator,
+              r.indicator,
               aggCalls,
               aggCallMapping,
               argTypes);
@@ -4547,7 +4550,7 @@ public class SqlToRelConverter {
       case GROUPING_ID:
       case GROUP_ID:
         final RelDataType type = validator.getValidatedNodeType(call);
-        if (!aggregatingSelectScope.indicator) {
+        if (!aggregatingSelectScope.resolved.get().indicator) {
           return rexBuilder.makeExactLiteral(
               TWO.pow(effectiveArgCount(call)).subtract(BigDecimal.ONE), type);
         } else {
@@ -4590,9 +4593,10 @@ public class SqlToRelConverter {
 
     private RexNode bitValue(RexNode previous, RelDataType type, int x,
         int shift) {
+      final AggregatingSelectScope.Resolved r =
+          aggregatingSelectScope.resolved.get();
       RexNode node = rexBuilder.makeCall(SqlStdOperatorTable.CASE,
-          rexBuilder.makeInputRef(bb.root,
-              aggregatingSelectScope.groupExprList.size() + x),
+          rexBuilder.makeInputRef(bb.root, r.groupExprList.size() + x),
           rexBuilder.makeExactLiteral(BigDecimal.ONE, type),
           rexBuilder.makeExactLiteral(BigDecimal.ZERO, type));
       if (shift > 0) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/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 78d9e1c..e161b26 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4780,9 +4780,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .fails("Table 'SALES.BAD' not found");
   }
 
-  @Ignore("does not work yet")
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-881">[CALCITE-881]
+   * Allow schema.table.column references in GROUP BY</a>. */
   @Test public void testSchemaTableColumnInGroupBy() {
-    sql("select 1 from sales.emp group by sales.emp.deptno").ok(); // TODO:
+    sql("select 1 from sales.emp group by sales.emp.deptno").ok();
+    sql("select deptno from sales.emp group by sales.emp.deptno").ok();
+    sql("select deptno + 1 from sales.emp group by sales.emp.deptno").ok();
   }
 
   @Test public void testInvalidGroupBy() {
@@ -6043,14 +6047,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         .type("RecordType(INTEGER NOT NULL DEPTNO, INTEGER EMPNO) NOT NULL");
   }
 
-  @Test public void testGroupByCorrelatedColumnFails() {
-    // -- this is not sql 2003 standard
-    // -- see sql2003 part2,  7.9
-    checkFails(
-        "select count(*)\n"
-            + "from emp\n"
-            + "where exists (select count(*) from dept group by ^emp^.empno)",
-        "Table 'EMP' not found");
+  @Test public void testGroupByCorrelatedColumn() {
+    // This is not sql 2003 standard; see sql2003 part2,  7.9
+    // But the extension seems harmless.
+    final String sql = "select count(*)\n"
+        + "from emp\n"
+        + "where exists (select count(*) from dept group by emp.empno)";
+    sql(sql).ok();
   }
 
   @Test public void testGroupExpressionEquivalence() {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/16512edc/core/src/test/resources/sql/misc.oq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.oq b/core/src/test/resources/sql/misc.oq
index 1d1bd57..2b9b3c9 100644
--- a/core/src/test/resources/sql/misc.oq
+++ b/core/src/test/resources/sql/misc.oq
@@ -33,6 +33,37 @@ from "hr"."emps";
 
 !ok
 
+# [CALCITE-881] Allow schema.table.column references in GROUP BY
+select "hr"."emps"."empid", count(*) as c
+from "hr"."emps"
+group by "hr"."emps"."empid";
++-------+---+
+| empid | C |
++-------+---+
+|   100 | 1 |
+|   110 | 1 |
+|   150 | 1 |
+|   200 | 1 |
++-------+---+
+(4 rows)
+
+!ok
+
+select distinct "hr"."emps"."empid" + 1 as e
+from "hr"."emps"
+group by "hr"."emps"."empid";
++-----+
+| E   |
++-----+
+| 101 |
+| 111 |
+| 151 |
+| 201 |
++-----+
+(4 rows)
+
+!ok
+
 # [CALCITE-307] CAST(timestamp AS DATE) gives ClassCastException
 # Based on [DRILL-1051]
 with data(c_row, c_timestamp) as (select * from (values