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 2017/01/13 19:22:41 UTC

[3/4] calcite git commit: [CALCITE-1549] More helpful error message when schema, table or column not found

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
index 49f109f..c8eb6b0 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorScope.java
@@ -25,13 +25,16 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.util.Pair;
 
+import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
 /**
  * Name-resolution scope. Represents any position in a parse tree than an
@@ -60,27 +63,38 @@ public interface SqlValidatorScope {
    * Looks up a node with a given name. Returns null if none is found.
    *
    * @param names       Name of node to find, maybe partially or fully qualified
+   * @param nameMatcher Name matcher
    * @param deep        Whether to look more than one level deep
    * @param resolved    Callback wherein to write the match(es) we find
    */
-  void resolve(List<String> names, boolean deep, Resolved resolved);
+  void resolve(List<String> names, SqlNameMatcher nameMatcher, boolean deep,
+      Resolved resolved);
+
+  /** @deprecated Use
+   * {@link #findQualifyingTableNames(String, SqlNode, SqlNameMatcher)} */
+  @Deprecated // to be removed before 2.0
+  Pair<String, SqlValidatorNamespace> findQualifyingTableName(String columnName,
+      SqlNode ctx);
 
   /**
-   * Finds the table alias which is implicitly qualifying an unqualified
-   * column name. Throws an error if there is not exactly one table.
+   * Finds all table aliases which are implicitly qualifying an unqualified
+   * column name.
    *
    * <p>This method is only implemented in scopes (such as
    * {@link org.apache.calcite.sql.validate.SelectScope}) which can be the
    * context for name-resolution. In scopes such as
    * {@link org.apache.calcite.sql.validate.IdentifierNamespace}, it throws
-   * {@link UnsupportedOperationException}.</p>
+   * {@link UnsupportedOperationException}.
    *
    * @param columnName Column name
    * @param ctx        Validation context, to appear in any error thrown
-   * @return Table alias and namespace
+   * @param nameMatcher Name matcher
+   *
+   * @return Map of applicable table alias and namespaces, never null, empty
+   * if no aliases found
    */
-  Pair<String, SqlValidatorNamespace> findQualifyingTableName(String columnName,
-      SqlNode ctx);
+  Map<String, ScopeChild> findQualifyingTableNames(String columnName,
+      SqlNode ctx, SqlNameMatcher nameMatcher);
 
   /**
    * Collects the {@link SqlMoniker}s of all possible columns in this scope.
@@ -163,34 +177,47 @@ public interface SqlValidatorScope {
    */
   void validateExpr(SqlNode expr);
 
+  /** @deprecated Use
+   * {@link #resolveTable(List, SqlNameMatcher, Path, Resolved)}. */
+  @Deprecated // to be removed before 2.0
+  SqlValidatorNamespace getTableNamespace(List<String> names);
+
   /**
-   * Looks up a table in this scope from its name. If found, returns the
+   * Looks up a table in this scope from its name. If found, calls
+   * {@link Resolved#resolve(List, SqlNameMatcher, boolean, Resolved)}.
    * {@link TableNamespace} that wraps it. If the "table" is defined in a
    * {@code WITH} clause it may be a query, not a table after all.
    *
+   * <p>The name matcher is not null, and one typically uses
+   * {@link SqlValidatorCatalogReader#nameMatcher()}.
+   *
    * @param names Name of table, may be qualified or fully-qualified
-   * @return Namespace of table
+   * @param nameMatcher Name matcher
+   * @param path List of names that we have traversed through so far
    */
-  SqlValidatorNamespace getTableNamespace(List<String> names);
+  void resolveTable(List<String> names, SqlNameMatcher nameMatcher, Path path,
+      Resolved resolved);
 
   /** Converts the type of an expression to nullable, if the context
    * warrants it. */
   RelDataType nullifyType(SqlNode node, RelDataType type);
 
-  /** Callback from
-   * {@link SqlValidatorScope#resolve(List, boolean, Resolved)}. */
+  /** Callback from {@link SqlValidatorScope#resolve}. */
   interface Resolved {
     void found(SqlValidatorNamespace namespace, boolean nullable,
-        SqlValidatorScope scope, Path path);
+        SqlValidatorScope scope, Path path, List<String> remainingNames);
     int count();
-    Path emptyPath();
   }
 
