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

[2/2] incubator-calcite git commit: [CALCITE-356] Allow column references of the form schema.table.column

[CALCITE-356] Allow column references of the form schema.table.column

SqlValidatorScope.fullyQualify now returns a new class SqlQualified, which contains more information than just name components after an identifier has been fully-qualified in a scope. We plan to cache these, at which point we should be able to make sql-to-rel translation faster and more predictable, since it builds on work already done by the validator.


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

Branch: refs/heads/master
Commit: 6c88546ea8a0deb544df57536ed24466fbb46b85
Parents: 0fda7af
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Dec 18 13:24:37 2014 -0800
Committer: julianhyde <jh...@apache.org>
Committed: Wed Dec 31 20:35:59 2014 -0800

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  36 +--
 .../calcite/prepare/CalciteCatalogReader.java   |   6 +-
 .../calcite/rel/type/RelDataTypeImpl.java       |   2 +-
 .../java/org/apache/calcite/sql/SqlCall.java    |   3 +-
 .../org/apache/calcite/sql/SqlIdentifier.java   |  47 ++-
 .../java/org/apache/calcite/sql/SqlNode.java    |   5 +-
 .../org/apache/calcite/sql/SqlOperator.java     |   3 +-
 .../java/org/apache/calcite/sql/SqlUtil.java    |  99 ++++++
 .../apache/calcite/sql/advise/SqlAdvisor.java   |  60 +++-
 .../calcite/sql/advise/SqlSimpleParser.java     |   8 +-
 .../apache/calcite/sql/parser/SqlParserPos.java | 116 ++++---
 .../apache/calcite/sql/validate/AggChecker.java |   4 +-
 .../sql/validate/AggregatingSelectScope.java    |   4 +-
 .../calcite/sql/validate/CatalogScope.java      |  88 ++++++
 .../calcite/sql/validate/DelegatingScope.java   |  76 +++--
 .../apache/calcite/sql/validate/EmptyScope.java |  13 +-
 .../apache/calcite/sql/validate/ListScope.java  |  59 +++-
 .../calcite/sql/validate/OrderByScope.java      |   4 +-
 .../calcite/sql/validate/ParameterScope.java    |  10 +-
 .../calcite/sql/validate/SchemaNamespace.java   |  60 ++++
 .../calcite/sql/validate/SelectScope.java       |   2 +-
 .../apache/calcite/sql/validate/SqlMoniker.java |  18 ++
 .../sql/validate/SqlMonikerComparator.java      |  39 ---
 .../calcite/sql/validate/SqlMonikerImpl.java    |  19 +-
 .../calcite/sql/validate/SqlMonikerType.java    |   2 +-
 .../calcite/sql/validate/SqlQualified.java      |  86 ++++++
 .../sql/validate/SqlValidatorCatalogReader.java |   2 +
 .../calcite/sql/validate/SqlValidatorImpl.java  | 306 ++++++++++---------
 .../calcite/sql/validate/SqlValidatorScope.java |  17 +-
 .../calcite/sql/validate/SqlValidatorUtil.java  |  24 +-
 .../sql/validate/SqlValidatorWithHints.java     |   4 +-
 .../apache/calcite/sql/validate/WithScope.java  |   6 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  42 +--
 .../main/java/org/apache/calcite/util/Bug.java  |   7 -
 .../main/java/org/apache/calcite/util/Util.java |  20 +-
 .../calcite/sql/parser/SqlParserTest.java       |  21 ++
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  25 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |  50 +--
 .../apache/calcite/test/MockCatalogReader.java  |  53 +++-
 .../apache/calcite/test/SqlValidatorTest.java   |  94 +++++-
 core/src/test/resources/sql/misc.oq             |  15 +
 41 files changed, 1078 insertions(+), 477 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/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 4b28e3b..5f540e7 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -1411,21 +1411,9 @@ SqlNode SelectItem() :
 SqlNode SelectExpression() :
 {
     SqlNode e;
-    String id;
-    SqlParserPos pos, starPos;
+    SqlParserPos pos;
 }
 {
-    LOOKAHEAD(3)
-    id = Identifier() { pos = getPos(); } <DOT> <STAR>
-    {
-        starPos = getPos();
-        return new SqlIdentifier(
-            ImmutableList.of(id, "*"),
-            null,
-            pos.plus(starPos),
-            ImmutableList.of(pos, starPos));
-    }
-    |
     <STAR>
     {
         pos = getPos();
@@ -3504,17 +3492,21 @@ SqlIdentifier CompoundIdentifier() :
         list.add(p);
     }
     (
-        <DOT> p = Identifier()
-        {
-            list.add(p);
-            posList.add(getPos());
-        }
+        <DOT>
+        (
+            p = Identifier() {
+                list.add(p);
+                posList.add(getPos());
+            }
+        |
+            <STAR> {
+                list.add("*");
+                posList.add(getPos());
+            }
+        )
     ) *
     {
-        SqlParserPos[] componentPositions =
-            (SqlParserPos []) posList.toArray(
-                new SqlParserPos[posList.size()]);
-        SqlParserPos pos = SqlParserPos.sum(componentPositions);
+        SqlParserPos pos = SqlParserPos.sum(posList);
         return new SqlIdentifier(list, null, pos, posList);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
index 0f4b4f5..15cf374 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -209,8 +209,12 @@ public class CalciteCatalogReader implements Prepare.CatalogReader,
     return field != null ? field.getIndex() : -1;
   }
 
+  public boolean matches(String string, String name) {
+    return Util.matches(caseSensitive, string, name);
+  }
+
   public int match(List<String> strings, String name) {
-    return Util.match2(strings, name, caseSensitive);
+    return Util.findMatch(strings, name, caseSensitive);
   }
 
   public RelDataType createTypeFromProjection(final RelDataType type,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
index d917667..3b13b9b 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeImpl.java
@@ -79,7 +79,7 @@ public abstract class RelDataTypeImpl
   // implement RelDataType
   public RelDataTypeField getField(String fieldName, boolean caseSensitive) {
     for (RelDataTypeField field : fieldList) {
-      if (Util.match(caseSensitive, field.getName(), fieldName)) {
+      if (Util.matches(caseSensitive, field.getName(), fieldName)) {
         return field;
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 07f5f5b..7548e01 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 /**
@@ -110,7 +111,7 @@ public abstract class SqlCall extends SqlNode {
       SqlValidator validator,
       SqlValidatorScope scope,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     for (SqlNode operand : getOperandList()) {
       if (operand instanceof SqlIdentifier) {
         SqlIdentifier id = (SqlIdentifier) operand;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
index c772331..c5e2420 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -19,10 +19,9 @@ package org.apache.calcite.sql;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlQualified;
 import org.apache.calcite.sql.validate.SqlValidator;
-import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
-import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -168,8 +167,41 @@ public class SqlIdentifier extends SqlNode {
    * available.
    */
   public SqlIdentifier getComponent(int ordinal) {
-    return new SqlIdentifier(names.get(ordinal),
-        getComponentParserPosition(ordinal));
+    return getComponent(ordinal, ordinal + 1);
+  }
+
+  public SqlIdentifier getComponent(int from, int to) {
+    final SqlParserPos pos;
+    final ImmutableList<SqlParserPos> pos2;
+    if (componentPositions == null) {
+      pos2 = null;
+      pos = this.pos;
+    } else {
+      pos2 = componentPositions.subList(from, to);
+      pos = SqlParserPos.sum(pos2);
+    }
+    return new SqlIdentifier(names.subList(from, to), collation, pos, pos2);
+  }
+
+  /**
+   * Creates an identifier that consists of this identifier plus a name segment.
+   * Does not modify this identifier.
+   */
+  public SqlIdentifier plus(String name, SqlParserPos pos) {
+    final ImmutableList<String> names =
+        ImmutableList.<String>builder().addAll(this.names).add(name).build();
+    final ImmutableList.Builder<SqlParserPos> builder = ImmutableList.builder();
+    final ImmutableList<SqlParserPos> componentPositions =
+        builder.addAll(this.componentPositions).add(pos).build();
+    final SqlParserPos pos2 =
+        SqlParserPos.sum(builder.add(this.pos).build());
+    return new SqlIdentifier(names, collation, pos2, componentPositions);
+  }
+
+  /** Creates an identifier that consists of all but the last {@code n}
+   * name segments of this one. */
+  public SqlIdentifier skipLast(int n) {
+    return getComponent(0, names.size() - n);
   }
 
   public void unparse(
@@ -270,10 +302,9 @@ public class SqlIdentifier extends SqlNode {
     if (call != null) {
       return call.getMonotonicity(scope);
     }
-    final SqlIdentifier fqId = scope.fullyQualify(this);
-    final SqlValidatorNamespace ns =
-        SqlValidatorUtil.lookup(scope, Util.skipLast(fqId.names));
-    return ns.resolve().getMonotonicity(Util.last(fqId.names));
+    final SqlQualified qualified = scope.fullyQualify(this);
+    final SqlIdentifier fqId = qualified.identifier;
+    return qualified.namespace.resolve().getMonotonicity(Util.last(fqId.names));
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 75e3979..6708ba5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.Util;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Set;
 
@@ -43,7 +44,7 @@ public abstract class SqlNode implements Cloneable {
 
   //~ Instance fields --------------------------------------------------------
 
-  private final SqlParserPos pos;
+  protected final SqlParserPos pos;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -212,7 +213,7 @@ public abstract class SqlNode implements Cloneable {
       SqlValidator validator,
       SqlValidatorScope scope,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     // no valid options
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index cab1ecc..08325ab 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -30,6 +30,7 @@ import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Util;
 
+import java.util.Arrays;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -232,7 +233,7 @@ public abstract class SqlOperator {
       SqlLiteral functionQualifier,
       SqlParserPos pos,
       SqlNode... operands) {
-    pos = pos.plusAll(operands);
+    pos = pos.plusAll(Arrays.asList(operands));
     return new SqlBasicCall(this, operands, pos, false, functionQualifier);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
index e2430e7..4ee7ece 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlUtil.java
@@ -27,12 +27,15 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.sql.util.SqlBasicVisitor;
 import org.apache.calcite.util.BarfingInvocationHandler;
 import org.apache.calcite.util.ConversionUtil;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.nio.charset.Charset;
@@ -764,6 +767,23 @@ public abstract class SqlUtil {
     }
   }
 
+  /** Returns a list of ancestors of {@code predicate} within a given
+   * {@code SqlNode} tree.
+   *
+   * <p>The first element of the list is {@code root}, and the last is
+   * the node that matched {@code predicate}. Throws if no node matches.
+   */
+  public static ImmutableList<SqlNode> getAncestry(SqlNode root,
+      Predicate<SqlNode> predicate, Predicate<SqlNode> postPredicate) {
+    try {
+      new Genealogist(predicate, postPredicate).visitChild(root);
+      throw new AssertionError("not found: " + predicate + " in " + root);
+    } catch (Util.FoundOne e) {
+      //noinspection unchecked
+      return (ImmutableList<SqlNode>) e.getNode();
+    }
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**
@@ -792,6 +812,85 @@ public abstract class SqlUtil {
       return identifierQuoteString;
     }
   }
+
+  /** Walks over a {@link org.apache.calcite.sql.SqlNode} tree and returns the
+   * ancestry stack when it finds a given node. */
+  private static class Genealogist extends SqlBasicVisitor<Void> {
+    private final List<SqlNode> ancestors = Lists.newArrayList();
+    private final Predicate<SqlNode> predicate;
+    private final Predicate<SqlNode> postPredicate;
+
+    Genealogist(Predicate<SqlNode> predicate,
+        Predicate<SqlNode> postPredicate) {
+      this.predicate = predicate;
+      this.postPredicate = postPredicate;
+    }
+
+    private Void check(SqlNode node) {
+      preCheck(node);
+      postCheck(node);
+      return null;
+    }
+
+    private Void preCheck(SqlNode node) {
+      if (predicate.apply(node)) {
+        throw new Util.FoundOne(ImmutableList.copyOf(ancestors));
+      }
+      return null;
+    }
+
+    private Void postCheck(SqlNode node) {
+      if (postPredicate.apply(node)) {
+        throw new Util.FoundOne(ImmutableList.copyOf(ancestors));
+      }
+      return null;
+    }
+
+    private void visitChild(SqlNode node) {
+      if (node == null) {
+        return;
+      }
+      ancestors.add(node);
+      node.accept(this);
+      ancestors.remove(ancestors.size() - 1);
+    }
+
+    @Override public Void visit(SqlIdentifier id) {
+      return check(id);
+    }
+
+    @Override public Void visit(SqlCall call) {
+      preCheck(call);
+      for (SqlNode node : call.getOperandList()) {
+        visitChild(node);
+      }
+      return postCheck(call);
+    }
+
+    @Override public Void visit(SqlIntervalQualifier intervalQualifier) {
+      return check(intervalQualifier);
+    }
+
+    @Override public Void visit(SqlLiteral literal) {
+      return check(literal);
+    }
+
+    @Override public Void visit(SqlNodeList nodeList) {
+      preCheck(nodeList);
+      for (SqlNode node : nodeList) {
+        visitChild(node);
+      }
+      return postCheck(nodeList);
+    }
+
+    @Override public Void visit(SqlDynamicParam param) {
+      return check(param);
+    }
+
+    @Override public Void visit(SqlDataTypeSpec type) {
+      return check(type);
+    }
+  }
 }
 
 // End SqlUtil.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
index 818d200..06d10c0 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlAdvisor.java
@@ -20,6 +20,8 @@ import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlSelect;
+import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.parser.SqlAbstractParserImpl;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
@@ -31,6 +33,10 @@ import org.apache.calcite.sql.validate.SqlValidatorWithHints;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
@@ -46,12 +52,13 @@ public class SqlAdvisor {
   //~ Static fields/initializers ---------------------------------------------
 
   public static final Logger LOGGER = CalciteTrace.PARSER_LOGGER;
+  private static final String HINT_TOKEN = "_suggest_";
+  private static final String UPPER_HINT_TOKEN = HINT_TOKEN.toUpperCase();
 
   //~ Instance fields --------------------------------------------------------
 
   // Flags indicating precision/scale combinations
   private final SqlValidatorWithHints validator;
-  private final String hintToken = "_suggest_";
 
   //~ Constructors -----------------------------------------------------------
 
@@ -68,7 +75,7 @@ public class SqlAdvisor {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Gets completion hints for a partially completed or syntatically incorrect
+   * Gets completion hints for a partially completed or syntactically incorrect
    * sql statement with cursor pointing to the position where completion hints
    * are requested.
    *
@@ -79,8 +86,8 @@ public class SqlAdvisor {
    * whitespace, the replaced string is empty. The replaced string is never
    * null.
    *
-   * @param sql      A partial or syntatically incorrect sql statement for which
-   *                 to retrieve completion hints
+   * @param sql      A partial or syntactically incorrect sql statement for
+   *                 which to retrieve completion hints
    * @param cursor   to indicate the 0-based cursor position in the query at
    * @param replaced String which is being replaced (output)
    * @return completion hints
@@ -164,7 +171,7 @@ public class SqlAdvisor {
 
   public List<SqlMoniker> getCompletionHints0(String sql, int cursor) {
     String simpleSql = simplifySql(sql, cursor);
-    int idx = simpleSql.indexOf(hintToken);
+    int idx = simpleSql.indexOf(HINT_TOKEN);
     if (idx < 0) {
       return Collections.emptyList();
     }
@@ -173,10 +180,10 @@ public class SqlAdvisor {
   }
 
   /**
-   * Gets completion hints for a syntatically correct sql statement with dummy
+   * Gets completion hints for a syntactically correct sql statement with dummy
    * SqlIdentifier
    *
-   * @param sql A syntatically correct sql statement for which to retrieve
+   * @param sql A syntactically correct sql statement for which to retrieve
    *            completion hints
    * @param pos to indicate the line and column position in the query at which
    *            completion hints need to be retrieved. For example, "select
@@ -205,6 +212,12 @@ public class SqlAdvisor {
         + sql.substring(x);
     tryParse(sql, hintList);
 
+    final SqlMoniker star =
+        new SqlMonikerImpl(ImmutableList.of("*"), SqlMonikerType.KEYWORD);
+    if (hintList.contains(star) && !isSelectListItem(sqlNode, pos)) {
+      hintList.remove(star);
+    }
+
     // Add the identifiers which are expected at the point of interest.
     try {
       validator.validate(sqlNode);
@@ -221,6 +234,29 @@ public class SqlAdvisor {
     return hintList;
   }
 
+  private static boolean isSelectListItem(SqlNode root,
+      final SqlParserPos pos) {
+    List<SqlNode> nodes = SqlUtil.getAncestry(root,
+        new Predicate<SqlNode>() {
+
+          public boolean apply(SqlNode input) {
+            return input instanceof SqlIdentifier
+                && Util.last(((SqlIdentifier) input).names)
+                    .equals(UPPER_HINT_TOKEN);
+          }
+        },
+        new Predicate<SqlNode>() {
+          public boolean apply(SqlNode input) {
+            return input.getParserPosition().startsAt(pos);
+          }
+        });
+    assert nodes.get(0) == root;
+    nodes = Lists.reverse(nodes);
+    return nodes.size() > 2
+        && nodes.get(2) instanceof SqlSelect
+        && nodes.get(1) == ((SqlSelect) nodes.get(2)).getSelectList();
+  }
+
   /**
    * Tries to parse a SQL statement.
    *
@@ -289,11 +325,11 @@ public class SqlAdvisor {
    * Attempts to complete and validate a given partially completed sql
    * statement, and returns whether it is valid.
    *
-   * @param sql A partial or syntatically incorrect sql statement to validate
+   * @param sql A partial or syntactically incorrect sql statement to validate
    * @return whether SQL statement is valid
    */
   public boolean isValid(String sql) {
-    SqlSimpleParser simpleParser = new SqlSimpleParser(hintToken);
+    SqlSimpleParser simpleParser = new SqlSimpleParser(HINT_TOKEN);
     String simpleSql = simpleParser.simplifySql(sql);
     SqlNode sqlNode;
     try {
@@ -351,16 +387,16 @@ public class SqlAdvisor {
   }
 
   /**
-   * Turns a partially completed or syntatically incorrect sql statement into
+   * Turns a partially completed or syntactically incorrect sql statement into
    * a simplified, valid one that can be passed into getCompletionHints()
    *
-   * @param sql    A partial or syntatically incorrect sql statement
+   * @param sql    A partial or syntactically incorrect sql statement
    * @param cursor to indicate column position in the query at which
    *               completion hints need to be retrieved.
    * @return a completed, valid (and possibly simplified SQL statement
    */
   public String simplifySql(String sql, int cursor) {
-    SqlSimpleParser parser = new SqlSimpleParser(hintToken);
+    SqlSimpleParser parser = new SqlSimpleParser(HINT_TOKEN);
     return parser.simplifySql(sql, cursor);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
index 0bf8ce2..0ad0d32 100644
--- a/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/advise/SqlSimpleParser.java
@@ -105,7 +105,7 @@ public class SqlSimpleParser {
    * Turns a partially completed or syntactically incorrect sql statement into
    * a simplified, valid one that can be passed into getCompletionHints().
    *
-   * @param sql    A partial or syntatically incorrect sql statement
+   * @param sql    A partial or syntactically incorrect sql statement
    * @param cursor to indicate column position in the query at which
    *               completion hints need to be retrieved.
    * @return a completed, valid (and possibly simplified SQL statement
@@ -123,10 +123,10 @@ public class SqlSimpleParser {
   }
 
   /**
-   * Turns a partially completed or syntatically incorrect sql statement into
+   * Turns a partially completed or syntactically incorrect sql statement into
    * a simplified, valid one that can be validated
    *
-   * @param sql A partial or syntatically incorrect sql statement
+   * @param sql A partial or syntactically incorrect sql statement
    * @return a completed, valid (and possibly simplified) SQL statement
    */
   public String simplifySql(String sql) {
@@ -140,7 +140,7 @@ public class SqlSimpleParser {
       list.add(token);
     }
 
-    // Gather consecutive subsequences of tokens into subqueries.
+    // Gather consecutive sub-sequences of tokens into subqueries.
     List<Token> outList = new ArrayList<Token>();
     consumeQuery(list.listIterator(), outList);
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
index 95d704c..5028282 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserPos.java
@@ -18,7 +18,11 @@ package org.apache.calcite.sql.parser;
 
 import org.apache.calcite.sql.SqlNode;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+
 import java.io.Serializable;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 
@@ -125,14 +129,14 @@ public class SqlParserPos implements Serializable {
     return endColumnNumber;
   }
 
-  // implements Object
-  public String toString() {
+  @Override public String toString() {
     return RESOURCE.parserContext(lineNumber, columnNumber).str();
   }
 
   /**
-   * Combines this parser position with another to create a position which
-   * spans from the first point in the first to the last point in the other.
+   * Combines this parser position with another to create a
+   * position that spans from the first point in the first to the last point
+   * in the other.
    */
   public SqlParserPos plus(SqlParserPos pos) {
     return new SqlParserPos(
@@ -144,32 +148,40 @@ public class SqlParserPos implements Serializable {
 
   /**
    * Combines this parser position with an array of positions to create a
-   * position which spans from the first point in the first to the last point
+   * position that spans from the first point in the first to the last point
    * in the other.
    */
   public SqlParserPos plusAll(SqlNode[] nodes) {
-    int line = getLineNum();
-    int column = getColumnNum();
-    int endLine = getEndLineNum();
-    int endColumn = getEndColumnNum();
-    return sum(nodes, line, column, endLine, endColumn);
+    return plusAll(Arrays.asList(nodes));
   }
 
   /**
    * Combines this parser position with a list of positions.
    */
   public SqlParserPos plusAll(Collection<SqlNode> nodeList) {
-    final SqlNode[] nodes = nodeList.toArray(new SqlNode[nodeList.size()]);
-    return plusAll(nodes);
+    int line = getLineNum();
+    int column = getColumnNum();
+    int endLine = getEndLineNum();
+    int endColumn = getEndColumnNum();
+    return sum(toPos(nodeList), line, column, endLine, endColumn);
   }
 
   /**
    * Combines the parser positions of an array of nodes to create a position
    * which spans from the beginning of the first to the end of the last.
    */
-  public static SqlParserPos sum(
-      SqlNode[] nodes) {
-    return sum(nodes, Integer.MAX_VALUE, Integer.MAX_VALUE, -1, -1);
+  public static SqlParserPos sum(SqlNode[] nodes) {
+    final Iterable<SqlParserPos> poses = toPos(Arrays.asList(nodes));
+    return sum(poses, Integer.MAX_VALUE, Integer.MAX_VALUE, -1, -1);
+  }
+
+  private static Iterable<SqlParserPos> toPos(Iterable<SqlNode> nodes) {
+    return Iterables.transform(nodes,
+        new Function<SqlNode, SqlParserPos>() {
+          public SqlParserPos apply(SqlNode input) {
+            return input.getParserPosition();
+          }
+        });
   }
 
   /**
@@ -181,56 +193,10 @@ public class SqlParserPos implements Serializable {
   }
 
   /**
-   * Computes the parser position which is the sum of the positions of an
-   * array of parse tree nodes and of a parser position represented by (line,
-   * column, endLine, endColumn).
-   *
-   * @param nodes     Array of parse tree nodes
-   * @param line      Start line
-   * @param column    Start column
-   * @param endLine   End line
-   * @param endColumn End column
-   * @return Sum of parser positions
-   */
-  private static SqlParserPos sum(
-      SqlNode[] nodes,
-      int line,
-      int column,
-      int endLine,
-      int endColumn) {
-    int testLine;
-    int testColumn;
-    for (SqlNode node : nodes) {
-      if (node == null) {
-        continue;
-      }
-      SqlParserPos pos = node.getParserPosition();
-      if (pos.equals(SqlParserPos.ZERO)) {
-        continue;
-      }
-      testLine = pos.getLineNum();
-      testColumn = pos.getColumnNum();
-      if (testLine < line || testLine == line && testColumn < column) {
-        line = testLine;
-        column = testColumn;
-      }
-
-      testLine = pos.getEndLineNum();
-      testColumn = pos.getEndColumnNum();
-      if (testLine > endLine || testLine == endLine && testColumn > endColumn) {
-        endLine = testLine;
-        endColumn = testColumn;
-      }
-    }
-    return new SqlParserPos(line, column, endLine, endColumn);
-  }
-
-  /**
    * Combines an array of parser positions to create a position which spans
    * from the beginning of the first to the end of the last.
    */
-  public static SqlParserPos sum(
-      SqlParserPos[] poses) {
+  public static SqlParserPos sum(Iterable<SqlParserPos> poses) {
     return sum(poses, Integer.MAX_VALUE, Integer.MAX_VALUE, -1, -1);
   }
 
@@ -247,7 +213,7 @@ public class SqlParserPos implements Serializable {
    * @return Sum of parser positions
    */
   private static SqlParserPos sum(
-      SqlParserPos[] poses,
+      Iterable<SqlParserPos> poses,
       int line,
       int column,
       int endLine,
@@ -255,7 +221,7 @@ public class SqlParserPos implements Serializable {
     int testLine;
     int testColumn;
     for (SqlParserPos pos : poses) {
-      if (pos == null) {
+      if (pos == null || pos.equals(SqlParserPos.ZERO)) {
         continue;
       }
       testLine = pos.getLineNum();
@@ -274,6 +240,28 @@ public class SqlParserPos implements Serializable {
     }
     return new SqlParserPos(line, column, endLine, endColumn);
   }
+
+  public boolean overlaps(SqlParserPos pos) {
+    return startsBefore(pos) && endsAfter(pos)
+        || pos.startsBefore(this) && pos.endsAfter(this);
+  }
+
+  private boolean startsBefore(SqlParserPos pos) {
+    return lineNumber < pos.lineNumber
+        || lineNumber == pos.lineNumber
+        && columnNumber <= pos.columnNumber;
+  }
+
+  private boolean endsAfter(SqlParserPos pos) {
+    return endLineNumber > pos.endLineNumber
+        || endLineNumber == pos.endLineNumber
+        && endColumnNumber >= pos.endColumnNumber;
+  }
+
+  public boolean startsAt(SqlParserPos pos) {
+    return lineNumber == pos.lineNumber
+        && columnNumber == pos.columnNumber;
+  }
 }
 
 // End SqlParserPos.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
index 1c32594..69bf51c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AggChecker.java
@@ -99,8 +99,8 @@ class AggChecker extends SqlBasicVisitor<Void> {
     // it fully-qualified.
     // TODO: It would be better if we always compared fully-qualified
     // to fully-qualified.
-    final SqlIdentifier fqId = Stacks.peek(scopes).fullyQualify(id);
-    if (isGroupExpr(fqId)) {
+    final SqlQualified fqId = Stacks.peek(scopes).fullyQualify(id);
+    if (isGroupExpr(fqId.identifier)) {
       return null;
     }
     SqlNode originalExpr = validator.getOriginal(id);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/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 4584d6f..58fda81 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
@@ -88,8 +88,8 @@ public class AggregatingSelectScope
       // We deep-copy the group-list in case subsequent validation
       // modifies it and makes it no longer equivalent. While copying,
       // we fully qualify all identifiers.
-      final SqlNodeList groupList =
-          SqlValidatorUtil.DeepCopier.copy(parent, select.getGroup());
+      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/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/CatalogScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/CatalogScope.java b/core/src/main/java/org/apache/calcite/sql/validate/CatalogScope.java
new file mode 100644
index 0000000..c2ef43e
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/CatalogScope.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.linq4j.Linq4j;
+import org.apache.calcite.linq4j.function.Function1;
+import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.sql.SqlNode;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Sets;
+
+import java.util.List;
+import java.util.Set;
+
+/**
+ * Implementation of {@link SqlValidatorScope} that can see all schemas in the
+ * current catalog.
+ *
+ * <p>Occurs near the root of the scope stack; its parent is typically
+ * {@link EmptyScope}.
+ *
+ * <p>Helps resolve {@code schema.table.column} column references, such as
+ *
+ * <blockquote><pre>select sales.emp.empno from sales.emp</pre></blockquote>
+ */
+class CatalogScope extends DelegatingScope {
+  /** Fully-qualified name of the catalog. Typically empty or ["CATALOG"]. */
+  final ImmutableList<String> names;
+  private final Set<List<String>> schemaNames;
+
+  //~ Constructors -----------------------------------------------------------
+
+  CatalogScope(SqlValidatorScope parent, List<String> names) {
+    super(parent);
+    this.names = ImmutableList.copyOf(names);
+    this.schemaNames =
+        Linq4j.asEnumerable(
+            validator.getCatalogReader()
+                .getAllSchemaObjectNames(ImmutableList.<String>of()))
+            .where(
+                new Predicate1<SqlMoniker>() {
+                  public boolean apply(SqlMoniker input) {
+                    return input.getType() == SqlMonikerType.SCHEMA;
+                  }
+                })
+            .select(
+                new Function1<SqlMoniker, List<String>>() {
+                  public List<String> apply(SqlMoniker input) {
+                    return input.getFullyQualifiedNames();
+                  }
+                })
+            .into(Sets.<List<String>>newHashSet());
+  }
+
+  //~ Methods ----------------------------------------------------------------
+
+  public SqlNode getNode() {
+    throw new UnsupportedOperationException();
+  }
+
+  public SqlValidatorNamespace resolve(List<String> names,
+      SqlValidatorScope[] ancestorOut, int[] offsetOut) {
+    final ImmutableList<String> nameList =
+        ImmutableList.<String>builder().addAll(this.names).addAll(names)
+            .build();
+    if (schemaNames.contains(nameList)) {
+      return new SchemaNamespace(validator, nameList);
+    }
+    return null;
+  }
+}
+
+// End CatalogScope.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/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 e23d886..72c533e 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
@@ -25,9 +25,11 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlSelect;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.parser.SqlParserPos;
+import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableList;
 
+import java.util.Collection;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -72,10 +74,10 @@ public abstract class DelegatingScope implements SqlValidatorScope {
   }
 
   public SqlValidatorNamespace resolve(
-      String name,
+      List<String> names,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
-    return parent.resolve(name, ancestorOut, offsetOut);
+    return parent.resolve(names, ancestorOut, offsetOut);
   }
 
   protected void addColumnNames(
@@ -101,11 +103,12 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     parent.findAllColumnNames(result);
   }
 
-  public void findAliases(List<SqlMoniker> result) {
+  public void findAliases(Collection<SqlMoniker> result) {
     parent.findAliases(result);
   }
 
-  public String findQualifyingTableName(String columnName, SqlNode ctx) {
+  public Pair<String, SqlValidatorNamespace>
+  findQualifyingTableName(String columnName, SqlNode ctx) {
     return parent.findQualifyingTableName(columnName, ctx);
   }
 
@@ -139,19 +142,19 @@ public abstract class DelegatingScope implements SqlValidatorScope {
    *
    * <p>If the identifier cannot be resolved, throws. Never returns null.
    */
-  public SqlIdentifier fullyQualify(SqlIdentifier identifier) {
+  public SqlQualified fullyQualify(SqlIdentifier identifier) {
     if (identifier.isStar()) {
-      return identifier;
+      return SqlQualified.create(this, 1, null, identifier);
     }
 
-    String tableName;
     String columnName;
-
     switch (identifier.names.size()) {
     case 1:
       columnName = identifier.names.get(0);
-      tableName =
+      final Pair<String, SqlValidatorNamespace> pair =
           findQualifyingTableName(columnName, identifier);
+      final String tableName = pair.left;
+      final SqlValidatorNamespace namespace = pair.right;
 
       // todo: do implicit collation here
       final SqlParserPos pos = identifier.getParserPosition();
@@ -162,32 +165,39 @@ public abstract class DelegatingScope implements SqlValidatorScope {
               pos,
               ImmutableList.of(SqlParserPos.ZERO, pos));
       validator.setOriginal(expanded, identifier);
-      return expanded;
-
-    case 2:
-      tableName = identifier.names.get(0);
-      final SqlValidatorNamespace fromNs = resolve(tableName, null, null);
-      if (fromNs == null) {
-        throw validator.newValidationError(identifier.getComponent(0),
-            RESOURCE.tableNameNotFound(tableName));
-      }
-      columnName = identifier.names.get(1);
-      final RelDataType fromRowType = fromNs.getRowType();
-      final RelDataTypeField field =
-          validator.catalogReader.field(fromRowType, columnName);
-      if (field != null) {
-        identifier.setName(1, field.getName()); // normalize case to match defn
-        return identifier; // it was fine already
-      } else {
-        throw validator.newValidationError(identifier.getComponent(1),
-            RESOURCE.columnNotFoundInTable(columnName, tableName));
-      }
+      return SqlQualified.create(this, 1, namespace, expanded);
 
     default:
-      // NOTE jvs 26-May-2004:  lengths greater than 2 are possible
-      // for row and structured types
-      assert identifier.names.size() > 0;
-      return identifier;
+      SqlValidatorNamespace fromNs = null;
+      final int size = identifier.names.size();
+      int i = size - 1;
+      for (; i > 0; i--) {
+        final SqlIdentifier prefix = identifier.getComponent(0, i);
+        fromNs = resolve(prefix.names, null, null);
+        if (fromNs != null) {
+          break;
+        }
+      }
+      if (fromNs == null || fromNs instanceof SchemaNamespace) {
+        final SqlIdentifier prefix1 = identifier.skipLast(1);
+        throw validator.newValidationError(prefix1,
+            RESOURCE.tableNameNotFound(prefix1.toString()));
+      }
+      RelDataType fromRowType = fromNs.getRowType();
+      for (int j = i; j < size; j++) {
+        final SqlIdentifier last = identifier.getComponent(j);
+        columnName = last.getSimple();
+        final RelDataTypeField field =
+            validator.catalogReader.field(fromRowType, columnName);
+        if (field == null) {
+          throw validator.newValidationError(last,
+              RESOURCE.columnNotFoundInTable(columnName,
+                  identifier.getComponent(0, j).toString()));
+        }
+        identifier.setName(j, field.getName()); // normalize case to match defn
+        fromRowType = field.getType();
+      }
+      return SqlQualified.create(this, i, fromNs, identifier);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java b/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
index 6376bd7..9da3bff 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/EmptyScope.java
@@ -25,7 +25,9 @@ import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlWindow;
+import org.apache.calcite.util.Pair;
 
+import java.util.Collection;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -54,8 +56,8 @@ class EmptyScope implements SqlValidatorScope {
     return validator;
   }
 
-  public SqlIdentifier fullyQualify(SqlIdentifier identifier) {
-    return identifier;
+  public SqlQualified fullyQualify(SqlIdentifier identifier) {
+    return SqlQualified.create(this, 1, null, identifier);
   }
 
   public SqlNode getNode() {
@@ -63,7 +65,7 @@ class EmptyScope implements SqlValidatorScope {
   }
 
   public SqlValidatorNamespace resolve(
-      String name,
+      List<String> names,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
     return null;
@@ -86,7 +88,7 @@ class EmptyScope implements SqlValidatorScope {
   public void findAllTableNames(List<SqlMoniker> result) {
   }
 
-  public void findAliases(List<SqlMoniker> result) {
+  public void findAliases(Collection<SqlMoniker> result) {
   }
 
   public RelDataType resolveColumn(String name, SqlNode ctx) {
@@ -101,7 +103,8 @@ class EmptyScope implements SqlValidatorScope {
     // valid
   }
 
-  public String findQualifyingTableName(String columnName, SqlNode ctx) {
+  public Pair<String, SqlValidatorNamespace>
+  findQualifyingTableName(String columnName, SqlNode ctx) {
     throw validator.newValidationError(ctx,
         RESOURCE.columnNotFound(columnName));
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
index 80af917..8567cc5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ListScope.java
@@ -23,6 +23,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -78,6 +79,47 @@ public abstract class ListScope extends DelegatingScope {
     }
   }
 
+  /** Returns a child namespace that matches a fully-qualified list of names.
+   * This will be a schema-qualified table, for example
+   *
+   * <blockquote><pre>SELECT sales.emp.empno FROM sales.emp</pre></blockquote>
+   */
+  protected SqlValidatorNamespace getChild(List<String> names) {
+    int i = findChild(names);
+    return i < 0 ? null : children.get(i).right;
+  }
+
+  protected int findChild(List<String> names) {
+    for (int i = 0; i < children.size(); i++) {
+      Pair<String, SqlValidatorNamespace> child = children.get(i);
+      if (child.left != null) {
+        if (validator.catalogReader.matches(child.left, Util.last(names))) {
+          if (names.size() == 1) {
+            return i;
+          }
+        } else {
+          // Alias does not match last segment. Don't consider the
+          // fully-qualified name. E.g.
+          //    SELECT sales.emp.name FROM sales.emp AS otherAlias
+          continue;
+        }
+      }
+
+      // Look up the 2 tables independently, in case one is qualified with
+      // catalog & schema and the other is not.
+      final SqlValidatorTable table = child.right.getTable();
+      if (table != null) {
+        final SqlValidatorTable table2 =
+            validator.catalogReader.getTable(names);
+        if (table2 != null
+            && table.getQualifiedName().equals(table2.getQualifiedName())) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+
   public void findAllColumnNames(List<SqlMoniker> result) {
     for (Pair<String, SqlValidatorNamespace> pair : children) {
       addColumnNames(pair.right, result);
@@ -85,22 +127,21 @@ public abstract class ListScope extends DelegatingScope {
     parent.findAllColumnNames(result);
   }
 
-  public void findAliases(List<SqlMoniker> result) {
+  public void findAliases(Collection<SqlMoniker> result) {
     for (Pair<String, SqlValidatorNamespace> pair : children) {
       result.add(new SqlMonikerImpl(pair.left, SqlMonikerType.TABLE));
     }
     parent.findAliases(result);
   }
 
-  public String findQualifyingTableName(
-      final String columnName,
-      SqlNode ctx) {
+  public Pair<String, SqlValidatorNamespace>
+  findQualifyingTableName(final String columnName, SqlNode ctx) {
     int count = 0;
-    String tableName = null;
+    Pair<String, SqlValidatorNamespace> tableName = null;
     for (Pair<String, SqlValidatorNamespace> child : children) {
       final RelDataType rowType = child.right.getRowType();
       if (validator.catalogReader.field(rowType, columnName) != null) {
-        tableName = child.left;
+        tableName = child;
         count++;
       }
     }
@@ -116,11 +157,11 @@ public abstract class ListScope extends DelegatingScope {
   }
 
   public SqlValidatorNamespace resolve(
-      String name,
+      List<String> names,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
     // First resolve by looking through the child namespaces.
-    final int i = validator.catalogReader.match(Pair.left(children), name);
+    final int i = findChild(names);
     if (i >= 0) {
       if (ancestorOut != null) {
         ancestorOut[0] = this;
@@ -133,7 +174,7 @@ public abstract class ListScope extends DelegatingScope {
 
     // Then call the base class method, which will delegate to the
     // parent scope.
-    return parent.resolve(name, ancestorOut, offsetOut);
+    return parent.resolve(names, ancestorOut, offsetOut);
   }
 
   public RelDataType resolveColumn(String columnName, SqlNode ctx) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
index ea38156..55b3af9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/OrderByScope.java
@@ -65,7 +65,7 @@ public class OrderByScope extends DelegatingScope {
     addColumnNames(ns, result);
   }
 
-  public SqlIdentifier fullyQualify(SqlIdentifier identifier) {
+  public SqlQualified fullyQualify(SqlIdentifier identifier) {
     // If it's a simple identifier, look for an alias.
     if (identifier.isSimple()
         && validator.getConformance().isSortByAlias()) {
@@ -74,7 +74,7 @@ public class OrderByScope extends DelegatingScope {
           validator.getNamespace(select);
       final RelDataType rowType = selectNs.getRowType();
       if (validator.catalogReader.field(rowType, name) != null) {
-        return identifier;
+        return SqlQualified.create(this, 1, selectNs, identifier);
       }
     }
     return super.fullyQualify(identifier);

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java b/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
index 6a5d86c..f1a4e91 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/ParameterScope.java
@@ -20,6 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlIdentifier;
 
+import java.util.List;
 import java.util.Map;
 
 /**
@@ -48,8 +49,8 @@ public class ParameterScope extends EmptyScope {
 
   //~ Methods ----------------------------------------------------------------
 
-  public SqlIdentifier fullyQualify(SqlIdentifier identifier) {
-    return identifier;
+  public SqlQualified fullyQualify(SqlIdentifier identifier) {
+    return SqlQualified.create(this, 1, null, identifier);
   }
 
   public SqlValidatorScope getOperandScope(SqlCall call) {
@@ -57,10 +58,11 @@ public class ParameterScope extends EmptyScope {
   }
 
   public SqlValidatorNamespace resolve(
-      String name,
+      List<String> names,
       SqlValidatorScope[] ancestorOut,
       int[] offsetOut) {
-    final RelDataType type = nameToTypeMap.get(name);
+    assert names.size() == 1;
+    final RelDataType type = nameToTypeMap.get(names.get(0));
     return new ParameterNamespace(validator, type);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SchemaNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SchemaNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SchemaNamespace.java
new file mode 100644
index 0000000..e8347c4
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SchemaNamespace.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.util.Util;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/** Namespace based on a schema.
+ *
+ * <p>The visible names are tables and sub-schemas.
+ */
+class SchemaNamespace extends AbstractNamespace {
+  /** The path of this schema. */
+  private final ImmutableList<String> names;
+
+  /** Creates a SchemaNamespace. */
+  SchemaNamespace(SqlValidatorImpl validator, ImmutableList<String> names) {
+    super(validator, null);
+    this.names = Preconditions.checkNotNull(names);
+  }
+
+  protected RelDataType validateImpl() {
+    final RelDataTypeFactory.FieldInfoBuilder builder =
+        validator.getTypeFactory().builder();
+    for (SqlMoniker moniker
+        : validator.catalogReader.getAllSchemaObjectNames(names)) {
+      final List<String> names1 = moniker.getFullyQualifiedNames();
+      final SqlValidatorTable table = validator.catalogReader.getTable(names1);
+      builder.add(Util.last(names1), table.getRowType());
+    }
+    return builder.build();
+  }
+
+  public SqlNode getNode() {
+    return null;
+  }
+}
+
+// End SchemaNamespace.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
index 76ddeb1..ea6db19 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SelectScope.java
@@ -209,7 +209,7 @@ public class SelectScope extends ListScope {
         final SelectScope parentScope = (SelectScope) walker;
         return parentScope.existingWindowName(winName);
       }
-      walker = parent;
+      walker = ((DelegatingScope) walker).parent;
     }
 
     return false;

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
index 72f8ffd..66c47f5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMoniker.java
@@ -18,12 +18,30 @@ package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.sql.SqlIdentifier;
 
+import com.google.common.collect.Ordering;
+
+import java.util.Comparator;
 import java.util.List;
 
 /**
  * An interface of an object identifier that represents a SqlIdentifier
  */
 public interface SqlMoniker {
+  Comparator<SqlMoniker> COMPARATOR =
+      new Comparator<SqlMoniker>() {
+        final Ordering<Iterable<String>> listOrdering =
+            Ordering.<String>natural().lexicographical();
+
+        public int compare(SqlMoniker o1, SqlMoniker o2) {
+          int c = o1.getType().compareTo(o2.getType());
+          if (c == 0) {
+            c = listOrdering.compare(o1.getFullyQualifiedNames(),
+                o2.getFullyQualifiedNames());
+          }
+          return c;
+        }
+      };
+
   //~ Methods ----------------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
deleted file mode 100644
index 85eb1c0..0000000
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerComparator.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to you under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.calcite.sql.validate;
-
-import java.util.Comparator;
-
-/**
- * A general-purpose implementation of {@link Comparator} to compare
- * {@link SqlMoniker} values.
- */
-public class SqlMonikerComparator implements Comparator<SqlMoniker> {
-  //~ Methods ----------------------------------------------------------------
-
-  public int compare(SqlMoniker m1, SqlMoniker m2) {
-    if (m1.getType().ordinal() > m2.getType().ordinal()) {
-      return 1;
-    } else if (m1.getType().ordinal() < m2.getType().ordinal()) {
-      return -1;
-    } else {
-      return m1.toString().compareTo(m2.toString());
-    }
-  }
-}
-
-// End SqlMonikerComparator.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
index c7eebd8..d14f76c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerImpl.java
@@ -20,6 +20,7 @@ import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -39,13 +40,8 @@ public class SqlMonikerImpl implements SqlMoniker {
    * Creates a moniker with an array of names.
    */
   public SqlMonikerImpl(List<String> names, SqlMonikerType type) {
-    assert names != null;
-    assert type != null;
-    for (String name : names) {
-      assert name != null;
-    }
     this.names = ImmutableList.copyOf(names);
-    this.type = type;
+    this.type = Preconditions.checkNotNull(type);
   }
 
   /**
@@ -57,6 +53,17 @@ public class SqlMonikerImpl implements SqlMoniker {
 
   //~ Methods ----------------------------------------------------------------
 
+  @Override public boolean equals(Object obj) {
+    return this == obj
+        || obj instanceof SqlMonikerImpl
+        && type == ((SqlMonikerImpl) obj).type
+        && names.equals(((SqlMonikerImpl) obj).names);
+  }
+
+  @Override public int hashCode() {
+    return Util.hash(type.ordinal(), names);
+  }
+
   public SqlMonikerType getType() {
     return type;
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
index 4b64859..7bf7a3e 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlMonikerType.java
@@ -22,7 +22,7 @@ package org.apache.calcite.sql.validate;
  * <p>Used in {@link SqlMoniker}.
  */
 public enum SqlMonikerType {
-  COLUMN, TABLE, VIEW, SCHEMA, REPOSITORY, FUNCTION, KEYWORD
+  COLUMN, TABLE, VIEW, SCHEMA, CATALOG, REPOSITORY, FUNCTION, KEYWORD
 }
 
 // End SqlMonikerType.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
new file mode 100644
index 0000000..4b99845
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.validate;
+
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+
+/**
+ * Fully-qualified identifier.
+ *
+ * <p>The result of calling
+ * {@link org.apache.calcite.sql.validate.SqlValidatorScope#fullyQualify(org.apache.calcite.sql.SqlIdentifier)},
+ * a fully-qualified identifier contains the name (in correct case),
+ * parser position, type, and scope of each component of the identifier.
+ *
+ * <p>It is immutable.
+ */
+public class SqlQualified {
+  private final SqlValidatorScope scope;
+  public final int prefixLength;
+  public final SqlValidatorNamespace namespace;
+  public final SqlIdentifier identifier;
+
+  private SqlQualified(SqlValidatorScope scope, int prefixLength,
+      SqlValidatorNamespace namespace, SqlIdentifier identifier) {
+    this.scope = scope;
+    this.prefixLength = prefixLength;
+    this.namespace = namespace;
+    this.identifier = identifier;
+  }
+
+  public static SqlQualified create(SqlValidatorScope scope, int prefixLength,
+      SqlValidatorNamespace namespace, SqlIdentifier identifier) {
+    return new SqlQualified(scope, prefixLength, namespace, identifier);
+  }
+
+  public List<String> translatedNames() {
+    if (scope == null) {
+      return identifier.names;
+    }
+    final ImmutableList.Builder<String> builder = ImmutableList.builder();
+    SqlValidatorNamespace namespace =
+        scope.resolve(Util.skipLast(identifier.names), null, null);
+    builder.add(identifier.names.get(0));
+    for (String name : Util.skip(identifier.names)) {
+      if (namespace != null) {
+        name = namespace.translate(name);
+        namespace = null;
+      }
+      builder.add(name);
+    }
+    return builder.build();
+  }
+
+  public final List<String> prefix() {
+    return identifier.names.subList(0, prefixLength);
+  }
+
+  public final List<String> suffix() {
+    return Util.skip(identifier.names, prefixLength);
+  }
+
+  public final List<String> suffixTranslated() {
+    return Util.skip(translatedNames(), prefixLength);
+  }
+}
+
+// End SqlQualified.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
index 915d9ec..4a1c16b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorCatalogReader.java
@@ -86,6 +86,8 @@ public interface SqlValidatorCatalogReader {
    */
   int fieldOrdinal(RelDataType rowType, String alias);
 
+  boolean matches(String string, String name);
+
   int match(List<String> strings, String name);
 
   RelDataType createTypeFromProjection(RelDataType type,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/6c88546e/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 cabc4d8..6e82f4c 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
@@ -74,7 +74,6 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.util.BitString;
-import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -92,6 +91,7 @@ import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.GregorianCalendar;
 import java.util.HashMap;
@@ -398,66 +398,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       List<Map.Entry<String, RelDataType>> types,
       final boolean includeSystemVars) {
     final SelectScope scope = (SelectScope) getWhereScope(select);
-    if (selectItem instanceof SqlIdentifier) {
-      SqlIdentifier identifier = (SqlIdentifier) selectItem;
-      if ((identifier.names.size() == 1)
-          && identifier.names.get(0).equals("*")) {
-        SqlParserPos starPosition = identifier.getParserPosition();
-        for (Pair<String, SqlValidatorNamespace> p : scope.children) {
-          final SqlNode from = p.right.getNode();
-          final SqlValidatorNamespace fromNs = getNamespace(from, scope);
-          assert fromNs != null;
-          final RelDataType rowType = fromNs.getRowType();
-          for (RelDataTypeField field : rowType.getFieldList()) {
-            String columnName = field.getName();
-
-            // TODO: do real implicit collation here
-            final SqlNode exp =
-                new SqlIdentifier(
-                    ImmutableList.of(p.left, columnName),
-                    starPosition);
-            addToSelectList(
-                selectItems,
-                aliases,
-                types,
-                exp,
-                scope,
-                includeSystemVars);
-          }
-        }
-        return true;
-      } else if (identifier.names.size() == 2
-          && identifier.names.get(1).equals("*")) {
-        final String tableName = identifier.names.get(0);
-        SqlParserPos starPosition = identifier.getParserPosition();
-        final SqlValidatorNamespace childNs = scope.getChild(tableName);
-        if (childNs == null) {
-          // e.g. "select r.* from e"
-          throw newValidationError(identifier.getComponent(0),
-              RESOURCE.unknownIdentifier(tableName));
-        }
-        final SqlNode from = childNs.getNode();
-        final SqlValidatorNamespace fromNs = getNamespace(from);
-        assert fromNs != null;
-        final RelDataType rowType = fromNs.getRowType();
-        for (RelDataTypeField field : rowType.getFieldList()) {
-          String columnName = field.getName();
-
-          // TODO: do real implicit collation here
-          final SqlIdentifier exp =
-              new SqlIdentifier(
-                  ImmutableList.of(tableName, columnName),
-                  starPosition);
-          addToSelectList(
-              selectItems,
-              aliases,
-              types,
-              exp,
-              scope,
-              includeSystemVars);
-        }
-        return true;
-      }
+    if (expandStar(selectItems, aliases, types, includeSystemVars, scope,
+        selectItem)) {
+      return true;
     }
 
     // Expand the select item: fully-qualify columns, and convert
@@ -495,8 +438,84 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     return false;
   }
 
+  private boolean expandStar(List<SqlNode> selectItems, Set<String> aliases,
+      List<Map.Entry<String, RelDataType>> types, boolean includeSystemVars,
+      SelectScope scope, SqlNode node) {
+    if (!(node instanceof SqlIdentifier)) {
+      return false;
+    }
+    final SqlIdentifier identifier = (SqlIdentifier) node;
+    if (!identifier.isStar()) {
+      return false;
+    }
+    final SqlParserPos starPosition = identifier.getParserPosition();
+    switch (identifier.names.size()) {
+    case 1:
+      for (Pair<String, SqlValidatorNamespace> p : scope.children) {
+        final SqlNode from = p.right.getNode();
+        final SqlValidatorNamespace fromNs = getNamespace(from, scope);
+        assert fromNs != null;
+        final RelDataType rowType = fromNs.getRowType();
+        for (RelDataTypeField field : rowType.getFieldList()) {
+          String columnName = field.getName();
+
+          // TODO: do real implicit collation here
+          final SqlNode exp =
+              new SqlIdentifier(
+                  ImmutableList.of(p.left, columnName),
+                  starPosition);
+          addToSelectList(
+              selectItems,
+              aliases,
+              types,
+              exp,
+              scope,
+              includeSystemVars);
+        }
+      }
+      return true;
+    default:
+      final SqlIdentifier prefixId = identifier.skipLast(1);
+      final SqlValidatorNamespace fromNs;
+      if (prefixId.names.size() == 1) {
+        String tableName = prefixId.names.get(0);
+        final SqlValidatorNamespace childNs = scope.getChild(tableName);
+        if (childNs == null) {
+          // e.g. "select r.* from e"
+          throw newValidationError(identifier.getComponent(0),
+              RESOURCE.unknownIdentifier(tableName));
+        }
+        final SqlNode from = childNs.getNode();
+        fromNs = getNamespace(from);
+      } else {
+        fromNs = scope.getChild(prefixId.names);
+        if (fromNs == null) {
+          // e.g. "select s.t.* from e"
+          throw newValidationError(prefixId,
+              RESOURCE.unknownIdentifier(prefixId.toString()));
+        }
+      }
+      assert fromNs != null;
+      final RelDataType rowType = fromNs.getRowType();
+      for (RelDataTypeField field : rowType.getFieldList()) {
+        String columnName = field.getName();
+
+        // TODO: do real implicit collation here
+        addToSelectList(
+            selectItems,
+            aliases,
+            types,
+            prefixId.plus(columnName, starPosition),
+            scope,
+            includeSystemVars);
+      }
+      return true;
+    }
+  }
+
   public SqlNode validate(SqlNode topNode) {
     SqlValidatorScope scope = new EmptyScope(this);
+    scope = new CatalogScope(scope, ImmutableList.of("CATALOG"));
     final SqlNode topNode2 = validateScopedExpression(topNode, scope);
     final RelDataType type = getValidatedNodeType(topNode2);
     Util.discard(type);
@@ -520,16 +539,17 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     if (ns == null) {
       throw Util.newInternal("Not a query: " + outermostNode);
     }
-    List<SqlMoniker> hintList = new ArrayList<SqlMoniker>();
+    Collection<SqlMoniker> hintList = Sets.newTreeSet(SqlMoniker.COMPARATOR);
     lookupSelectHints(ns, pos, hintList);
-    return hintList;
+    return ImmutableList.copyOf(hintList);
   }
 
   public SqlMoniker lookupQualifiedName(SqlNode topNode, SqlParserPos pos) {
     final String posString = pos.toString();
     IdInfo info = idPositions.get(posString);
     if (info != null) {
-      return new SqlIdentifierMoniker(info.scope.fullyQualify(info.id));
+      final SqlQualified qualified = info.scope.fullyQualify(info.id);
+      return new SqlIdentifierMoniker(qualified.identifier);
     } else {
       return null;
     }
@@ -548,7 +568,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   void lookupSelectHints(
       SqlSelect select,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     IdInfo info = idPositions.get(pos.toString());
     if ((info == null) || (info.scope == null)) {
       SqlNode fromNode = select.getFrom();
@@ -563,7 +583,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   private void lookupSelectHints(
       SqlValidatorNamespace ns,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     final SqlNode node = ns.getNode();
     if (node instanceof SqlSelect) {
       lookupSelectHints((SqlSelect) node, pos, hintList);
@@ -574,7 +594,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNode node,
       SqlValidatorScope scope,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     final SqlValidatorNamespace ns = getNamespace(node);
     if (ns.isWrapperFor(IdentifierNamespace.class)) {
       IdentifierNamespace idNs = ns.unwrap(IdentifierNamespace.class);
@@ -610,7 +630,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlJoin join,
       SqlValidatorScope scope,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     SqlNode left = join.getLeft();
     SqlNode right = join.getRight();
     SqlNode condition = join.getCondition();
@@ -648,7 +668,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlValidatorScope scope,
       List<String> names,
       SqlParserPos pos,
-      List<SqlMoniker> hintList) {
+      Collection<SqlMoniker> hintList) {
     // Remove the last part of name - it is a dummy
     List<String> subNames = Util.skipLast(names);
 
@@ -657,7 +677,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlValidatorNamespace ns = null;
       for (String name : subNames) {
         if (ns == null) {
-          ns = scope.resolve(name, null, null);
+          ns = scope.resolve(ImmutableList.of(name), null, null);
         } else {
           ns = ns.lookupChild(name);
         }
@@ -705,7 +725,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   private static void findAllValidUdfNames(
       List<String> names,
       SqlValidator validator,
-      List<SqlMoniker> result) {
+      Collection<SqlMoniker> result) {
     List<SqlMoniker> objNames = new ArrayList<SqlMoniker>();
     SqlValidatorUtil.getSchemaObjectMonikers(
         validator.getCatalogReader(),
@@ -721,7 +741,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   private static void findAllValidFunctionNames(
       List<String> names,
       SqlValidator validator,
-      List<SqlMoniker> result,
+      Collection<SqlMoniker> result,
       SqlParserPos pos) {
     // a function name can only be 1 part
     if (names.size() > 1) {
@@ -878,7 +898,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           ((DelegatingScope) scope).getParent();
       if (id.isSimple()) {
         SqlValidatorNamespace ns =
-            parentScope.resolve(id.getSimple(), null, null);
+            parentScope.resolve(id.names, null, null);
         if (ns != null) {
           return ns;
         }
@@ -2523,11 +2543,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   public void validateIdentifier(SqlIdentifier id, SqlValidatorScope scope) {
-    final SqlIdentifier fqId = scope.fullyQualify(id);
+    final SqlQualified fqId = scope.fullyQualify(id);
     if (expandColumnReferences) {
       // NOTE jvs 9-Apr-2007: this doesn't cover ORDER BY, which has its
       // own ideas about qualification.
-      id.assignNamesFrom(fqId);
+      id.assignNamesFrom(fqId.identifier);
     } else {
       Util.discard(fqId);
     }
@@ -3157,10 +3177,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     // Validate SELECT list. Expand terms of the form "*" or "TABLE.*".
     final SqlValidatorScope selectScope = getSelectScope(select);
-    final List<SqlNode> expandedSelectItems = new ArrayList<SqlNode>();
-    final Set<String> aliases = new HashSet<String>();
-    final List<Map.Entry<String, RelDataType>> fieldList =
-        new ArrayList<Map.Entry<String, RelDataType>>();
+    final List<SqlNode> expandedSelectItems = Lists.newArrayList();
+    final Set<String> aliases = Sets.newHashSet();
+    final List<Map.Entry<String, RelDataType>> fieldList = Lists.newArrayList();
 
     for (int i = 0; i < selectItems.size(); i++) {
       SqlNode selectItem = selectItems.get(i);
@@ -3848,26 +3867,21 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final List<SqlNode> selectList = scope.getExpandedSelectList();
       final SqlNode selectItem = stripAs(selectList.get(i));
       if (selectItem instanceof SqlIdentifier) {
-        SqlIdentifier id = (SqlIdentifier) selectItem;
-        SqlValidatorNamespace namespace = null;
-        List<String> origin = new ArrayList<String>();
-        for (String name : id.names) {
+        final SqlQualified qualified =
+            scope.fullyQualify((SqlIdentifier) selectItem);
+        SqlValidatorNamespace namespace = qualified.namespace;
+        final SqlValidatorTable table = namespace.getTable();
+        if (table == null) {
+          return null;
+        }
+        final List<String> origin =
+            Lists.newArrayList(table.getQualifiedName());
+        for (String name : qualified.suffix()) {
+          namespace = namespace.lookupChild(name);
           if (namespace == null) {
-            namespace = scope.resolve(name, null, null);
-            final SqlValidatorTable table = namespace.getTable();
-            if (table != null) {
-              origin.addAll(table.getQualifiedName());
-            } else {
-              return null;
-            }
-          } else {
-            namespace = namespace.lookupChild(name);
-            if (namespace != null) {
-              origin.add(name);
-            } else {
-              return null;
-            }
+            return null;
           }
+          origin.add(name);
         }
         return origin;
       }
@@ -4030,56 +4044,56 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
       RelDataType type = null;
       if (!(scope instanceof EmptyScope)) {
-        id = scope.fullyQualify(id);
+        id = scope.fullyQualify(id).identifier;
       }
-      for (Ord<String> ord : Ord.zip(id.names)) {
-        String name = ord.e;
-        if (ord.i == 0) {
-          // REVIEW jvs 9-June-2005: The name resolution rules used
-          // here are supposed to match SQL:2003 Part 2 Section 6.6
-          // (identifier chain), but we don't currently have enough
-          // information to get everything right.  In particular,
-          // routine parameters are currently looked up via resolve;
-          // we could do a better job if they were looked up via
-          // resolveColumn.
-
-          // TODO jvs 9-June-2005:  Support schema-qualified table
-          // names here (FRG-140).  This was illegal in SQL-92, but
-          // became legal in SQL:1999.  (SQL:2003 Part 2 Section
-          // 6.6 Syntax Rule 8.b.vi)
-          Util.discard(Bug.FRG140_FIXED);
-
-          SqlValidatorNamespace resolvedNs =
-              scope.resolve(name, null, null);
-
-          if (resolvedNs != null) {
-            // There's a namespace with the name we seek.
-            type = resolvedNs.getRowType();
-          }
 
-          // Give precedence to namespace found, unless there
-          // are no more identifier components.
-          if (type == null || id.names.size() == 1) {
-            // See if there's a column with the name we seek in
-            // precisely one of the namespaces in this scope.
-            RelDataType colType = scope.resolveColumn(name, id);
-            if (colType != null) {
-              type = colType;
-            }
-          }
+      // Resolve the longest prefix of id that we can
+      SqlValidatorNamespace resolvedNs;
+      int i;
+      for (i = id.names.size() - 1; i > 0; i--) {
+        // REVIEW jvs 9-June-2005: The name resolution rules used
+        // here are supposed to match SQL:2003 Part 2 Section 6.6
+        // (identifier chain), but we don't currently have enough
+        // information to get everything right.  In particular,
+        // routine parameters are currently looked up via resolve;
+        // we could do a better job if they were looked up via
+        // resolveColumn.
+
+        resolvedNs = scope.resolve(id.names.subList(0, i), null, null);
+        if (resolvedNs != null) {
+          // There's a namespace with the name we seek.
+          type = resolvedNs.getRowType();
+          break;
+        }
+      }
 
-          if (type == null) {
-            throw newValidationError(id.getComponent(ord.i),
-                RESOURCE.unknownIdentifier(name));
-          }
-        } else {
-          final RelDataTypeField field = catalogReader.field(type, name);
-          if (field == null) {
-            throw newValidationError(id.getComponent(ord.i),
-                RESOURCE.unknownField(name));
-          }
-          type = field.getType();
+      // Give precedence to namespace found, unless there
+      // are no more identifier components.
+      if (type == null || id.names.size() == 1) {
+        // See if there's a column with the name we seek in
+        // precisely one of the namespaces in this scope.
+        RelDataType colType = scope.resolveColumn(id.names.get(0), id);
+        if (colType != null) {
+          type = colType;
+        }
+        ++i;
+      }
+
+      if (type == null) {
+        final SqlIdentifier last = id.getComponent(i - 1, i);
+        throw newValidationError(last,
+            RESOURCE.unknownIdentifier(last.toString()));
+      }
+
+      // Resolve rest of identifier
+      for (; i < id.names.size(); i++) {
+        String name = id.names.get(i);
+        final RelDataTypeField field = catalogReader.field(type, name);
+        if (field == null) {
+          throw newValidationError(id.getComponent(i),
+              RESOURCE.unknownField(name));
         }
+        type = field.getType();
       }
       type =
           SqlTypeUtil.addCharsetAndCollation(
@@ -4129,7 +4143,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       if (call != null) {
         return call.accept(this);
       }
-      final SqlIdentifier fqId = getScope().fullyQualify(id);
+      final SqlIdentifier fqId = getScope().fullyQualify(id).identifier;
       validator.setOriginal(fqId, id);
       return fqId;
     }
@@ -4213,7 +4227,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNode expr = expandedSelectList.get(ordinal);
       expr = stripAs(expr);
       if (expr instanceof SqlIdentifier) {
-        expr = getScope().fullyQualify((SqlIdentifier) expr);
+        expr = getScope().fullyQualify((SqlIdentifier) expr).identifier;
       }
 
       // Create a copy of the expression with the position of the order
@@ -4239,7 +4253,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
 
       // No match. Return identifier unchanged.
-      return getScope().fullyQualify(id);
+      return getScope().fullyQualify(id).identifier;
     }
 
     protected SqlNode visitScoped(SqlCall call) {