-  /** A sequence of steps by which an identifier was resolved. */
+  /** A sequence of steps by which an identifier was resolved. Immutable. */
   abstract class Path {
-    /** Creates a path which consists of this path plus one additional step. */
-    Step add(RelDataType rowType, int i, StructKind kind) {
-      return new Step(this, rowType, i, kind);
+    /** The empty path. */
+    @SuppressWarnings("StaticInitializerReferencesSubClass")
+    public static final EmptyPath EMPTY = new EmptyPath();
+
+    /** Creates a path that consists of this path plus one additional step. */
+    public Step plus(RelDataType rowType, int i, String name, StructKind kind) {
+      return new Step(this, rowType, i, name, kind);
     }
 
     /** Number of steps in this path. */
@@ -205,6 +232,16 @@ public interface SqlValidatorScope {
       return paths.build();
     }
 
+    /** Returns a list ["step1", "step2"]. */
+    List<String> stepNames() {
+      return Lists.transform(steps(),
+          new Function<Step, String>() {
+            public String apply(Step input) {
+              return input.name;
+            }
+          });
+    }
+
     protected void build(ImmutableList.Builder<Step> paths) {
     }
   }
@@ -218,12 +255,15 @@ public interface SqlValidatorScope {
     final Path parent;
     final RelDataType rowType;
     public final int i;
+    public final String name;
     final StructKind kind;
 
-    Step(Path parent, RelDataType rowType, int i, StructKind kind) {
+    Step(Path parent, RelDataType rowType, int i, String name,
+        StructKind kind) {
       this.parent = Preconditions.checkNotNull(parent);
       this.rowType = rowType; // may be null
       this.i = i;
+      this.name = name;
       this.kind = Preconditions.checkNotNull(kind);
     }
 
@@ -241,21 +281,17 @@ public interface SqlValidatorScope {
    * {@link org.apache.calcite.sql.validate.SqlValidatorScope.Resolved}. */
   class ResolvedImpl implements Resolved {
     final List<Resolve> resolves = new ArrayList<>();
-    private final EmptyPath emptyPath = new EmptyPath();
 
     public void found(SqlValidatorNamespace namespace, boolean nullable,
-        SqlValidatorScope scope, Path path) {
-      resolves.add(new Resolve(namespace, nullable, scope, path));
+        SqlValidatorScope scope, Path path, List<String> remainingNames) {
+      resolves.add(
+          new Resolve(namespace, nullable, scope, path, remainingNames));
     }
 
     public int count() {
       return resolves.size();
     }
 
-    public Path emptyPath() {
-      return emptyPath;
-    }
-
     public Resolve only() {
       return Iterables.getOnlyElement(resolves);
     }
@@ -273,13 +309,17 @@ public interface SqlValidatorScope {
     private final boolean nullable;
     public final SqlValidatorScope scope; // may be null
     public final Path path;
+    /** Names not matched; empty if it was a full match. */
+    final List<String> remainingNames;
 
     Resolve(SqlValidatorNamespace namespace, boolean nullable,
-        SqlValidatorScope scope, Path path) {
+        SqlValidatorScope scope, Path path, List<String> remainingNames) {
       this.namespace = Preconditions.checkNotNull(namespace);
       this.nullable = nullable;
       this.scope = scope;
       this.path = Preconditions.checkNotNull(path);
+      this.remainingNames = remainingNames == null ? ImmutableList.<String>of()
+          : ImmutableList.copyOf(remainingNames);
     }
 
     /** The row type of the found namespace, nullable if the lookup has

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index c4b07f1..34149f9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -108,14 +108,7 @@ public class SqlValidatorUtil {
     return table;
   }
 
-  /**
-   * Looks up a field with a given name, returning null if not found.
-   *
-   * @param caseSensitive Whether match is case-sensitive
-   * @param rowType    Row type
-   * @param columnName Field name
-   * @return Field, or null if not found
-   */
+  @Deprecated // to be removed before 2.0
   public static RelDataTypeField lookupField(boolean caseSensitive,
       final RelDataType rowType, String columnName) {
     return rowType.getField(columnName, caseSensitive, false);
@@ -452,7 +445,8 @@ public class SqlValidatorUtil {
       RelOptTable table) {
     final Table t = table == null ? null : table.unwrap(Table.class);
     if (!(t instanceof CustomColumnResolvingTable)) {
-      return catalogReader.field(rowType, id.getSimple());
+      final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+      return nameMatcher.field(rowType, id.getSimple());
     }
 
     final List<Pair<RelDataTypeField, List<String>>> entries =
@@ -478,9 +472,11 @@ public class SqlValidatorUtil {
       SqlValidatorScope scope,
       List<String> names) {
     assert names.size() > 0;
+    final SqlNameMatcher nameMatcher =
+        scope.getValidator().getCatalogReader().nameMatcher();
     final SqlValidatorScope.ResolvedImpl resolved =
         new SqlValidatorScope.ResolvedImpl();
-    scope.resolve(ImmutableList.of(names.get(0)), false, resolved);
+    scope.resolve(ImmutableList.of(names.get(0)), nameMatcher, false, resolved);
     assert resolved.count() == 1;
     SqlValidatorNamespace namespace = resolved.only().namespace;
     for (String name : Util.skip(names)) {
@@ -498,15 +494,10 @@ public class SqlValidatorUtil {
     List<String> subNames = Util.skipLast(names);
 
     // Try successively with catalog.schema, catalog and no prefix
-    List<String> x = catalogReader.getSchemaName();
-    for (;;) {
+    for (List<String> x : catalogReader.getSchemaPaths()) {
       final List<String> names2 =
           ImmutableList.<String>builder().addAll(x).addAll(subNames).build();
       hints.addAll(catalogReader.getAllSchemaObjectNames(names2));
-      if (x.isEmpty()) {
-        break;
-      }
-      x = Util.skipLast(x);
     }
   }
 
@@ -689,9 +680,12 @@ public class SqlValidatorUtil {
       String originalRelName = expr.names.get(0);
       String originalFieldName = expr.names.get(1);
 
+      final SqlNameMatcher nameMatcher =
+          scope.getValidator().getCatalogReader().nameMatcher();
       final SqlValidatorScope.ResolvedImpl resolved =
           new SqlValidatorScope.ResolvedImpl();
-      scope.resolve(ImmutableList.of(originalRelName), false, resolved);
+      scope.resolve(ImmutableList.of(originalRelName), nameMatcher, false,
+          resolved);
 
       assert resolved.count() == 1;
       final SqlValidatorScope.Resolve resolve = resolved.only();
@@ -714,9 +708,7 @@ public class SqlValidatorUtil {
         }
       }
 
-      RelDataTypeField field =
-          scope.getValidator().getCatalogReader().field(rowType,
-              originalFieldName);
+      RelDataTypeField field = nameMatcher.field(rowType, originalFieldName);
       int origPos = namespaceOffset + field.getIndex();
 
       groupExprProjection.put(origPos, ref);
@@ -835,6 +827,7 @@ public class SqlValidatorUtil {
 
     /** Copies a list of nodes. */
     public static SqlNodeList copy(SqlValidatorScope scope, SqlNodeList list) {
+      //noinspection deprecation
       return (SqlNodeList) list.accept(new DeepCopier(scope));
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
index 4d7bd65..8a6f29d 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/WithScope.java
@@ -54,17 +54,30 @@ class WithScope extends ListScope {
     return super.getTableNamespace(names);
   }
 
-  @Override public void resolve(List<String> names, boolean deep,
-      Resolved resolved) {
+  @Override public void resolveTable(List<String> names,
+      SqlNameMatcher nameMatcher, Path path, Resolved resolved) {
     if (names.size() == 1
         && names.equals(withItem.name.names)) {
       final SqlValidatorNamespace ns = validator.getNamespace(withItem);
-      final Step path = resolved.emptyPath()
-          .add(ns.getRowType(), 0, StructKind.FULLY_QUALIFIED);
-      resolved.found(ns, false, null, path);
+      final Step path2 = path
+          .plus(ns.getRowType(), 0, names.get(0), StructKind.FULLY_QUALIFIED);
+      resolved.found(ns, false, null, path2, null);
       return;
     }
-    super.resolve(names, deep, resolved);
+    super.resolveTable(names, nameMatcher, path, resolved);
+  }
+
+  @Override public void resolve(List<String> names, SqlNameMatcher nameMatcher,
+      boolean deep, Resolved resolved) {
+    if (names.size() == 1
+        && names.equals(withItem.name.names)) {
+      final SqlValidatorNamespace ns = validator.getNamespace(withItem);
+      final Step path = Path.EMPTY.plus(ns.getRowType(), 0, names.get(0),
+          StructKind.FULLY_QUALIFIED);
+      resolved.found(ns, false, null, path, null);
+      return;
+    }
+    super.resolve(names, nameMatcher, deep, resolved);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 fb27449..857eb7c 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -142,6 +142,7 @@ import org.apache.calcite.sql.validate.ListScope;
 import org.apache.calcite.sql.validate.ParameterScope;
 import org.apache.calcite.sql.validate.SelectScope;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
 import org.apache.calcite.sql.validate.SqlQualified;
 import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
 import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
@@ -434,7 +435,7 @@ public class SqlToRelConverter {
         validator.getTypeFactory().createStructType(
             Pair.right(validatedFields),
             SqlValidatorUtil.uniquify(Pair.left(validatedFields),
-                catalogReader.isCaseSensitive()));
+                catalogReader.nameMatcher().isCaseSensitive()));
 
     final List<RelDataTypeField> convertedFields =
         result.getRowType().getFieldList().subList(0, validatedFields.size());
@@ -2202,10 +2203,12 @@ public class SqlToRelConverter {
       String originalRelName = lookup.getOriginalRelName();
       String originalFieldName = fieldAccess.getField().getName();
 
-      SqlValidatorScope.ResolvedImpl resolved =
+      final SqlNameMatcher nameMatcher =
+          lookup.bb.scope.getValidator().getCatalogReader().nameMatcher();
+      final SqlValidatorScope.ResolvedImpl resolved =
           new SqlValidatorScope.ResolvedImpl();
-      lookup.bb.scope.resolve(ImmutableList.of(originalRelName), false,
-          resolved);
+      lookup.bb.scope.resolve(ImmutableList.of(originalRelName),
+          nameMatcher, false, resolved);
       assert resolved.count() == 1;
       final SqlValidatorScope.Resolve resolve = resolved.only();
       final SqlValidatorNamespace foundNs = resolve.namespace;
@@ -2308,10 +2311,12 @@ public class SqlToRelConverter {
       DeferredLookup lookup = mapCorrelToDeferred.get(correlName);
       String originalRelName = lookup.getOriginalRelName();
 
+      final SqlNameMatcher nameMatcher =
+          lookup.bb.scope.getValidator().getCatalogReader().nameMatcher();
       final SqlValidatorScope.ResolvedImpl resolved =
           new SqlValidatorScope.ResolvedImpl();
-      lookup.bb.scope.resolve(ImmutableList.of(originalRelName), false,
-          resolved);
+      lookup.bb.scope.resolve(ImmutableList.of(originalRelName), nameMatcher,
+          false, resolved);
 
       SqlValidatorScope ancestorScope = resolved.only().scope;
 
@@ -2385,6 +2390,7 @@ public class SqlToRelConverter {
   private RexNode convertUsing(SqlValidatorNamespace leftNamespace,
       SqlValidatorNamespace rightNamespace,
       List<String> nameList) {
+    final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
     final List<RexNode> list = Lists.newArrayList();
     for (String name : nameList) {
       List<RexNode> operands = new ArrayList<>();
@@ -2392,7 +2398,7 @@ public class SqlToRelConverter {
       for (SqlValidatorNamespace n : ImmutableList.of(leftNamespace,
           rightNamespace)) {
         final RelDataType rowType = n.getRowType();
-        final RelDataTypeField field = catalogReader.field(rowType, name);
+        final RelDataTypeField field = nameMatcher.field(rowType, name);
         operands.add(
             rexBuilder.makeInputRef(field.getType(),
                 offset + field.getIndex()));
@@ -3064,8 +3070,9 @@ public class SqlToRelConverter {
     // expression list according to the ordinal value returned from
     // the table construct, leaving nulls in the list for columns
     // that are not referenced.
+    final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
     for (Pair<String, RexNode> p : Pair.zip(targetColumnNames, columnExprs)) {
-      RelDataTypeField field = catalogReader.field(targetRowType, p.left);
+      RelDataTypeField field = nameMatcher.field(targetRowType, p.left);
       assert field != null : "column " + p.left + " not found";
       sourceExps.set(field.getIndex(), p.right);
     }
@@ -3505,7 +3512,8 @@ public class SqlToRelConverter {
       fieldNames.add(deriveAlias(expr, aliases, i));
     }
 
-    fieldNames = SqlValidatorUtil.uniquify(fieldNames, catalogReader.isCaseSensitive());
+    fieldNames = SqlValidatorUtil.uniquify(fieldNames,
+        catalogReader.nameMatcher().isCaseSensitive());
 
     bb.setRoot(
         RelOptUtil.createProject(bb.root, exprs, fieldNames),
@@ -3902,9 +3910,11 @@ public class SqlToRelConverter {
         }
         return Pair.of(node, null);
       }
+      final SqlNameMatcher nameMatcher =
+          scope.getValidator().getCatalogReader().nameMatcher();
       final SqlValidatorScope.ResolvedImpl resolved =
           new SqlValidatorScope.ResolvedImpl();
-      scope.resolve(qualified.prefix(), false, resolved);
+      scope.resolve(qualified.prefix(), nameMatcher, false, resolved);
       if (!(resolved.count() == 1)) {
         return null;
       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/util/Util.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 4117a8d..fee522d 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2358,6 +2358,29 @@ public class Util {
     return "".equals(v) || "true".equalsIgnoreCase(v);
   }
 
+  /** Returns a copy of a list of lists, making the component lists immutable if
+   * they are not already. */
+  public static <E> List<List<E>>
+  immutableCopy(Iterable<? extends Iterable<E>> lists) {
+    int n = 0;
+    for (Iterable<E> list : lists) {
+      if (!(list instanceof ImmutableList)) {
+        ++n;
+      }
+    }
+    if (n == 0) {
+      // Lists are already immutable. Furthermore, if the outer list is
+      // immutable we will just return "lists" unchanged.
+      return ImmutableList.copyOf((Iterable) lists);
+    }
+    final ImmutableList.Builder<List<E>> builder =
+        ImmutableList.builder();
+    for (Iterable<E> list : lists) {
+      builder.add(ImmutableList.copyOf(list));
+    }
+    return builder.build();
+  }
+
   //~ Inner Classes ----------------------------------------------------------
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index 5a278b6..7be1266 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -57,9 +57,17 @@ DuplicateTargetColumn=Target column ''{0}'' is assigned more than once
 UnmatchInsertColumn=Number of INSERT target columns ({0,number}) does not equal number of source items ({1,number})
 TypeNotAssignable=Cannot assign to target field ''{0}'' of type {1} from source field ''{2}'' of type {3}
 TableNameNotFound=Table ''{0}'' not found
+TableNotFound=Table ''{0}'' not found
+TableNameNotFoundDidYouMean=Table ''{0}'' not found; did you mean ''{1}''?
+ObjectNotFound=Object ''{0}'' not found
+ObjectNotFoundWithin=Object ''{0}'' not found within ''{1}''
+ObjectNotFoundDidYouMean=Object ''{0}'' not found; did you mean ''{1}''?
+ObjectNotFoundWithinDidYouMean=Object ''{0}'' not found within ''{1}''; did you mean ''{2}''?
 NotASequence=Table ''{0}'' is not a sequence
 ColumnNotFound=Column ''{0}'' not found in any table
+ColumnNotFoundDidYouMean=Column ''{0}'' not found in any table; did you mean ''{1}''?
 ColumnNotFoundInTable=Column ''{0}'' not found in table ''{1}''
+ColumnNotFoundInTableDidYouMean=Column ''{0}'' not found in table ''{1}''; did you mean ''{2}''?
 ColumnAmbiguous=Column ''{0}'' is ambiguous
 NeedQueryOp=Operand {0} must be a query
 NeedSameTypeParameter=Parameters must be of the same type
@@ -199,7 +207,6 @@ CannotStreamValues=Cannot stream VALUES
 ModifiableViewMustBeBasedOnSingleTable=Modifiable view must be based on a single table
 MoreThanOneMappedColumn=View is not modifiable. More than one expression maps to column ''{0}'' of base table ''{1}''
 NoValueSuppliedForViewColumn=View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''
-TableNotFound=Table ''{0}'' not found
 StarRequiresRecordType=Not a record type. The ''*'' operator requires a record
 FilterMustBeBoolean=FILTER expression must be of type BOOLEAN
 CannotStreamResultsForNonStreamingInputs=Cannot stream results of a query with no streaming inputs: ''{0}''. At least one input should be convertible to a stream

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java b/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
index e760cd0..ddc997d 100644
--- a/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
+++ b/core/src/test/java/org/apache/calcite/prepare/LookupOperatorOverloadsTest.java
@@ -32,6 +32,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
 import org.apache.calcite.util.Smalls;
 
+import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import org.junit.Test;
@@ -141,8 +142,8 @@ public class LookupOperatorOverloadsTest {
           statement.createPrepareContext();
       final JavaTypeFactory typeFactory = prepareContext.getTypeFactory();
       CalciteCatalogReader reader =
-          new CalciteCatalogReader(prepareContext.getRootSchema(), false, null,
-              typeFactory);
+          new CalciteCatalogReader(prepareContext.getRootSchema(), false,
+              ImmutableList.<String>of(), typeFactory);
 
       final List<SqlOperator> operatorList = new ArrayList<>();
       SqlIdentifier myFuncIdentifier =

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index be651ff..63c1cdd 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -16,15 +16,15 @@
  */
 package org.apache.calcite.sql.test;
 
+import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.avatica.util.Casing;
 import org.apache.calcite.avatica.util.Quoting;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.advise.SqlAdvisor;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
-import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -90,8 +90,8 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
     final boolean caseSensitive = (Boolean) factory.get("caseSensitive");
     final SqlConformance conformance =
         (SqlConformance) factory.get("conformance");
-    final RelDataTypeFactory typeFactory =
-        new SqlTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
+    final JavaTypeFactory typeFactory =
+        new JavaTypeFactoryImpl(RelDataTypeSystem.DEFAULT);
     return SqlValidatorUtil.newValidator(operatorTable,
         new MockCatalogReader(typeFactory, caseSensitive).init(),
         typeFactory, conformance);

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
index 326a460..24a4626 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlAdvisorTest.java
@@ -37,10 +37,12 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.TreeSet;
 
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.fail;
@@ -65,6 +67,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
 
   protected static final List<String> SALES_TABLES =
       Arrays.asList(
+          "SCHEMA(CATALOG.SALES)",
           "TABLE(CATALOG.SALES.EMP)",
           "TABLE(CATALOG.SALES.EMP_B)",
           "TABLE(CATALOG.SALES.EMP_20)",
@@ -359,8 +362,8 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
       String sql,
       List<String>... expectedLists) throws Exception {
     List<String> expectedList = plus(expectedLists);
-    Collections.sort(expectedList);
-    assertHint(sql, toString(expectedList));
+    final String expected = toString(new TreeSet<>(expectedList));
+    assertHint(sql, expected);
   }
 
   /**
@@ -410,8 +413,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
       String sql,
       List<String>... expectedResults) {
     List<String> expectedList = plus(expectedResults);
-    Collections.sort(expectedList);
-    String expected = toString(expectedList);
+    String expected = toString(new TreeSet<>(expectedList));
     assertComplete(sql, expected, null);
   }
 
@@ -480,7 +482,7 @@ public class SqlAdvisorTest extends SqlValidatorTestCase {
    * @param list List
    * @return String with one item of the list per line
    */
-  private static <T> String toString(List<T> list) {
+  private static <T> String toString(Collection<T> list) {
     StringBuilder buf = new StringBuilder();
     for (T t : list) {
       buf.append(t).append("\n");

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/CollectionTypeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CollectionTypeTest.java b/core/src/test/java/org/apache/calcite/test/CollectionTypeTest.java
index 2a2c52b..23d51db 100644
--- a/core/src/test/java/org/apache/calcite/test/CollectionTypeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/CollectionTypeTest.java
@@ -387,6 +387,7 @@ public class CollectionTypeTest {
           .build();
     }
 
+
     public Statistic getStatistic() {
       return Statistics.UNKNOWN;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
index 59a3bfa..a2ec482 100644
--- a/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ExceptionMessageTest.java
@@ -141,7 +141,7 @@ public class ExceptionMessageTest {
       fail("Query should fail");
     } catch (SQLException e) {
       assertThat(e.getMessage(),
-          containsString("Table 'nonexistentTable' not found"));
+          containsString("Object 'nonexistentTable' not found"));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 126df14..25d3dcf 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -5192,7 +5192,7 @@ public class JdbcTest {
             + "empid=150; deptno=10; name=Sebastian; salary=7000.0; commission=null\n"
             + "empid=110; deptno=10; name=Theodore; salary=11500.0; commission=250\n");
     that.query("select * from \"adhoc\".EMPLOYEES")
-        .throws_("Table 'adhoc.EMPLOYEES' not found");
+        .throws_("Object 'EMPLOYEES' not found within 'adhoc'");
   }
 
   /** Test case for
@@ -6113,7 +6113,15 @@ public class JdbcTest {
     final CalciteAssert.AssertThat with2 =
         CalciteAssert.that().with(Lex.JAVA);
     with2.query("select COUNT(*) as c from `metaData`.`tAbles`")
-        .throws_("Table 'metaData.tAbles' not found");
+        .throws_("Object 'metaData' not found; did you mean 'metadata'?");
+    with2.query("select COUNT(*) as c from `metaData`.`TABLES`")
+        .throws_("Object 'metaData' not found; did you mean 'metadata'?");
+    with2.query("select COUNT(*) as c from `metaData`.`tables`")
+        .throws_("Object 'metaData' not found; did you mean 'metadata'?");
+    with2.query("select COUNT(*) as c from `metaData`.`nonExistent`")
+        .throws_("Object 'metaData' not found; did you mean 'metadata'?");
+    with2.query("select COUNT(*) as c from `metadata`.`tAbles`")
+        .throws_("Object 'tAbles' not found within 'metadata'; did you mean 'TABLES'?");
   }
 
   /** Test case for
@@ -6126,7 +6134,7 @@ public class JdbcTest {
     // With [CALCITE-1563], the following query succeeded; it queried
     // metadata.tables.
     with.query("select COUNT(*) as c from `metaData`.`zoo`")
-        .throws_("Table 'metaData.zoo' not found");
+        .throws_("Object 'zoo' not found within 'metadata'");
     with.query("select COUNT(*) as c from `metaData`.`tAbLes`")
         .returns("c=2\n");
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/LatticeTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/LatticeTest.java b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
index be87b1c..54f4ea7 100644
--- a/core/src/test/java/org/apache/calcite/test/LatticeTest.java
+++ b/core/src/test/java/org/apache/calcite/test/LatticeTest.java
@@ -172,7 +172,7 @@ public class LatticeTest {
   @Test public void testLatticeInvalidSqlFails() {
     modelWithLattice("star", "select foo from nonexistent")
         .connectThrows("Error instantiating JsonLattice(name=star, ")
-        .connectThrows("Table 'NONEXISTENT' not found");
+        .connectThrows("Object 'NONEXISTENT' not found");
   }
 
   /** Tests a lattice whose SQL is invalid because it contains a GROUP BY. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
index a21218b..18ce756 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -16,13 +16,14 @@
  */
 package org.apache.calcite.test;
 
+import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.QueryProvider;
 import org.apache.calcite.linq4j.Queryable;
 import org.apache.calcite.linq4j.tree.Expression;
-import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptSchema;
 import org.apache.calcite.plan.RelOptTable;
+import org.apache.calcite.prepare.CalciteCatalogReader;
 import org.apache.calcite.prepare.Prepare;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
@@ -54,25 +55,24 @@ import org.apache.calcite.schema.Path;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Schemas;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.StreamableTable;
 import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.Wrapper;
+import org.apache.calcite.schema.impl.AbstractSchema;
 import org.apache.calcite.sql.SqlAccessType;
 import org.apache.calcite.sql.SqlCollation;
-import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlIntervalQualifier;
-import org.apache.calcite.sql.SqlOperator;
-import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.ObjectSqlType;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlModality;
-import org.apache.calcite.sql.validate.SqlMoniker;
-import org.apache.calcite.sql.validate.SqlMonikerImpl;
-import org.apache.calcite.sql.validate.SqlMonikerType;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
+import org.apache.calcite.sql.validate.SqlNameMatcher;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
-import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Litmus;
@@ -82,7 +82,6 @@ import org.apache.calcite.util.Util;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
-import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 
 import java.lang.reflect.Type;
@@ -105,22 +104,15 @@ import java.util.Set;
  * Also two streams "ORDERS", "SHIPMENTS";
  * and a view "EMP_20".
  */
-public class MockCatalogReader implements Prepare.CatalogReader {
+public class MockCatalogReader extends CalciteCatalogReader {
   //~ Static fields/initializers ---------------------------------------------
 
-  protected static final String DEFAULT_CATALOG = "CATALOG";
-  protected static final String DEFAULT_SCHEMA = "SALES";
-
-  public static final Ordering<Iterable<String>>
-  CASE_INSENSITIVE_LIST_COMPARATOR =
-      Ordering.from(String.CASE_INSENSITIVE_ORDER).lexicographical();
+  static final String DEFAULT_CATALOG = "CATALOG";
+  static final String DEFAULT_SCHEMA = "SALES";
+  static final List<String> PREFIX = ImmutableList.of(DEFAULT_SCHEMA);
 
   //~ Instance fields --------------------------------------------------------
 
-  protected final RelDataTypeFactory typeFactory;
-  private final boolean caseSensitive;
-  private final Map<List<String>, MockTable> tables;
-  protected final Map<String, MockSchema> schemas;
   private RelDataType addressType;
 
   //~ Constructors -----------------------------------------------------------
@@ -134,19 +126,18 @@ public class MockCatalogReader implements Prepare.CatalogReader {
    */
   public MockCatalogReader(RelDataTypeFactory typeFactory,
       boolean caseSensitive) {
-    this.typeFactory = typeFactory;
-    this.caseSensitive = caseSensitive;
-    if (caseSensitive) {
-      tables = Maps.newHashMap();
-      schemas = Maps.newHashMap();
-    } else {
-      tables = Maps.newTreeMap(CASE_INSENSITIVE_LIST_COMPARATOR);
-      schemas = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER);
-    }
+    super(CalciteSchema.createRootSchema(false, true, DEFAULT_CATALOG),
+        SqlNameMatchers.withCaseSensitive(caseSensitive),
+        ImmutableList.of(PREFIX, ImmutableList.<String>of()),
+        typeFactory);
   }
 
   @Override public boolean isCaseSensitive() {
-    return caseSensitive;
+    return nameMatcher.isCaseSensitive();
+  }
+
+  public SqlNameMatcher nameMatcher() {
+    return nameMatcher;
   }
 
   /**
@@ -449,55 +440,26 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   //~ Methods ----------------------------------------------------------------
 
-  public void lookupOperatorOverloads(SqlIdentifier opName,
-      SqlFunctionCategory category, SqlSyntax syntax,
-      List<SqlOperator> operatorList) {
-  }
-
-  public List<SqlOperator> getOperatorList() {
-    return ImmutableList.of();
-  }
-
-  public Prepare.CatalogReader withSchemaPath(List<String> schemaPath) {
-    return this;
-  }
-
-  public Prepare.PreparingTable getTableForMember(List<String> names) {
-    return getTable(names);
-  }
-
-  public RelDataTypeFactory getTypeFactory() {
-    return typeFactory;
-  }
-
-  public void registerRules(RelOptPlanner planner) {
-  }
-
-  protected void registerTable(MockTable table) {
+  protected void registerTable(final MockTable table) {
     table.onRegister(typeFactory);
-    tables.put(table.getQualifiedName(), table);
+    assert table.names.get(0).equals(DEFAULT_CATALOG);
+    final CalciteSchema schema =
+        rootSchema.getSubSchema(table.names.get(1), true);
+    final WrapperTable wrapperTable = new WrapperTable(table);
+    if (table.stream) {
+      schema.add(table.names.get(2),
+          new StreamableWrapperTable(table) {
+            public Table stream() {
+              return wrapperTable;
+            }
+          });
+    } else {
+      schema.add(table.names.get(2), wrapperTable);
+    }
   }
 
   protected void registerSchema(MockSchema schema) {
-    schemas.put(schema.name, schema);
-  }
-
-  public Prepare.PreparingTable getTable(final List<String> names) {
-    switch (names.size()) {
-    case 1:
-      // assume table in SALES schema (the original default)
-      // if it's not supplied, because SqlValidatorTest is effectively
-      // using SALES as its default schema.
-      return tables.get(
-          ImmutableList.of(DEFAULT_CATALOG, DEFAULT_SCHEMA, names.get(0)));
-    case 2:
-      return tables.get(
-          ImmutableList.of(DEFAULT_CATALOG, names.get(0), names.get(1)));
-    case 3:
-      return tables.get(names);
-    default:
-      return null;
-    }
+    rootSchema.add(schema.name, new AbstractSchema());
   }
 
   public RelDataType getNamedType(SqlIdentifier typeName) {
@@ -508,65 +470,6 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     }
   }
 
-  public List<SqlMoniker> getAllSchemaObjectNames(List<String> names) {
-    List<SqlMoniker> result;
-    switch (names.size()) {
-    case 0:
-      // looking for catalog and schema names
-      return ImmutableList.<SqlMoniker>builder()
-          .add(new SqlMonikerImpl(DEFAULT_CATALOG, SqlMonikerType.CATALOG))
-          .addAll(getAllSchemaObjectNames(ImmutableList.of(DEFAULT_CATALOG)))
-          .build();
-    case 1:
-      // looking for schema names
-      result = Lists.newArrayList();
-      for (MockSchema schema : schemas.values()) {
-        final String catalogName = names.get(0);
-        if (schema.getCatalogName().equals(catalogName)) {
-          final ImmutableList<String> names1 =
-              ImmutableList.of(catalogName, schema.name);
-          result.add(new SqlMonikerImpl(names1, SqlMonikerType.SCHEMA));
-        }
-      }
-      return result;
-    case 2:
-      // looking for table names in the given schema
-      MockSchema schema = schemas.get(names.get(1));
-      if (schema == null) {
-        return Collections.emptyList();
-      }
-      result = Lists.newArrayList();
-      for (String tableName : schema.tableNames) {
-        result.add(
-            new SqlMonikerImpl(
-                ImmutableList.of(schema.getCatalogName(), schema.name,
-                    tableName),
-                SqlMonikerType.TABLE));
-      }
-      return result;
-    default:
-      return Collections.emptyList();
-    }
-  }
-
-  public List<String> getSchemaName() {
-    return ImmutableList.of(DEFAULT_CATALOG, DEFAULT_SCHEMA);
-  }
-
-  public RelDataTypeField field(RelDataType rowType, String alias) {
-    return SqlValidatorUtil.lookupField(caseSensitive, rowType, alias);
-  }
-
-  public boolean matches(String string, String name) {
-    return Util.matches(caseSensitive, string, name);
-  }
-
-  public RelDataType createTypeFromProjection(final RelDataType type,
-      final List<String> columnNameList) {
-    return SqlValidatorUtil.createTypeFromProjection(type, columnNameList,
-        typeFactory, caseSensitive);
-  }
-
   private static List<RelCollation> deduceMonotonicity(
       Prepare.PreparingTable table) {
     final List<RelCollation> collationList = Lists.newArrayList();
@@ -1197,6 +1100,63 @@ public class MockCatalogReader implements Prepare.CatalogReader {
           });
     }
   }
+
+  /** Wrapper around a {@link MockTable}, giving it a {@link Table} interface.
+   * You can get the {@code MockTable} by calling {@link #unwrap(Class)}. */
+  private static class WrapperTable implements Table, Wrapper {
+    private final MockTable table;
+
+    WrapperTable(MockTable table) {
+      this.table = table;
+    }
+
+    public <C> C unwrap(Class<C> aClass) {
+      return aClass.isInstance(this) ? aClass.cast(this)
+          : aClass.isInstance(table) ? aClass.cast(table)
+          : null;
+    }
+
+    public RelDataType getRowType(RelDataTypeFactory typeFactory) {
+      return table.getRowType();
+    }
+
+    public Statistic getStatistic() {
+      return new Statistic() {
+        public Double getRowCount() {
+          return table.rowCount;
+        }
+
+        public boolean isKey(ImmutableBitSet columns) {
+          return table.isKey(columns);
+        }
+
+        public List<RelCollation> getCollations() {
+          return table.collationList;
+        }
+
+        public RelDistribution getDistribution() {
+          return table.getDistribution();
+        }
+      };
+    }
+
+    public Schema.TableType getJdbcTableType() {
+      return table.stream ? Schema.TableType.STREAM : Schema.TableType.TABLE;
+    }
+  }
+
+  /** Wrapper around a {@link MockTable}, giving it a {@link StreamableTable}
+   * interface. */
+  private static class StreamableWrapperTable extends WrapperTable
+      implements StreamableTable {
+    StreamableWrapperTable(MockTable table) {
+      super(table);
+    }
+
+    public Table stream() {
+      return this;
+    }
+  }
 }
 
 // End MockCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
index bf2ad40..80ee5b1 100644
--- a/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
+++ b/core/src/test/java/org/apache/calcite/test/MultiJdbcSchemaJoinTest.java
@@ -196,7 +196,7 @@ public class MultiJdbcSchemaJoinTest {
       fail("expected error, got " + rs);
     } catch (SQLException e) {
       assertThat(e.getCause().getCause().getMessage(),
-          equalTo("Table 'DB.TABLE2' not found"));
+          equalTo("Object 'TABLE2' not found within 'DB'"));
     }
 
     stmt1.execute("create table table2(id varchar(10) not null primary key, "
@@ -209,7 +209,7 @@ public class MultiJdbcSchemaJoinTest {
       fail("expected error, got " + rs);
     } catch (SQLException e) {
       assertThat(e.getCause().getCause().getMessage(),
-          equalTo("Table 'DB.TABLE2' not found"));
+          equalTo("Object 'TABLE2' not found within 'DB'"));
     }
 
     // disable caching and table becomes visible

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
index da6a0f5..606f1d6 100644
--- a/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
+++ b/core/src/test/java/org/apache/calcite/test/ReflectiveSchemaTest.java
@@ -621,7 +621,7 @@ public class ReflectiveSchemaTest {
     // BitSet is not a valid relation type. It's as if "bitSet" field does
     // not exist.
     with.query("select * from \"s\".\"bitSet\"")
-        .throws_("Table 's.bitSet' not found");
+        .throws_("Object 'bitSet' not found within 's'");
     // Enumerable field returns 3 records with 0 fields
     with.query("select * from \"s\".\"enumerable\"")
         .returns("\n"

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index 3cfbf35..7520b4a 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -254,7 +254,8 @@ public abstract class SqlToRelTestBase {
     }
 
     public RelOptTable getTableForMember(List<String> names) {
-      final SqlValidatorTable table = catalogReader.getTable(names);
+      final SqlValidatorTable table =
+          catalogReader.getTable(names);
       final RelDataType rowType = table.getRowType();
       final List<RelCollation> collationList = deduceMonotonicity(table);
       if (names.size() < 3) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 02d242b..898ce80 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -4661,7 +4661,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
    */
   @Test public void testStarInFromFails() {
     sql("select emp.empno AS x from ^sales.*^")
-        .fails("Table 'SALES.\\*' not found");
+        .fails("Object '\\*' not found within 'SALES'");
+    sql("select * from ^emp.*^")
+        .fails("Object '\\*' not found within 'SALES.EMP'");
+    sql("select emp.empno AS x from ^emp.*^")
+        .fails("Object '\\*' not found within 'SALES.EMP'");
     sql("select emp.empno from emp where emp.^*^ is not null")
         .fails("Unknown field '\\*'");
   }
@@ -5552,14 +5556,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "with emp3 as (select * from ^emp2^),\n"
             + " emp2 as (select * from emp)\n"
             + "select * from emp3",
-        "Table 'EMP2' not found");
+        "Object 'EMP2' not found");
 
     // forward reference in with-item not used; should still fail
     checkFails(
         "with emp3 as (select * from ^emp2^),\n"
             + " emp2 as (select * from emp)\n"
             + "select * from emp2",
-        "Table 'EMP2' not found");
+        "Object 'EMP2' not found");
 
     // table not used is ok
     checkResultType(
@@ -5572,11 +5576,11 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     checkFails("with emp2 as (select * from emp),\n"
             + " emp3 as (select * from ^emp3^)\n"
             + "values (1)",
-        "Table 'EMP3' not found");
+        "Object 'EMP3' not found");
 
     // self-reference not ok
     checkFails("with emp2 as (select * from ^emp2^)\n"
-        + "select * from emp2 where false", "Table 'EMP2' not found");
+        + "select * from emp2 where false", "Object 'EMP2' not found");
 
     // refer to 2 previous tables, not just immediately preceding
     checkResultType("with emp2 as (select * from emp),\n"
@@ -7097,7 +7101,10 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
             + " BOOLEAN NOT NULL SLACKER) NOT NULL";
     checkResultType("select * from (table emp)", empRecordType);
     checkResultType("table emp", empRecordType);
-    checkFails("table ^nonexistent^", "Table 'NONEXISTENT' not found");
+    checkFails("table ^nonexistent^", "Object 'NONEXISTENT' not found");
+    checkFails("table ^sales.nonexistent^",
+        "Object 'NONEXISTENT' not found within 'SALES'");
+    checkFails("table ^nonexistent.foo^", "Object 'NONEXISTENT' not found");
   }
 
   @Test public void testCollectionTable() {
@@ -7162,7 +7169,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "RecordType(VARCHAR(1024) NOT NULL NAME) NOT NULL");
     checkFails(
         "select * from table(dedup(cursor(select * from ^bloop^),'ename'))",
-        "Table 'BLOOP' not found");
+        "Object 'BLOOP' not found");
   }
 
   @Test public void testScalarSubQuery() {
@@ -7483,11 +7490,16 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     tester1.checkQueryFails(
         "select ^e^.EMPNO from [EMP] as [e]",
-        "Table 'E' not found");
+        "Table 'E' not found; did you mean 'e'\\?");
 
     tester1.checkQueryFails(
         "select ^x^ from (\n"
             + "  select [e].EMPNO as [x] from [EMP] as [e])",
+        "Column 'X' not found in any table; did you mean 'x'\\?");
+
+    tester1.checkQueryFails(
+        "select ^x^ from (\n"
+            + "  select [e].EMPNO as [x ] from [EMP] as [e])",
         "Column 'X' not found in any table");
 
     tester1.checkQueryFails("select EMP.^\"x\"^ from EMP",
@@ -7506,15 +7518,20 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
     tester1.checkQueryFails(
         "select ^e^.EMPNO from EMP as E",
-        "Table 'e' not found");
+        "Table 'e' not found; did you mean 'E'\\?");
 
     tester1.checkQueryFails(
         "select ^E^.EMPNO from EMP as e",
-        "Table 'E' not found");
+        "Table 'E' not found; did you mean 'e'\\?");
 
     tester1.checkQueryFails(
         "select ^x^ from (\n"
             + "  select e.EMPNO as X from EMP as e)",
+        "Column 'x' not found in any table; did you mean 'X'\\?");
+
+    tester1.checkQueryFails(
+        "select ^x^ from (\n"
+            + "  select e.EMPNO as Xx from EMP as e)",
         "Column 'x' not found in any table");
 
     // double-quotes are not valid in this lexical convention
@@ -7546,15 +7563,15 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "RecordType(INTEGER NOT NULL path, INTEGER NOT NULL x) NOT NULL");
     tester1.checkFails(
         "select ^PATH^ from (select 1 as path from (values (true)))",
-        "Column 'PATH' not found in any table",
+        "Column 'PATH' not found in any table; did you mean 'path'\\?",
         false);
     tester1.checkFails(
         "select t.^PATH^ from (select 1 as path from (values (true))) as t",
-        "Column 'PATH' not found in table 't'",
+        "Column 'PATH' not found in table 't'; did you mean 'path'\\?",
         false);
     tester1.checkQueryFails(
         "select t.x, t.^PATH^ from (values (true, 1)) as t(path, x)",
-        "Column 'PATH' not found in table 't'");
+        "Column 'PATH' not found in table 't'; did you mean 'path'\\?");
 
     // Built-in functions can be written in any case, even those with no args,
     // and regardless of spaces between function name and open parenthesis.
@@ -7598,7 +7615,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester2.checkQueryFails(
         "select * from emp as [e] where exists (\n"
             + "select 1 from dept where dept.deptno = ^[E]^.deptno)",
-        "(?s).*Table 'E' not found");
+        "(?s).*Table 'E' not found; did you mean 'e'\\?");
 
     checkFails("select count(1), ^empno^ from emp",
         "Expression 'EMPNO' is not being grouped");
@@ -7643,6 +7660,91 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     tester1.checkQuery("select deptno, count(*) from EMP group by DEPTNO");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-1549">[CALCITE-1549]
+   * Improve error message when table or column not found</a>. */
+  @Test public void testTableNotFoundDidYouMean() {
+    // No table in default schema
+    tester.checkQueryFails("select * from ^unknownTable^",
+        "Object 'UNKNOWNTABLE' not found");
+
+    // Similar table exists in default schema
+    tester.checkQueryFails("select * from ^\"Emp\"^",
+        "Object 'Emp' not found within 'SALES'; did you mean 'EMP'\\?");
+
+    // Schema correct, but no table in specified schema
+    tester.checkQueryFails("select * from ^sales.unknownTable^",
+        "Object 'UNKNOWNTABLE' not found within 'SALES'");
+    // Similar table exists in specified schema
+    tester.checkQueryFails("select * from ^sales.\"Emp\"^",
+        "Object 'Emp' not found within 'SALES'; did you mean 'EMP'\\?");
+
+    // No schema found
+    tester.checkQueryFails("select * from ^unknownSchema.unknownTable^",
+        "Object 'UNKNOWNSCHEMA' not found");
+    // Similar schema found
+    tester.checkQueryFails("select * from ^\"sales\".emp^",
+        "Object 'sales' not found; did you mean 'SALES'\\?");
+    tester.checkQueryFails("select * from ^\"saLes\".\"eMp\"^",
+        "Object 'saLes' not found; did you mean 'SALES'\\?");
+
+    // Spurious after table
+    tester.checkQueryFails("select * from ^emp.foo^",
+        "Object 'FOO' not found within 'SALES\\.EMP'");
+    tester.checkQueryFails("select * from ^sales.emp.foo^",
+        "Object 'FOO' not found within 'SALES\\.EMP'");
+
+    // Alias not found
+    tester.checkQueryFails("select ^aliAs^.\"name\"\n"
+            + "from sales.emp as \"Alias\"",
+        "Table 'ALIAS' not found; did you mean 'Alias'\\?");
+    // Alias not found, fully-qualified
+    tester.checkQueryFails("select ^sales.\"emp\"^.\"name\" from sales.emp",
+        "Table 'SALES\\.emp' not found; did you mean 'EMP'\\?");
+  }
+
+  @Test public void testColumnNotFoundDidYouMean() {
+    // Column not found
+    tester.checkQueryFails("select ^\"unknownColumn\"^ from emp",
+        "Column 'unknownColumn' not found in any table");
+    // Similar column in table, unqualified table name
+    tester.checkQueryFails("select ^\"empNo\"^ from emp",
+        "Column 'empNo' not found in any table; did you mean 'EMPNO'\\?");
+    // Similar column in table, table name qualified with schema
+    tester.checkQueryFails("select ^\"empNo\"^ from sales.emp",
+        "Column 'empNo' not found in any table; did you mean 'EMPNO'\\?");
+    // Similar column in table, table name qualified with catalog and schema
+    tester.checkQueryFails("select ^\"empNo\"^ from catalog.sales.emp",
+        "Column 'empNo' not found in any table; did you mean 'EMPNO'\\?");
+    // With table alias
+    tester.checkQueryFails("select e.^\"empNo\"^ from catalog.sales.emp as e",
+        "Column 'empNo' not found in table 'E'; did you mean 'EMPNO'\\?");
+    // With fully-qualified table alias
+    tester.checkQueryFails("select catalog.sales.emp.^\"empNo\"^\n"
+            + "from catalog.sales.emp",
+        "Column 'empNo' not found in table 'CATALOG\\.SALES\\.EMP'; "
+            + "did you mean 'EMPNO'\\?");
+    // Similar column in table; multiple tables
+    tester.checkQueryFails("select ^\"name\"^ from emp, dept",
+        "Column 'name' not found in any table; did you mean 'NAME'\\?");
+    // Similar column in table; table and a query
+    tester.checkQueryFails("select ^\"name\"^ from emp,\n"
+            + "  (select * from dept) as d",
+        "Column 'name' not found in any table; did you mean 'NAME'\\?");
+    // Similar column in table; table and an un-aliased query
+    tester.checkQueryFails("select ^\"name\"^ from emp, (select * from dept)",
+        "Column 'name' not found in any table; did you mean 'NAME'\\?");
+    // Similar column in table, multiple tables
+    tester.checkQueryFails("select ^\"deptno\"^ from emp,\n"
+            + "  (select deptno as \"deptNo\" from dept)",
+        "Column 'deptno' not found in any table; "
+            + "did you mean 'DEPTNO', 'deptNo'\\?");
+    tester.checkQueryFails("select ^\"deptno\"^ from emp,\n"
+            + "  (select * from dept) as t(\"deptNo\", name)",
+        "Column 'deptno' not found in any table; "
+            + "did you mean 'DEPTNO', 'deptNo'\\?");
+  }
+
   /** Tests matching of built-in operator names. */
   @Test public void testUnquotedBuiltInFunctionNames() {
     final SqlTester mysql = tester

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/java/org/apache/calcite/util/UtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index 4b970ce..42f06c6 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -1285,7 +1285,8 @@ public class UtilTest {
     assertFalse(Util.isDistinct(Arrays.asList("a", null, "b", null)));
   }
 
-  /** Unit test for {@link Util#intersects(Collection, Collection)}. */
+  /** Unit test for
+   * {@link Util#intersects(java.util.Collection, java.util.Collection)}. */
   @Test public void testIntersects() {
     final List<String> empty = Collections.emptyList();
     final List<String> listA = Collections.singletonList("a");
@@ -1698,6 +1699,42 @@ public class UtilTest {
 
   }
 
+  /** Tests {@link Util#immutableCopy(Iterable)}. */
+  @Test public void testImmutableCopy() {
+    final List<Integer> list3 = Arrays.asList(1, 2, 3);
+    final List<Integer> immutableList3 = ImmutableList.copyOf(list3);
+    final List<Integer> list0 = Arrays.asList();
+    final List<Integer> immutableList0 = ImmutableList.copyOf(list0);
+    final List<Integer> list1 = Arrays.asList(1);
+    final List<Integer> immutableList1 = ImmutableList.copyOf(list1);
+
+    final List<List<Integer>> list301 = Arrays.asList(list3, list0, list1);
+    final List<List<Integer>> immutableList301 = Util.immutableCopy(list301);
+    assertThat(immutableList301.size(), is(3));
+    assertThat(immutableList301, is(list301));
+    assertThat(immutableList301, not(sameInstance(list301)));
+    for (List<Integer> list : immutableList301) {
+      assertThat(list, isA((Class) ImmutableList.class));
+    }
+
+    // if you copy the copy, you get the same instance
+    final List<List<Integer>> immutableList301b =
+        Util.immutableCopy(immutableList301);
+    assertThat(immutableList301b, sameInstance(immutableList301));
+    assertThat(immutableList301b, not(sameInstance(list301)));
+
+    // if the elements of the list are immutable lists, they are not copied
+    final List<List<Integer>> list301c =
+        Arrays.asList(immutableList3, immutableList0, immutableList1);
+    final List<List<Integer>> list301d = Util.immutableCopy(list301c);
+    assertThat(list301d.size(), is(3));
+    assertThat(list301d, is(list301));
+    assertThat(list301d, not(sameInstance(list301)));
+    assertThat(list301d.get(0), sameInstance(immutableList3));
+    assertThat(list301d.get(1), sameInstance(immutableList0));
+    assertThat(list301d.get(2), sameInstance(immutableList1));
+  }
+
   @Test public void testAsIndexView() {
     final List<String> values  = Lists.newArrayList("abCde", "X", "y");
     final Map<String, String> map = Util.asIndexMap(values,

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 36a2f93..cc3a4af 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -64,6 +64,71 @@ group by "hr"."emps"."empid";
 
 !ok
 
+# Case-sensitive errors
+select empid from "hr"."emps";
+Column 'EMPID' not found in any table; did you mean 'empid'?
+!error
+
+select empid from "hr".emps;
+Object 'EMPS' not found within 'hr'; did you mean 'emps'?
+!error
+
+select empid from hr.emps;
+Object 'HR' not found; did you mean 'hr'?
+!error
+
+select empid from bad_schema.bad_table;
+Object 'BAD_SCHEMA' not found
+!error
+
+select empid from bad_cat.bad_schema.bad_table;
+Object 'BAD_CAT' not found
+!error
+
+select empid from "catalog".bad_schema.bad_table;
+Object 'catalog' not found
+!error
+
+select empid from catalog.bad_schema.bad_table;
+Object 'CATALOG' not found
+!error
+
+select empid from catalog.HR.bad_table;
+Object 'CATALOG' not found
+!error
+
+select empid from catalog."hr".bad_table;
+Object 'CATALOG' not found
+!error
+
+select empid from catalog."hr".emp;
+Object 'CATALOG' not found
+!error
+
+select empid from HR.bad_table;
+Object 'HR' not found; did you mean 'hr'?
+!error
+
+select empid from "HR".bad_table;
+Object 'HR' not found; did you mean 'hr'?
+!error
+
+select empid from HR."emps";
+Object 'HR' not found; did you mean 'hr'?
+!error
+
+select empid from "hr".bad_table;
+Object 'BAD_TABLE' not found within 'hr'
+!error
+
+select empid from "hr".emps;
+Object 'EMPS' not found within 'hr'; did you mean 'emps'?
+!error
+
+select empid from "hr";
+Object 'hr' not found
+!error
+
 # [CALCITE-307] CAST(timestamp AS DATE) gives ClassCastException
 # Based on [DRILL-1051]
 with data(c_row, c_timestamp) as (select * from (values