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:42 UTC

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

[CALCITE-1549] More helpful error message when schema, table or column not found


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

Branch: refs/heads/master
Commit: 5f9c019080c7231acaf3df80732d915351051d93
Parents: 28ae333
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Jan 3 14:45:27 2017 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jan 13 09:12:17 2017 -0800

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java |   5 +-
 .../org/apache/calcite/jdbc/CalciteSchema.java  |  17 +-
 .../calcite/jdbc/JavaTypeFactoryImpl.java       |  15 +-
 .../calcite/prepare/CalciteCatalogReader.java   | 164 ++++++++-----
 .../org/apache/calcite/prepare/Prepare.java     |   2 +-
 .../apache/calcite/prepare/RelOptTableImpl.java |   4 +-
 .../apache/calcite/runtime/CalciteResource.java |  32 ++-
 .../java/org/apache/calcite/schema/Table.java   |   4 +
 .../java/org/apache/calcite/schema/Wrapper.java |  27 +++
 .../org/apache/calcite/sql/SqlIdentifier.java   |  16 +-
 .../calcite/sql/validate/AbstractNamespace.java |   2 +-
 .../calcite/sql/validate/DelegatingScope.java   | 144 +++++++++---
 .../DelegatingSqlValidatorCatalogReader.java    |   4 +-
 .../apache/calcite/sql/validate/EmptyScope.java | 103 ++++++++-
 .../sql/validate/IdentifierNamespace.java       |  78 ++++++-
 .../apache/calcite/sql/validate/ListScope.java  |  70 ++++--
 .../calcite/sql/validate/OrderByScope.java      |   6 +-
 .../calcite/sql/validate/SqlNameMatcher.java    |  62 +++++
 .../calcite/sql/validate/SqlNameMatchers.java   | 154 +++++++++++++
 .../calcite/sql/validate/SqlQualified.java      |   5 +-
 .../sql/validate/SqlValidatorCatalogReader.java |  41 +++-
 .../calcite/sql/validate/SqlValidatorImpl.java  |  67 +++---
 .../calcite/sql/validate/SqlValidatorScope.java |  94 +++++---
 .../calcite/sql/validate/SqlValidatorUtil.java  |  33 ++-
 .../apache/calcite/sql/validate/WithScope.java  |  25 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  30 ++-
 .../main/java/org/apache/calcite/util/Util.java |  23 ++
 .../calcite/runtime/CalciteResource.properties  |   9 +-
 .../prepare/LookupOperatorOverloadsTest.java    |   5 +-
 .../calcite/sql/test/DefaultSqlTestFactory.java |   8 +-
 .../apache/calcite/sql/test/SqlAdvisorTest.java |  12 +-
 .../apache/calcite/test/CollectionTypeTest.java |   1 +
 .../calcite/test/ExceptionMessageTest.java      |   2 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |  14 +-
 .../org/apache/calcite/test/LatticeTest.java    |   2 +-
 .../apache/calcite/test/MockCatalogReader.java  | 228 ++++++++-----------
 .../calcite/test/MultiJdbcSchemaJoinTest.java   |   4 +-
 .../calcite/test/ReflectiveSchemaTest.java      |   2 +-
 .../apache/calcite/test/SqlToRelTestBase.java   |   3 +-
 .../apache/calcite/test/SqlValidatorTest.java   | 130 +++++++++--
 .../java/org/apache/calcite/util/UtilTest.java  |  39 +++-
 core/src/test/resources/sql/misc.iq             |  65 ++++++
 42 files changed, 1335 insertions(+), 416 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index f06680b..3d87041 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -1661,8 +1661,9 @@ public class RexImpTable {
       assert translatedOperands.size() == 1;
       ConstantExpression x = (ConstantExpression) translatedOperands.get(0);
       List<String> names = Util.stringToList((String) x.value);
-      RelOptTable table =
-          Prepare.CatalogReader.THREAD_LOCAL.get().getTable(names);
+      final Prepare.CatalogReader catalogReader =
+          Prepare.CatalogReader.THREAD_LOCAL.get();
+      RelOptTable table = catalogReader.getTable(names);
       System.out.println("Now, do something with table " + table);
       return super.implement(translator, call, translatedOperands);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
index 89e377d..cb61627 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteSchema.java
@@ -376,12 +376,25 @@ public abstract class CalciteSchema {
    */
   public static CalciteSchema createRootSchema(boolean addMetadataSchema,
       boolean cache) {
+    return createRootSchema(addMetadataSchema, cache, "");
+  }
+
+  /** Creates a root schema.
+   *
+   * @param addMetadataSchema Whether to add a "metadata" schema containing
+   *              definitions of tables, columns etc.
+   * @param cache If true create {@link CachingCalciteSchema};
+   *                if false create {@link SimpleCalciteSchema}
+   * @param name Schema name
+   */
+  public static CalciteSchema createRootSchema(boolean addMetadataSchema,
+      boolean cache, String name) {
     CalciteSchema rootSchema;
     final Schema schema = new CalciteConnectionImpl.RootSchema();
     if (cache) {
-      rootSchema = new CachingCalciteSchema(null, schema, "");
+      rootSchema = new CachingCalciteSchema(null, schema, name);
     } else {
-      rootSchema = new SimpleCalciteSchema(null, schema, "");
+      rootSchema = new SimpleCalciteSchema(null, schema, name);
     }
     if (addMetadataSchema) {
       rootSchema.add("metadata", MetadataSchema.INSTANCE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
index a1f6e3f..c600d6f 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/JavaTypeFactoryImpl.java
@@ -22,6 +22,7 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.linq4j.tree.Types;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
@@ -233,19 +234,25 @@ public class JavaTypeFactoryImpl
   }
 
   public RelDataType toSql(RelDataType type) {
+    return toSql(this, type);
+  }
+
+  /** Converts a type in Java format to a SQL-oriented type. */
+  public static RelDataType toSql(final RelDataTypeFactory typeFactory,
+      RelDataType type) {
     if (type instanceof RelRecordType) {
-      return createStructType(
+      return typeFactory.createStructType(
           Lists.transform(type.getFieldList(),
               new Function<RelDataTypeField, RelDataType>() {
                 public RelDataType apply(RelDataTypeField a0) {
-                  return toSql(a0.getType());
+                  return toSql(typeFactory, a0.getType());
                 }
               }),
           type.getFieldNames());
     }
     if (type instanceof JavaType) {
-      return createTypeWithNullability(
-          createSqlType(type.getSqlTypeName()),
+      return typeFactory.createTypeWithNullability(
+          typeFactory.createSqlType(type.getSqlTypeName()),
           type.isNullable());
     }
     return type;

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 0e14c83..7f0823c 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalciteCatalogReader.java
@@ -16,8 +16,8 @@
  */
 package org.apache.calcite.prepare;
 
-import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
@@ -30,6 +30,7 @@ import org.apache.calcite.schema.ScalarFunction;
 import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableFunction;
 import org.apache.calcite.schema.TableMacro;
+import org.apache.calcite.schema.Wrapper;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlIdentifier;
 import org.apache.calcite.sql.SqlOperator;
@@ -45,6 +46,8 @@ import org.apache.calcite.sql.type.SqlTypeName;
 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.SqlNameMatcher;
+import org.apache.calcite.sql.validate.SqlNameMatchers;
 import org.apache.calcite.sql.validate.SqlUserDefinedAggFunction;
 import org.apache.calcite.sql.validate.SqlUserDefinedFunction;
 import org.apache.calcite.sql.validate.SqlUserDefinedTableFunction;
@@ -52,6 +55,7 @@ import org.apache.calcite.sql.validate.SqlUserDefinedTableMacro;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 import com.google.common.collect.Collections2;
@@ -61,6 +65,7 @@ import com.google.common.collect.Lists;
 
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableSet;
@@ -71,56 +76,74 @@ import java.util.NavigableSet;
  * functions defined schemas.
  */
 public class CalciteCatalogReader implements Prepare.CatalogReader {
-  final CalciteSchema rootSchema;
-  final JavaTypeFactory typeFactory;
-  private final List<String> defaultSchema;
-  private final boolean caseSensitive;
-
-  public CalciteCatalogReader(
-      CalciteSchema rootSchema,
-      boolean caseSensitive,
-      List<String> defaultSchema,
-      JavaTypeFactory typeFactory) {
-    super();
-    assert rootSchema != defaultSchema;
-    this.rootSchema = rootSchema;
-    this.caseSensitive = caseSensitive;
-    this.defaultSchema = defaultSchema;
+  protected final CalciteSchema rootSchema;
+  protected final RelDataTypeFactory typeFactory;
+  private final List<List<String>> schemaPaths;
+  protected final SqlNameMatcher nameMatcher;
+
+  public CalciteCatalogReader(CalciteSchema rootSchema, boolean caseSensitive,
+      List<String> defaultSchema, RelDataTypeFactory typeFactory) {
+    this(rootSchema, SqlNameMatchers.withCaseSensitive(caseSensitive),
+        ImmutableList.of(Preconditions.checkNotNull(defaultSchema),
+            ImmutableList.<String>of()),
+        typeFactory);
+  }
+
+  protected CalciteCatalogReader(CalciteSchema rootSchema,
+      SqlNameMatcher nameMatcher, List<List<String>> schemaPaths,
+      RelDataTypeFactory typeFactory) {
+    this.rootSchema = Preconditions.checkNotNull(rootSchema);
+    this.nameMatcher = nameMatcher;
+    this.schemaPaths =
+        Util.immutableCopy(Util.isDistinct(schemaPaths)
+            ? schemaPaths
+            : new LinkedHashSet<>(schemaPaths));
     this.typeFactory = typeFactory;
   }
 
   public CalciteCatalogReader withSchemaPath(List<String> schemaPath) {
-    return new CalciteCatalogReader(rootSchema, caseSensitive, schemaPath,
-        typeFactory);
+    return new CalciteCatalogReader(rootSchema, nameMatcher,
+        ImmutableList.of(schemaPath, ImmutableList.<String>of()), typeFactory);
   }
 
-  public RelOptTableImpl getTable(final List<String> names) {
+  public Prepare.PreparingTable getTable(final List<String> names) {
     // First look in the default schema, if any.
-    if (defaultSchema != null) {
-      RelOptTableImpl table = getTableFrom(names, defaultSchema);
+    // If not found, look in the root schema.
+    for (List<String> schemaPath : schemaPaths) {
+      Prepare.PreparingTable table =
+          getTableFrom(names, schemaPath, nameMatcher);
       if (table != null) {
         return table;
       }
     }
-    // If not found, look in the root schema
-    return getTableFrom(names, ImmutableList.<String>of());
+    return null;
   }
 
-  private RelOptTableImpl getTableFrom(List<String> names,
-      List<String> schemaNames) {
+  private Prepare.PreparingTable getTableFrom(List<String> names,
+      List<String> schemaNames, SqlNameMatcher nameMatcher) {
     CalciteSchema schema =
-        getSchema(Iterables.concat(schemaNames, Util.skipLast(names)));
+        getSchema(Iterables.concat(schemaNames, Util.skipLast(names)),
+            nameMatcher);
     if (schema == null) {
       return null;
     }
     final String name = Util.last(names);
-    CalciteSchema.TableEntry entry = schema.getTable(name, caseSensitive);
+    CalciteSchema.TableEntry entry =
+        schema.getTable(name, nameMatcher.isCaseSensitive());
     if (entry == null) {
-      entry = schema.getTableBasedOnNullaryFunction(name, caseSensitive);
+      entry = schema.getTableBasedOnNullaryFunction(name,
+          nameMatcher.isCaseSensitive());
     }
     if (entry != null) {
       final Table table = entry.getTable();
       final String name2 = entry.name;
+      if (table instanceof Wrapper) {
+        final Prepare.PreparingTable relOptTable =
+            ((Wrapper) table).unwrap(Prepare.PreparingTable.class);
+        if (relOptTable != null) {
+          return relOptTable;
+        }
+      }
       return RelOptTableImpl.create(this, table.getRowType(typeFactory),
           schema.add(name2, table), null);
     }
@@ -129,21 +152,27 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
 
   private Collection<Function> getFunctionsFrom(List<String> names) {
     final List<Function> functions2 = Lists.newArrayList();
-    final List<? extends List<String>> schemaNameList;
+    final List<List<String>> schemaNameList = new ArrayList<>();
     if (names.size() > 1) {
-      // If name is qualified, ignore path.
-      schemaNameList = ImmutableList.of(ImmutableList.<String>of());
-    } else {
-      CalciteSchema schema = getSchema(defaultSchema);
-      if (schema == null) {
-        schemaNameList = ImmutableList.of();
+      // Name qualified: ignore path. But we do look in "/catalog" and "/",
+      // the last 2 items in the path.
+      if (schemaPaths.size() > 1) {
+        schemaNameList.addAll(Util.skip(schemaPaths));
       } else {
-        schemaNameList = schema.getPath();
+        schemaNameList.addAll(schemaPaths);
+      }
+    } else {
+      for (List<String> schemaPath : schemaPaths) {
+        CalciteSchema schema = getSchema(schemaPath, nameMatcher);
+        if (schema != null) {
+          schemaNameList.addAll(schema.getPath());
+        }
       }
     }
     for (List<String> schemaNames : schemaNameList) {
       CalciteSchema schema =
-          getSchema(Iterables.concat(schemaNames, Util.skipLast(names)));
+          getSchema(Iterables.concat(schemaNames, Util.skipLast(names)),
+              nameMatcher);
       if (schema != null) {
         final String name = Util.last(names);
         functions2.addAll(schema.getFunctions(name, true));
@@ -152,10 +181,15 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
     return functions2;
   }
 
-  private CalciteSchema getSchema(Iterable<String> schemaNames) {
+  private CalciteSchema getSchema(Iterable<String> schemaNames,
+      SqlNameMatcher nameMatcher) {
     CalciteSchema schema = rootSchema;
     for (String schemaName : schemaNames) {
-      schema = schema.getSubSchema(schemaName, caseSensitive);
+      if (schema == rootSchema
+          && nameMatcher.matches(schemaName, schema.getName())) {
+        continue;
+      }
+      schema = schema.getSubSchema(schemaName, nameMatcher.isCaseSensitive());
       if (schema == null) {
         return null;
       }
@@ -168,51 +202,65 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   }
 
   public List<SqlMoniker> getAllSchemaObjectNames(List<String> names) {
-    final CalciteSchema schema = getSchema(names);
+    final CalciteSchema schema = getSchema(names, nameMatcher);
     if (schema == null) {
       return ImmutableList.of();
     }
     final List<SqlMoniker> result = new ArrayList<>();
+
+    // Add root schema if not anonymous
+    if (!schema.name.equals("")) {
+      result.add(moniker(schema, null, SqlMonikerType.SCHEMA));
+    }
+
     final Map<String, CalciteSchema> schemaMap = schema.getSubSchemaMap();
 
     for (String subSchema : schemaMap.keySet()) {
-      result.add(
-          new SqlMonikerImpl(schema.path(subSchema), SqlMonikerType.SCHEMA));
+      result.add(moniker(schema, subSchema, SqlMonikerType.SCHEMA));
     }
 
     for (String table : schema.getTableNames()) {
-      result.add(
-          new SqlMonikerImpl(schema.path(table), SqlMonikerType.TABLE));
+      result.add(moniker(schema, table, SqlMonikerType.TABLE));
     }
 
     final NavigableSet<String> functions = schema.getFunctionNames();
     for (String function : functions) { // views are here as well
-      result.add(
-          new SqlMonikerImpl(schema.path(function), SqlMonikerType.FUNCTION));
+      result.add(moniker(schema, function, SqlMonikerType.FUNCTION));
     }
     return result;
   }
 
-  public List<String> getSchemaName() {
-    return defaultSchema;
+  private SqlMonikerImpl moniker(CalciteSchema schema, String name,
+      SqlMonikerType type) {
+    final List<String> path = schema.path(name);
+    if (path.size() == 1
+        && !schema.root().name.equals("")
+        && type == SqlMonikerType.SCHEMA) {
+      type = SqlMonikerType.CATALOG;
+    }
+    return new SqlMonikerImpl(path, type);
   }
 
-  public RelOptTableImpl getTableForMember(List<String> names) {
+  public List<List<String>> getSchemaPaths() {
+    return schemaPaths;
+  }
+
+  public Prepare.PreparingTable getTableForMember(List<String> names) {
     return getTable(names);
   }
 
   public RelDataTypeField field(RelDataType rowType, String alias) {
-    return SqlValidatorUtil.lookupField(caseSensitive, rowType, alias);
+    return nameMatcher.field(rowType, alias);
   }
 
   public boolean matches(String string, String name) {
-    return Util.matches(caseSensitive, string, name);
+    return nameMatcher.matches(string, name);
   }
 
   public RelDataType createTypeFromProjection(final RelDataType type,
       final List<String> columnNameList) {
     return SqlValidatorUtil.createTypeFromProjection(type, columnNameList,
-        typeFactory, caseSensitive);
+        typeFactory, nameMatcher.isCaseSensitive());
   }
 
   public void lookupOperatorOverloads(final SqlIdentifier opName,
@@ -327,13 +375,17 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
       return typeFactory.createTypeWithNullability(
           typeFactory.createSqlType(SqlTypeName.ANY), true);
     }
-    return typeFactory.toSql(type);
+    return JavaTypeFactoryImpl.toSql(typeFactory, type);
   }
 
   public List<SqlOperator> getOperatorList() {
     return null;
   }
 
+  public CalciteSchema getRootSchema() {
+    return rootSchema;
+  }
+
   public RelDataTypeFactory getTypeFactory() {
     return typeFactory;
   }
@@ -342,10 +394,12 @@ public class CalciteCatalogReader implements Prepare.CatalogReader {
   }
 
   @Override public boolean isCaseSensitive() {
-    return caseSensitive;
+    return nameMatcher.isCaseSensitive();
   }
 
-
+  public SqlNameMatcher nameMatcher() {
+    return nameMatcher;
+  }
 }
 
 // End CalciteCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/prepare/Prepare.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/Prepare.java b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
index b5ceec8..5de9b35 100644
--- a/core/src/main/java/org/apache/calcite/prepare/Prepare.java
+++ b/core/src/main/java/org/apache/calcite/prepare/Prepare.java
@@ -388,7 +388,7 @@ public abstract class Prepare {
      * different schema path. */
     CatalogReader withSchemaPath(List<String> schemaPath);
 
-    PreparingTable getTable(List<String> names);
+    @Override PreparingTable getTable(List<String> names);
 
     ThreadLocal<CatalogReader> THREAD_LOCAL = new ThreadLocal<>();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index 6d7ffac..a9c381e 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -326,11 +326,11 @@ public class RelOptTableImpl implements Prepare.PreparingTable {
     return SqlAccessType.ALL;
   }
 
-  /** Im0plementation of {@link SchemaPlus} that wraps a regular schema and knows
+  /** Implementation of {@link SchemaPlus} that wraps a regular schema and knows
    * its name and parent.
    *
    * <p>It is read-only, and functionality is limited in other ways, it but
-   * allows table expressions to be genenerated. */
+   * allows table expressions to be generated. */
   private static class MySchemaPlus implements SchemaPlus {
     private final SchemaPlus parent;
     private final String name;

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 77a6bf9..9ada4f2 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -158,15 +158,44 @@ public interface CalciteResource {
   @BaseMessage("Table ''{0}'' not found")
   ExInst<SqlValidatorException> tableNameNotFound(String a0);
 
+  @BaseMessage("Table ''{0}'' not found; did you mean ''{1}''?")
+  ExInst<SqlValidatorException> tableNameNotFoundDidYouMean(String a0,
+      String a1);
+
+  /** Same message as {@link #tableNameNotFound(String)} but a different kind
+   * of exception, so it can be used in {@code RelBuilder}. */
+  @BaseMessage("Table ''{0}'' not found")
+  ExInst<CalciteException> tableNotFound(String tableName);
+
+  @BaseMessage("Object ''{0}'' not found")
+  ExInst<SqlValidatorException> objectNotFound(String a0);
+
+  @BaseMessage("Object ''{0}'' not found within ''{1}''")
+  ExInst<SqlValidatorException> objectNotFoundWithin(String a0, String a1);
+
+  @BaseMessage("Object ''{0}'' not found; did you mean ''{1}''?")
+  ExInst<SqlValidatorException> objectNotFoundDidYouMean(String a0, String a1);
+
+  @BaseMessage("Object ''{0}'' not found within ''{1}''; did you mean ''{2}''?")
+  ExInst<SqlValidatorException> objectNotFoundWithinDidYouMean(String a0,
+      String a1, String a2);
+
   @BaseMessage("Table ''{0}'' is not a sequence")
   ExInst<SqlValidatorException> notASequence(String a0);
 
   @BaseMessage("Column ''{0}'' not found in any table")
   ExInst<SqlValidatorException> columnNotFound(String a0);
 
+  @BaseMessage("Column ''{0}'' not found in any table; did you mean ''{1}''?")
+  ExInst<SqlValidatorException> columnNotFoundDidYouMean(String a0, String a1);
+
   @BaseMessage("Column ''{0}'' not found in table ''{1}''")
   ExInst<SqlValidatorException> columnNotFoundInTable(String a0, String a1);
 
+  @BaseMessage("Column ''{0}'' not found in table ''{1}''; did you mean ''{2}''?")
+  ExInst<SqlValidatorException> columnNotFoundInTableDidYouMean(String a0,
+      String a1, String a2);
+
   @BaseMessage("Column ''{0}'' is ambiguous")
   ExInst<SqlValidatorException> columnAmbiguous(String a0);
 
@@ -610,9 +639,6 @@ public interface CalciteResource {
   @BaseMessage("View is not modifiable. No value is supplied for NOT NULL column ''{0}'' of base table ''{1}''")
   ExInst<SqlValidatorException> noValueSuppliedForViewColumn(String columnName, String tableName);
 
-  @BaseMessage("Table ''{0}'' not found")
-  ExInst<CalciteException> tableNotFound(String tableName);
-
   @BaseMessage("Not a record type. The ''*'' operator requires a record")
   ExInst<SqlValidatorException> starRequiresRecordType();
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/schema/Table.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Table.java b/core/src/main/java/org/apache/calcite/schema/Table.java
index b25745e..8062565 100644
--- a/core/src/main/java/org/apache/calcite/schema/Table.java
+++ b/core/src/main/java/org/apache/calcite/schema/Table.java
@@ -34,6 +34,9 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
  * <a href="http://en.wikipedia.org/wiki/Inode">i-node</a> concept in the UNIX
  * filesystem.)</p>
  *
+ * <p>A particular table instance may also implement {@link Wrapper},
+ * to give access to sub-objects.
+ *
  * @see TableMacro
  */
 public interface Table {
@@ -56,6 +59,7 @@ public interface Table {
 
   /** Type of table. */
   Schema.TableType getJdbcTableType();
+
 }
 
 // End Table.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/schema/Wrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/Wrapper.java b/core/src/main/java/org/apache/calcite/schema/Wrapper.java
new file mode 100644
index 0000000..c14439d
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/schema/Wrapper.java
@@ -0,0 +1,27 @@
+/*
+ * 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.schema;
+
+/**
+ * Mix-in interface that allows you to find sub-objects.
+ */
+public interface Wrapper {
+  /** Returns an instance of a class, or null. */
+  <C> C unwrap(Class<C> aClass);
+}
+
+// End Wrapper.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 7e2294c..4839805 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -141,12 +141,22 @@ public class SqlIdentifier extends SqlNode {
     return SqlKind.IDENTIFIER;
   }
 
-  public SqlNode clone(SqlParserPos pos) {
+  @Override public SqlNode clone(SqlParserPos pos) {
     return new SqlIdentifier(names, collation, pos, componentPositions);
   }
 
-  public String toString() {
-    return Util.sepList(Lists.transform(names, EMPTY_TO_STAR), ".");
+  @Override public String toString() {
+    return getString(names);
+  }
+
+  /** Converts a list of strings to a qualified identifier. */
+  public static String getString(List<String> names) {
+    return Util.sepList(toStar(names), ".");
+  }
+
+  /** Converts empty strings in a list of names to stars. */
+  public static List<String> toStar(List<String> names) {
+    return Lists.transform(names, EMPTY_TO_STAR);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
index d5f3c18..b6bd058 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
@@ -149,7 +149,7 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
 
   public boolean fieldExists(String name) {
     final RelDataType rowType = getRowType();
-    return validator.catalogReader.field(rowType, name) != null;
+    return validator.catalogReader.nameMatcher().field(rowType, name) != null;
   }
 
   public List<Pair<SqlNode, SqlMonotonicity>> getMonotonicExprs() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 52983d1..8b004cd 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
@@ -31,10 +31,12 @@ 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 org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
@@ -83,16 +85,18 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     throw new UnsupportedOperationException();
   }
 
-  public void resolve(List<String> names, boolean deep, Resolved resolved) {
-    parent.resolve(names, deep, resolved);
+  public void resolve(List<String> names, SqlNameMatcher nameMatcher,
+      boolean deep, Resolved resolved) {
+    parent.resolve(names, nameMatcher, deep, resolved);
   }
 
   /** If a record type allows implicit references to fields, recursively looks
    * into the fields. Otherwise returns immediately. */
   void resolveInNamespace(SqlValidatorNamespace ns, boolean nullable,
-      List<String> names, Path path, Resolved resolved) {
+      List<String> names, SqlNameMatcher nameMatcher, Path path,
+      Resolved resolved) {
     if (names.isEmpty()) {
-      resolved.found(ns, nullable, this, path);
+      resolved.found(ns, nullable, this, path, null);
       return;
     }
     final RelDataType rowType = ns.getRowType();
@@ -109,32 +113,33 @@ public abstract class DelegatingScope implements SqlValidatorScope {
             final List<String> remainder = entry.getValue();
             final SqlValidatorNamespace ns2 =
                 new FieldNamespace(validator, field.getType());
-            final Step path2 = path.add(rowType, field.getIndex(),
-                StructKind.FULLY_QUALIFIED);
-            resolveInNamespace(ns2, nullable, remainder, path2, resolved);
+            final Step path2 = path.plus(rowType, field.getIndex(),
+                field.getName(), StructKind.FULLY_QUALIFIED);
+            resolveInNamespace(ns2, nullable, remainder, nameMatcher, path2,
+                resolved);
           }
           return;
         }
       }
 
       final String name = names.get(0);
-      final RelDataTypeField field0 =
-          validator.catalogReader.field(rowType, name);
+      final RelDataTypeField field0 = nameMatcher.field(rowType, name);
       if (field0 != null) {
         final SqlValidatorNamespace ns2 = ns.lookupChild(field0.getName());
-        final Step path2 = path.add(rowType, field0.getIndex(),
-            StructKind.FULLY_QUALIFIED);
+        final Step path2 = path.plus(rowType, field0.getIndex(),
+            field0.getName(), StructKind.FULLY_QUALIFIED);
         resolveInNamespace(ns2, nullable, names.subList(1, names.size()),
-            path2, resolved);
+            nameMatcher, path2, resolved);
       } else {
         for (RelDataTypeField field : rowType.getFieldList()) {
           switch (field.getType().getStructKind()) {
           case PEEK_FIELDS:
           case PEEK_FIELDS_DEFAULT:
-            final Step path2 = path.add(rowType, field.getIndex(),
-                field.getType().getStructKind());
+            final Step path2 = path.plus(rowType, field.getIndex(),
+                field.getName(), field.getType().getStructKind());
             final SqlValidatorNamespace ns2 = ns.lookupChild(field.getName());
-            resolveInNamespace(ns2, nullable, names, path2, resolved);
+            resolveInNamespace(ns2, nullable, names, nameMatcher, path2,
+                resolved);
           }
         }
       }
@@ -170,13 +175,20 @@ public abstract class DelegatingScope implements SqlValidatorScope {
 
   public Pair<String, SqlValidatorNamespace>
   findQualifyingTableName(String columnName, SqlNode ctx) {
+    //noinspection deprecation
     return parent.findQualifyingTableName(columnName, ctx);
   }
 
-  protected Map<String, ScopeChild> findQualifyingTables(String columnName) {
+  protected Map<String, ScopeChild> findQualifyingTables(String columnName,
+      SqlNameMatcher nameMatcher) {
     return ImmutableMap.of();
   }
 
+  public Map<String, ScopeChild> findQualifyingTableNames(String columnName,
+      SqlNode ctx, SqlNameMatcher nameMatcher) {
+    return parent.findQualifyingTableNames(columnName, ctx, nameMatcher);
+  }
+
   public RelDataType resolveColumn(String name, SqlNode ctx) {
     return parent.resolveColumn(name, ctx);
   }
@@ -189,6 +201,11 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     return parent.getTableNamespace(names);
   }
 
+  public void resolveTable(List<String> names, SqlNameMatcher nameMatcher,
+      Path path, Resolved resolved) {
+    parent.resolveTable(names, nameMatcher, path, resolved);
+  }
+
   public SqlValidatorScope getOperandScope(SqlCall call) {
     if (call instanceof SqlSelect) {
       return validator.getSelectScope((SqlSelect) call);
@@ -213,20 +230,51 @@ public abstract class DelegatingScope implements SqlValidatorScope {
     }
 
     final SqlIdentifier previous = identifier;
+    final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
     String columnName;
+    final String tableName;
+    final SqlValidatorNamespace namespace;
     switch (identifier.names.size()) {
     case 1: {
       columnName = identifier.names.get(0);
-      final Pair<String, SqlValidatorNamespace> pair =
-          findQualifyingTableName(columnName, identifier);
-      final String tableName = pair.left;
-      final SqlValidatorNamespace namespace = pair.right;
+      final Map<String, ScopeChild> map =
+          findQualifyingTableNames(columnName, identifier, nameMatcher);
+      switch (map.size()) {
+      case 0:
+        if (nameMatcher.isCaseSensitive()) {
+          final SqlNameMatcher liberalMatcher = SqlNameMatchers.liberal();
+          final Map<String, ScopeChild> map2 =
+              findQualifyingTableNames(columnName, identifier, liberalMatcher);
+          if (!map2.isEmpty()) {
+            final List<String> list = new ArrayList<>();
+            for (ScopeChild entry : map2.values()) {
+              final RelDataTypeField field =
+                  liberalMatcher.field(entry.namespace.getRowType(),
+                      columnName);
+              list.add(field.getName());
+            }
+            Collections.sort(list);
+            throw validator.newValidationError(identifier,
+                RESOURCE.columnNotFoundDidYouMean(columnName,
+                    Util.sepList(list, "', '")));
+          }
+        }
+        throw validator.newValidationError(identifier,
+            RESOURCE.columnNotFound(columnName));
+      case 1:
+        tableName = map.keySet().iterator().next();
+        namespace = map.get(tableName).namespace;
+        break;
+      default:
+        throw validator.newValidationError(identifier,
+            RESOURCE.columnAmbiguous(columnName));
+      }
 
       final ResolvedImpl resolved = new ResolvedImpl();
-      resolveInNamespace(namespace, false, identifier.names,
-          resolved.emptyPath(), resolved);
+      resolveInNamespace(namespace, false, identifier.names, nameMatcher,
+          Path.EMPTY, resolved);
       final RelDataTypeField field =
-          validator.catalogReader.field(namespace.getRowType(), columnName);
+          nameMatcher.field(namespace.getRowType(), columnName);
       if (field != null) {
         if (hasAmbiguousUnresolvedStar(namespace.getRowType(), field,
             columnName)) {
@@ -253,7 +301,7 @@ public abstract class DelegatingScope implements SqlValidatorScope {
       for (; i > 0; i--) {
         final SqlIdentifier prefix = identifier.getComponent(0, i);
         resolved.clear();
-        resolve(prefix.names, false, resolved);
+        resolve(prefix.names, nameMatcher, false, resolved);
         if (resolved.count() == 1) {
           final Resolve resolve = resolved.only();
           fromNs = resolve.namespace;
@@ -261,11 +309,24 @@ public abstract class DelegatingScope implements SqlValidatorScope {
           fromRowType = resolve.rowType();
           break;
         }
+        // Look for a table alias that is the wrong case.
+        if (nameMatcher.isCaseSensitive()) {
+          final SqlNameMatcher liberalMatcher = SqlNameMatchers.liberal();
+          resolved.clear();
+          resolve(prefix.names, liberalMatcher, false, resolved);
+          if (resolved.count() == 1) {
+            final Step lastStep = Util.last(resolved.only().path.steps());
+            throw validator.newValidationError(prefix,
+                RESOURCE.tableNameNotFoundDidYouMean(prefix.toString(),
+                    lastStep.name));
+          }
+        }
       }
       if (fromNs == null || fromNs instanceof SchemaNamespace) {
         // Look for a column not qualified by a table alias.
         columnName = identifier.names.get(0);
-        final Map<String, ScopeChild> map = findQualifyingTables(columnName);
+        final Map<String, ScopeChild> map =
+            findQualifyingTables(columnName, nameMatcher);
         switch (map.size()) {
         default:
           final SqlIdentifier prefix1 = identifier.skipLast(1);
@@ -274,21 +335,22 @@ public abstract class DelegatingScope implements SqlValidatorScope {
         case 1: {
           final Map.Entry<String, ScopeChild> entry =
               map.entrySet().iterator().next();
-          final String tableName = entry.getKey();
+          final String tableName2 = map.keySet().iterator().next();
           fromNs = entry.getValue().namespace;
-          fromPath = resolved.emptyPath();
+          fromPath = Path.EMPTY;
 
           // Adding table name is for RecordType column with StructKind.PEEK_FIELDS or
           // StructKind.PEEK_FIELDS only. Access to a field in a RecordType column of
           // other StructKind should always be qualified with table name.
           final RelDataTypeField field =
-              validator.catalogReader.field(fromNs.getRowType(), columnName);
+              nameMatcher.field(fromNs.getRowType(), columnName);
           if (field != null) {
             switch (field.getType().getStructKind()) {
             case PEEK_FIELDS:
             case PEEK_FIELDS_DEFAULT:
               columnName = field.getName(); // use resolved field name
-              resolve(ImmutableList.of(tableName), false, resolved);
+              resolve(ImmutableList.of(tableName2), nameMatcher, false,
+                  resolved);
               if (resolved.count() == 1) {
                 final Resolve resolve = resolved.only();
                 fromNs = resolve.namespace;
@@ -296,7 +358,7 @@ public abstract class DelegatingScope implements SqlValidatorScope {
                 fromRowType = resolve.rowType();
                 identifier = identifier
                     .setName(0, columnName)
-                    .add(0, tableName, SqlParserPos.ZERO);
+                    .add(0, tableName2, SqlParserPos.ZERO);
                 ++i;
                 ++size;
               }
@@ -339,11 +401,27 @@ public abstract class DelegatingScope implements SqlValidatorScope {
       }
       final SqlIdentifier suffix = identifier.getComponent(i, size);
       resolved.clear();
-      resolveInNamespace(fromNs, false, suffix.names, resolved.emptyPath(),
+      resolveInNamespace(fromNs, false, suffix.names, nameMatcher, Path.EMPTY,
           resolved);
       final Path path;
       switch (resolved.count()) {
       case 0:
+        // Maybe the last component was correct, just wrong case
+        if (nameMatcher.isCaseSensitive()) {
+          SqlNameMatcher liberalMatcher = SqlNameMatchers.liberal();
+          resolved.clear();
+          resolveInNamespace(fromNs, false, suffix.names, liberalMatcher,
+              Path.EMPTY, resolved);
+          if (resolved.count() > 0) {
+            int k = size - 1;
+            final SqlIdentifier prefix = identifier.getComponent(0, i);
+            final SqlIdentifier suffix3 = identifier.getComponent(i, k + 1);
+            final Step step = Util.last(resolved.resolves.get(0).path.steps());
+            throw validator.newValidationError(suffix3,
+                RESOURCE.columnNotFoundInTableDidYouMean(suffix3.toString(),
+                    prefix.toString(), step.name));
+          }
+        }
         // Find the shortest suffix that also fails. Suppose we cannot resolve
         // "a.b.c"; we find we cannot resolve "a.b" but can resolve "a". So,
         // the error will be "Column 'a.b' not found".
@@ -351,8 +429,8 @@ public abstract class DelegatingScope implements SqlValidatorScope {
         for (; k > i; --k) {
           SqlIdentifier suffix2 = identifier.getComponent(i, k);
           resolved.clear();
-          resolveInNamespace(fromNs, false, suffix2.names,
-              resolved.emptyPath(), resolved);
+          resolveInNamespace(fromNs, false, suffix2.names, nameMatcher,
+              Path.EMPTY, resolved);
           if (resolved.count() > 0) {
             break;
           }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
index b1d9c39..244afb5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/DelegatingSqlValidatorCatalogReader.java
@@ -52,8 +52,8 @@ public abstract class DelegatingSqlValidatorCatalogReader
     return catalogReader.getAllSchemaObjectNames(names);
   }
 
-  public List<String> getSchemaName() {
-    return catalogReader.getSchemaName();
+  public List<List<String>> getSchemaPaths() {
+    return catalogReader.getSchemaPaths();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 3e3ce3f..b9a38cf 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
@@ -16,7 +16,13 @@
  */
 package org.apache.calcite.sql.validate;
 
+import org.apache.calcite.jdbc.CalciteSchema;
+import org.apache.calcite.prepare.Prepare;
+import org.apache.calcite.prepare.RelOptTableImpl;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.StructKind;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.Wrapper;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlDynamicParam;
@@ -26,9 +32,15 @@ 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 org.apache.calcite.util.Util;
 
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Map;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -64,7 +76,8 @@ class EmptyScope implements SqlValidatorScope {
     throw new UnsupportedOperationException();
   }
 
-  public void resolve(List<String> names, boolean deep, Resolved resolved) {
+  public void resolve(List<String> names, SqlNameMatcher nameMatcher,
+      boolean deep, Resolved resolved) {
   }
 
   public SqlValidatorNamespace getTableNamespace(List<String> names) {
@@ -74,6 +87,89 @@ class EmptyScope implements SqlValidatorScope {
         : null;
   }
 
+  public void resolveTable(List<String> names, SqlNameMatcher nameMatcher,
+      Path path, Resolved resolved) {
+    final List<Resolve> imperfectResolves = new ArrayList<>();
+    final List<Resolve> resolves = ((ResolvedImpl) resolved).resolves;
+
+    // Look in the default schema, then default catalog, then root schema.
+    for (List<String> schemaPath : validator.catalogReader.getSchemaPaths()) {
+      resolve_(validator.catalogReader.getRootSchema(), names, schemaPath,
+          nameMatcher, path, resolved);
+      for (Resolve resolve : resolves) {
+        if (resolve.remainingNames.isEmpty()) {
+          // There is a full match. Return it as the only match.
+          ((ResolvedImpl) resolved).clear();
+          resolves.add(resolve);
+          return;
+        }
+      }
+      imperfectResolves.addAll(resolves);
+    }
+    // If there were no matches in the last round, restore those found in
+    // previous rounds
+    if (resolves.isEmpty()) {
+      resolves.addAll(imperfectResolves);
+    }
+  }
+
+  private void resolve_(final CalciteSchema rootSchema, List<String> names,
+      List<String> schemaNames, SqlNameMatcher nameMatcher, Path path,
+      Resolved resolved) {
+    final List<String> concat = ImmutableList.<String>builder()
+        .addAll(schemaNames).addAll(names).build();
+    CalciteSchema schema = rootSchema;
+    SqlValidatorNamespace namespace = null;
+    List<String> remainingNames = concat;
+    for (String schemaName : concat) {
+      if (schema == rootSchema
+          && nameMatcher.matches(schemaName, schema.name)) {
+        remainingNames = Util.skip(remainingNames);
+        continue;
+      }
+      final CalciteSchema subSchema =
+          schema.getSubSchema(schemaName, nameMatcher.isCaseSensitive());
+      if (subSchema != null) {
+        path = path.plus(null, -1, subSchema.name, StructKind.NONE);
+        remainingNames = Util.skip(remainingNames);
+        schema = subSchema;
+        namespace = new SchemaNamespace(validator,
+            ImmutableList.copyOf(path.stepNames()));
+        continue;
+      }
+      CalciteSchema.TableEntry entry =
+          schema.getTable(schemaName, nameMatcher.isCaseSensitive());
+      if (entry == null) {
+        entry = schema.getTableBasedOnNullaryFunction(schemaName,
+            nameMatcher.isCaseSensitive());
+      }
+      if (entry != null) {
+        path = path.plus(null, -1, entry.name, StructKind.NONE);
+        remainingNames = Util.skip(remainingNames);
+        final Table table = entry.getTable();
+        final String name2 = entry.name;
+        SqlValidatorTable table2 = null;
+        if (table instanceof Wrapper) {
+          table2 = ((Wrapper) table).unwrap(Prepare.PreparingTable.class);
+        }
+        if (table2 == null) {
+          table2 = RelOptTableImpl.create(null,
+              table.getRowType(validator.typeFactory), schema.add(name2, table),
+              null);
+        }
+        namespace = new TableNamespace(validator, table2);
+        resolved.found(namespace, false, this, path, remainingNames);
+        return;
+      }
+      // neither sub-schema nor table
+      if (namespace != null
+          && !remainingNames.equals(names)) {
+        resolved.found(namespace, false, this, path, remainingNames);
+      }
+      return;
+    }
+  }
+
   public RelDataType nullifyType(SqlNode node, RelDataType type) {
     return type;
   }
@@ -105,6 +201,11 @@ class EmptyScope implements SqlValidatorScope {
         RESOURCE.columnNotFound(columnName));
   }
 
+  public Map<String, ScopeChild> findQualifyingTableNames(String columnName,
+      SqlNode ctx, SqlNameMatcher nameMatcher) {
+    return ImmutableMap.of();
+  }
+
   public void addChild(SqlValidatorNamespace ns, String alias,
       boolean nullable) {
     // cannot add to the empty scope

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
index 5282ddb..f198389 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/IdentifierNamespace.java
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.SqlNodeList;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.util.Pair;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -77,8 +78,7 @@ public class IdentifierNamespace extends AbstractNamespace {
     super(validator, enclosingNode);
     this.id = id;
     this.extendList = extendList;
-    this.parentScope = parentScope;
-    assert parentScope != null;
+    this.parentScope = Preconditions.checkNotNull(parentScope);
   }
 
   IdentifierNamespace(SqlValidatorImpl validator, SqlNode node,
@@ -100,13 +100,75 @@ public class IdentifierNamespace extends AbstractNamespace {
     }
   }
 
-  public RelDataType validateImpl(RelDataType targetRowType) {
-    resolvedNamespace = parentScope.getTableNamespace(id.names);
-    if (resolvedNamespace == null) {
-      throw validator.newValidationError(id,
-          RESOURCE.tableNameNotFound(id.toString()));
+  private SqlValidatorNamespace resolveImpl(SqlIdentifier id) {
+    final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
+    final SqlValidatorScope.ResolvedImpl resolved =
+        new SqlValidatorScope.ResolvedImpl();
+    final List<String> names = SqlIdentifier.toStar(id.names);
+    parentScope.resolveTable(names, nameMatcher,
+        SqlValidatorScope.Path.EMPTY, resolved);
+    SqlValidatorScope.Resolve previousResolve = null;
+    if (resolved.count() == 1) {
+      final SqlValidatorScope.Resolve resolve =
+          previousResolve = resolved.only();
+      if (resolve.remainingNames.isEmpty()) {
+        return resolve.namespace;
+      }
+      // If we're not case sensitive, give an error.
+      // If we're case sensitive, we'll shortly try again and give an error
+      // then.
+      if (!nameMatcher.isCaseSensitive()) {
+        throw validator.newValidationError(id,
+            RESOURCE.objectNotFoundWithin(resolve.remainingNames.get(0),
+                SqlIdentifier.getString(resolve.path.stepNames())));
+      }
+    }
+
+    // Failed to match.  If we're matching case-sensitively, try a more
+    // lenient match. If we find something we can offer a helpful hint.
+    if (nameMatcher.isCaseSensitive()) {
+      final SqlNameMatcher liberalMatcher = SqlNameMatchers.liberal();
+      resolved.clear(); // TODO: remove?
+      parentScope.resolveTable(names, liberalMatcher,
+          SqlValidatorScope.Path.EMPTY, resolved);
+      if (resolved.count() == 1) {
+        final SqlValidatorScope.Resolve resolve = resolved.only();
+        if (resolve.remainingNames.isEmpty()
+            || previousResolve == null) {
+          // We didn't match it case-sensitive, so they must have had the
+          // right identifier, wrong case.
+          //
+          // If previousResolve is null, we matched nothing case-sensitive and
+          // everything case-insensitive, so the mismatch must have been at
+          // position 0.
+          final int i = previousResolve == null ? 0
+              : previousResolve.path.stepCount();
+          final int offset = resolve.path.stepCount()
+              + resolve.remainingNames.size() - names.size();
+          final List<String> prefix =
+              resolve.path.stepNames().subList(0, offset + i);
+          final String next = resolve.path.stepNames().get(i + offset);
+          if (prefix.isEmpty()) {
+            throw validator.newValidationError(id,
+                RESOURCE.objectNotFoundDidYouMean(names.get(i), next));
+          } else {
+            throw validator.newValidationError(id,
+                RESOURCE.objectNotFoundWithinDidYouMean(names.get(i),
+                    SqlIdentifier.getString(prefix), next));
+          }
+        } else {
+          throw validator.newValidationError(id,
+              RESOURCE.objectNotFoundWithin(resolve.remainingNames.get(0),
+                  SqlIdentifier.getString(resolve.path.stepNames())));
+        }
+      }
     }
+    throw validator.newValidationError(id,
+        RESOURCE.objectNotFound(id.getComponent(0).toString()));
+  }
 
+  public RelDataType validateImpl(RelDataType targetRowType) {
+    resolvedNamespace = Preconditions.checkNotNull(resolveImpl(id));
     if (resolvedNamespace instanceof TableNamespace) {
       SqlValidatorTable table = resolvedNamespace.getTable();
       if (validator.shouldExpandIdentifiers()) {
@@ -117,7 +179,7 @@ public class IdentifierNamespace extends AbstractNamespace {
           // identifier, as best we can. We assume that qualification
           // adds names to the front, e.g. FOO.BAR becomes BAZ.FOO.BAR.
           List<SqlParserPos> poses =
-              new ArrayList<SqlParserPos>(
+              new ArrayList<>(
                   Collections.nCopies(
                       qualifiedNames.size(), id.getParserPosition()));
           int offset = qualifiedNames.size() - id.names.size();

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 7c869c5..7228f63 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
@@ -79,18 +79,19 @@ public abstract class ListScope extends DelegatingScope {
     return Lists.transform(children, ScopeChild.NAME_FN);
   }
 
-  private int findChild(List<String> names) {
+  private ScopeChild findChild(List<String> names,
+      SqlNameMatcher nameMatcher) {
     for (ScopeChild child : children) {
       String lastName = Util.last(names);
       if (child.name != null) {
-        if (!validator.catalogReader.matches(child.name, lastName)) {
+        if (!nameMatcher.matches(child.name, lastName)) {
           // 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;
         }
         if (names.size() == 1) {
-          return child.ordinal;
+          return child;
         }
       }
 
@@ -98,15 +99,18 @@ public abstract class ListScope extends DelegatingScope {
       // catalog & schema and the other is not.
       final SqlValidatorTable table = child.namespace.getTable();
       if (table != null) {
-        final SqlValidatorTable table2 =
-            validator.catalogReader.getTable(names);
-        if (table2 != null
-            && table.getQualifiedName().equals(table2.getQualifiedName())) {
-          return child.ordinal;
+        final ResolvedImpl resolved = new ResolvedImpl();
+        resolveTable(names, nameMatcher, Path.EMPTY, resolved);
+        if (resolved.count() == 1
+            && resolved.only().remainingNames.isEmpty()
+            && resolved.only().namespace instanceof TableNamespace
+            && resolved.only().namespace.getTable().getQualifiedName().equals(
+                table.getQualifiedName())) {
+          return child;
         }
       }
     }
-    return -1;
+    return null;
   }
 
   public void findAllColumnNames(List<SqlMoniker> result) {
@@ -125,9 +129,12 @@ public abstract class ListScope extends DelegatingScope {
 
   @Override public Pair<String, SqlValidatorNamespace>
   findQualifyingTableName(final String columnName, SqlNode ctx) {
-    final Map<String, ScopeChild> map = findQualifyingTables(columnName);
+    final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
+    final Map<String, ScopeChild> map =
+        findQualifyingTables(columnName, nameMatcher);
     switch (map.size()) {
     case 0:
+      //noinspection deprecation
       return parent.findQualifyingTableName(columnName, ctx);
     case 1:
       final Map.Entry<String, ScopeChild> entry =
@@ -140,11 +147,25 @@ public abstract class ListScope extends DelegatingScope {
   }
 
   @Override public Map<String, ScopeChild>
-  findQualifyingTables(String columnName) {
+  findQualifyingTableNames(String columnName, SqlNode ctx,
+      SqlNameMatcher nameMatcher) {
+    final Map<String, ScopeChild> map =
+        findQualifyingTables(columnName, nameMatcher);
+    switch (map.size()) {
+    case 0:
+      return parent.findQualifyingTableNames(columnName, ctx, nameMatcher);
+    default:
+      return map;
+    }
+  }
+
+  @Override public Map<String, ScopeChild>
+  findQualifyingTables(String columnName, SqlNameMatcher nameMatcher) {
     final Map<String, ScopeChild> map = new HashMap<>();
     for (ScopeChild child : children) {
       final ResolvedImpl resolved = new ResolvedImpl();
-      resolve(ImmutableList.of(child.name, columnName), true, resolved);
+      resolve(ImmutableList.of(child.name, columnName), nameMatcher, true,
+          resolved);
       if (resolved.count() > 0) {
         map.put(child.name, child);
       }
@@ -152,15 +173,15 @@ public abstract class ListScope extends DelegatingScope {
     return map;
   }
 
-  @Override public void resolve(List<String> names, boolean deep,
-      Resolved resolved) {
+  @Override public void resolve(List<String> names, SqlNameMatcher nameMatcher,
+      boolean deep, Resolved resolved) {
     // First resolve by looking through the child namespaces.
-    final int i = findChild(names);
-    if (i >= 0) {
+    final ScopeChild child0 = findChild(names, nameMatcher);
+    if (child0 != null) {
       final Step path =
-          resolved.emptyPath().add(null, i, StructKind.FULLY_QUALIFIED);
-      final ScopeChild child = children.get(i);
-      resolved.found(child.namespace, child.nullable, this, path);
+          Path.EMPTY.plus(child0.namespace.getRowType(), child0.ordinal,
+              child0.name, StructKind.FULLY_QUALIFIED);
+      resolved.found(child0.namespace, child0.nullable, this, path, null);
       return;
     }
 
@@ -170,11 +191,11 @@ public abstract class ListScope extends DelegatingScope {
       for (ScopeChild child : children) {
         // If identifier starts with table alias, remove the alias.
         final List<String> names2 =
-            validator.catalogReader.matches(child.name, names.get(0))
+            nameMatcher.matches(child.name, names.get(0))
                 ? names.subList(1, names.size())
                 : names;
-        resolveInNamespace(child.namespace, child.nullable, names2,
-            resolved.emptyPath(), resolved);
+        resolveInNamespace(child.namespace, child.nullable, names2, nameMatcher,
+            Path.EMPTY, resolved);
       }
       if (resolved.count() > 0) {
         return;
@@ -183,17 +204,18 @@ public abstract class ListScope extends DelegatingScope {
 
     // Then call the base class method, which will delegate to the
     // parent scope.
-    super.resolve(names, deep, resolved);
+    super.resolve(names, nameMatcher, deep, resolved);
   }
 
   public RelDataType resolveColumn(String columnName, SqlNode ctx) {
+    final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
     int found = 0;
     RelDataType type = null;
     for (ScopeChild child : children) {
       SqlValidatorNamespace childNs = child.namespace;
       final RelDataType childRowType = childNs.getRowType();
       final RelDataTypeField field =
-          validator.catalogReader.field(childRowType, columnName);
+          nameMatcher.field(childRowType, columnName);
       if (field != null) {
         found++;
         type = field.getType();

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 65958fd..117390f 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
@@ -74,7 +74,8 @@ public class OrderByScope extends DelegatingScope {
           validator.getNamespace(select);
       final RelDataType rowType = selectNs.getRowType();
 
-      final RelDataTypeField field = validator.catalogReader.field(rowType, name);
+      final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
+      final RelDataTypeField field = nameMatcher.field(rowType, name);
       if (field != null && !field.isDynamicStar()) {
         // if identifier is resolved to a dynamic star, use super.fullyQualify() for such case.
         return SqlQualified.create(this, 1, selectNs, identifier);
@@ -86,7 +87,8 @@ public class OrderByScope extends DelegatingScope {
   public RelDataType resolveColumn(String name, SqlNode ctx) {
     final SqlValidatorNamespace selectNs = validator.getNamespace(select);
     final RelDataType rowType = selectNs.getRowType();
-    final RelDataTypeField field = validator.catalogReader.field(rowType, name);
+    final SqlNameMatcher nameMatcher = validator.catalogReader.nameMatcher();
+    final RelDataTypeField field = nameMatcher.field(rowType, name);
     if (field != null) {
       return field.getType();
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatcher.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatcher.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatcher.java
new file mode 100644
index 0000000..104f3d2
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatcher.java
@@ -0,0 +1,62 @@
+/*
+ * 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.RelDataTypeField;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Checks whether two names are the same according to a case-sensitivity policy.
+ *
+ * @see SqlNameMatchers
+ */
+public interface SqlNameMatcher {
+  /** Returns whether name matching is case-sensitive. */
+  boolean isCaseSensitive();
+
+  /** Returns a name matches another.
+   *
+   * @param string Name written in code
+   * @param name Name of object we are trying to match
+   * @return Whether matches
+   */
+  boolean matches(String string, String name);
+
+  /** Looks up an item in a map. */
+  <K extends List<String>, V> V get(Map<K, V> map, List<String> prefixNames,
+      List<String> names);
+
+  /** Returns the most recent match.
+   *
+   * <p>In the default implementation,
+   * throws {@link UnsupportedOperationException}. */
+  String bestString();
+
+  /** Finds a field with a given name, using the currenct case-sensitivity,
+   * returning null if not found.
+   *
+   * @param rowType    Row type
+   * @param fieldName Field name
+   * @return Field, or null if not found
+   */
+  RelDataTypeField field(RelDataType rowType, String fieldName);
+}
+
+// End SqlNameMatcher.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatchers.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatchers.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatchers.java
new file mode 100644
index 0000000..0ccc09f
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlNameMatchers.java
@@ -0,0 +1,154 @@
+/*
+ * 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.RelDataTypeField;
+import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableList;
+
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Helpers for {@link SqlNameMatcher}.
+ */
+public class SqlNameMatchers {
+
+  private static final BaseMatcher CASE_SENSITIVE = new BaseMatcher(true);
+  private static final BaseMatcher CASE_INSENSITIVE = new BaseMatcher(false);
+
+  private SqlNameMatchers() {}
+
+  /** Returns a name matcher with the given case sensitivity. */
+  public static SqlNameMatcher withCaseSensitive(final boolean caseSensitive) {
+    return caseSensitive ? CASE_SENSITIVE : CASE_INSENSITIVE;
+  }
+
+  /** Creates a name matcher that can suggest corrections to what the user
+   * typed. It matches liberally (case-insensitively) and also records the last
+   * match. */
+  public static SqlNameMatcher liberal() {
+    return new LiberalNameMatcher();
+  }
+
+  /** Partial implementation of {@link SqlNameMatcher}. */
+  private static class BaseMatcher implements SqlNameMatcher {
+    private final boolean caseSensitive;
+
+    BaseMatcher(boolean caseSensitive) {
+      this.caseSensitive = caseSensitive;
+    }
+
+    public boolean isCaseSensitive() {
+      return caseSensitive;
+    }
+
+    public boolean matches(String string, String name) {
+      return caseSensitive ? string.equals(name)
+          : string.equalsIgnoreCase(name);
+    }
+
+    protected boolean listMatches(List<String> list0, List<String> list1) {
+      if (list0.size() != list1.size()) {
+        return false;
+      }
+      for (int i = 0; i < list0.size(); i++) {
+        String s0 = list0.get(i);
+        String s1 = list1.get(i);
+        if (!matches(s0, s1)) {
+          return false;
+        }
+      }
+      return true;
+    }
+
+    public <K extends List<String>, V> V get(Map<K, V> map,
+        List<String> prefixNames, List<String> names) {
+      final List<String> key = concat(prefixNames, names);
+      if (caseSensitive) {
+        //noinspection SuspiciousMethodCalls
+        return map.get(key);
+      }
+      for (Map.Entry<K, V> entry : map.entrySet()) {
+        if (listMatches(key, entry.getKey())) {
+          matched(prefixNames, entry.getKey());
+          return entry.getValue();
+        }
+      }
+      return null;
+    }
+
+    private List<String> concat(List<String> prefixNames, List<String> names) {
+      if (prefixNames.isEmpty()) {
+        return names;
+      } else {
+        return ImmutableList.<String>builder().addAll(prefixNames).addAll(names)
+            .build();
+      }
+    }
+
+    protected void matched(List<String> prefixNames, List<String> names) {
+    }
+
+    protected List<String> bestMatch() {
+      throw new UnsupportedOperationException();
+    }
+
+    public String bestString() {
+      return SqlIdentifier.getString(bestMatch());
+    }
+
+    public RelDataTypeField field(RelDataType rowType, String fieldName) {
+      return rowType.getField(fieldName, caseSensitive, false);
+    }
+  }
+
+  /** Matcher that remembers the requests that were made of it. */
+  private static class LiberalNameMatcher extends BaseMatcher {
+    List<String> matchedNames;
+
+    LiberalNameMatcher() {
+      super(false);
+    }
+
+    @Override protected boolean listMatches(List<String> list0,
+        List<String> list1) {
+      final boolean b = super.listMatches(list0, list1);
+      if (b) {
+        matchedNames = ImmutableList.copyOf(list1);
+      }
+      return b;
+    }
+
+    @Override protected void matched(List<String> prefixNames,
+        List<String> names) {
+      matchedNames = ImmutableList.copyOf(
+          Util.startsWith(names, prefixNames)
+              ? Util.skip(names, prefixNames.size())
+              : names);
+    }
+
+    @Override public List<String> bestMatch() {
+      return matchedNames;
+    }
+  }
+}
+
+// End SqlNameMatchers.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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
index 55c1d01..52d4e1b 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlQualified.java
@@ -60,10 +60,13 @@ public class SqlQualified {
     if (scope == null) {
       return identifier.names;
     }
+    final SqlNameMatcher nameMatcher =
+        scope.getValidator().getCatalogReader().nameMatcher();
     final ImmutableList.Builder<String> builder = ImmutableList.builder();
     final SqlValidatorScope.ResolvedImpl resolved =
         new SqlValidatorScope.ResolvedImpl();
-    scope.resolve(Util.skipLast(identifier.names), false, resolved);
+    final List<String> prefix = Util.skipLast(identifier.names);
+    scope.resolve(prefix, nameMatcher, false, resolved);
     SqlValidatorNamespace namespace =
         resolved.count() == 1 ? resolved.only().namespace : null;
     builder.add(identifier.names.get(0));

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 b43f85c..a71f8f4 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
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.validate;
 
+import org.apache.calcite.jdbc.CalciteSchema;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlIdentifier;
@@ -35,10 +36,17 @@ public interface SqlValidatorCatalogReader {
   //~ Methods ----------------------------------------------------------------
 
   /**
-   * Finds a table with the given name, possibly qualified.
+   * Finds a table or schema with the given name, possibly qualified.
    *
-   * @param names Qualified name of table
-   * @return named table, or null if not found
+   * <p>Uses the case-sensitivity policy of the catalog reader.
+   *
+   * <p>If not found, returns null. If you want a more descriptive error
+   * message or to override the case-sensitivity of the match, use
+   * {@link SqlValidatorScope#resolveTable}.
+   *
+   * @param names Name of table, may be qualified or fully-qualified
+   *
+   * @return Table with the given name, or null
    */
   SqlValidatorTable getTable(List<String> names);
 
@@ -68,24 +76,37 @@ public interface SqlValidatorCatalogReader {
   List<SqlMoniker> getAllSchemaObjectNames(List<String> names);
 
   /**
-   * Returns the name of the current schema.
+   * Returns the paths of all schemas to look in for tables.
    *
-   * @return name of the current schema
+   * @return paths of current schema and root schema
    */
-  List<String> getSchemaName();
+  List<List<String>> getSchemaPaths();
 
-  /**
-   * Finds a field with a given name, using the case-sensitivity of the current
-   * session.
-   */
+  /** @deprecated Use
+   * {@link #nameMatcher()}.{@link SqlNameMatcher#field(RelDataType, String)} */
+  @Deprecated // to be removed before 2.0
   RelDataTypeField field(RelDataType rowType, String alias);
 
+  /** Returns an implementation of
+   * {@link org.apache.calcite.sql.validate.SqlNameMatcher}
+   * that matches the case-sensitivity policy. */
+  SqlNameMatcher nameMatcher();
+
+  /** @deprecated Use
+   * {@link #nameMatcher()}.{@link SqlNameMatcher#matches(String, String)} */
+  @Deprecated // to be removed before 2.0
   boolean matches(String string, String name);
 
   RelDataType createTypeFromProjection(RelDataType type,
       List<String> columnNameList);
 
+  /** @deprecated Use
+   * {@link #nameMatcher()}.{@link SqlNameMatcher#isCaseSensitive()} */
+  @Deprecated // to be removed before 2.0
   boolean isCaseSensitive();
+
+  /** Returns the root namespace for name resolution. */
+  CalciteSchema getRootSchema();
 }
 
 // End SqlValidatorCatalogReader.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5f9c0190/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 7b2645c..3766258 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
@@ -320,9 +320,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           select,
           unknownType,
           list,
-          catalogReader.isCaseSensitive()
-          ? new LinkedHashSet<String>()
-              : new TreeSet<String>(String.CASE_INSENSITIVE_ORDER),
+          catalogReader.nameMatcher().isCaseSensitive()
+              ? new LinkedHashSet<String>()
+              : new TreeSet<>(String.CASE_INSENSITIVE_ORDER),
           types,
           includeSystemVars);
     }
@@ -504,7 +504,9 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final SqlIdentifier prefixId = identifier.skipLast(1);
       final SqlValidatorScope.ResolvedImpl resolved =
           new SqlValidatorScope.ResolvedImpl();
-      scope.resolve(prefixId.names, true, resolved);
+      final SqlNameMatcher nameMatcher =
+          scope.validator.catalogReader.nameMatcher();
+      scope.resolve(prefixId.names, nameMatcher, true, resolved);
       if (resolved.count() == 0) {
         // e.g. "select s.t.* from e"
         // or "select r.* from e"
@@ -737,7 +739,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         if (ns == null) {
           final SqlValidatorScope.ResolvedImpl resolved =
               new SqlValidatorScope.ResolvedImpl();
-          scope.resolve(ImmutableList.of(name), false, resolved);
+          final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+          scope.resolve(ImmutableList.of(name), nameMatcher, false, resolved);
           if (resolved.count() == 1) {
             ns = resolved.only().namespace;
           }
@@ -971,9 +974,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final SqlValidatorScope parentScope =
           ((DelegatingScope) scope).getParent();
       if (id.isSimple()) {
+        final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
         final SqlValidatorScope.ResolvedImpl resolved =
             new SqlValidatorScope.ResolvedImpl();
-        parentScope.resolve(id.names, false, resolved);
+        parentScope.resolve(id.names, nameMatcher, false, resolved);
         if (resolved.count() == 1) {
           return resolved.only().namespace;
         }
@@ -2929,11 +2933,12 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
               rightRowType);
 
       // Check compatibility of the chosen columns.
+      final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
       for (String name : naturalColumnNames) {
         final RelDataType leftColType =
-            catalogReader.field(leftRowType, name).getType();
+            nameMatcher.field(leftRowType, name).getType();
         final RelDataType rightColType =
-            catalogReader.field(rightRowType, name).getType();
+            nameMatcher.field(rightRowType, name).getType();
         if (!SqlTypeUtil.isComparable(leftColType, rightColType)) {
           throw newValidationError(join,
               RESOURCE.naturalOrUsingColumnNotCompatible(name,
@@ -2994,7 +2999,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       String name = id.names.get(0);
       final SqlValidatorNamespace namespace = getNamespace(leftOrRight);
       final RelDataType rowType = namespace.getRowType();
-      final RelDataTypeField field = catalogReader.field(rowType, name);
+      final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+      final RelDataTypeField field = nameMatcher.field(rowType, name);
       if (field != null) {
         if (Collections.frequency(rowType.getFieldNames(), name) > 1) {
           throw newValidationError(id,
@@ -3053,7 +3059,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     // Make sure that items in FROM clause have distinct aliases.
     final SelectScope fromScope = (SelectScope) getFromScope(select);
     List<String> names = fromScope.getChildNames();
-    if (!catalogReader.isCaseSensitive()) {
+    if (!catalogReader.nameMatcher().isCaseSensitive()) {
       names = Lists.transform(names,
           new Function<String, String>() {
             public String apply(String s) {
@@ -3325,24 +3331,24 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
   public void validateSequenceValue(SqlValidatorScope scope, SqlIdentifier id) {
     // Resolve identifier as a table.
-    final SqlValidatorNamespace ns = scope.getTableNamespace(id.names);
-    if (ns == null) {
+    final SqlValidatorScope.ResolvedImpl resolved =
+        new SqlValidatorScope.ResolvedImpl();
+    scope.resolveTable(id.names, catalogReader.nameMatcher(),
+        SqlValidatorScope.Path.EMPTY, resolved);
+    if (resolved.count() != 1) {
       throw newValidationError(id, RESOURCE.tableNameNotFound(id.toString()));
     }
-
     // We've found a table. But is it a sequence?
-    if (!(ns instanceof TableNamespace)) {
-      throw newValidationError(id, RESOURCE.notASequence(id.toString()));
-    }
-    final SqlValidatorTable table = ns.getTable();
-    final Table table1 = ((RelOptTable) table).unwrap(Table.class);
-    switch (table1.getJdbcTableType()) {
-    case SEQUENCE:
-    case TEMPORARY_SEQUENCE:
-      break;
-    default:
-      throw newValidationError(id, RESOURCE.notASequence(id.toString()));
+    final SqlValidatorNamespace ns = resolved.only().namespace;
+    if (ns instanceof TableNamespace) {
+      final Table table = ((RelOptTable) ns.getTable()).unwrap(Table.class);
+      switch (table.getJdbcTableType()) {
+      case SEQUENCE:
+      case TEMPORARY_SEQUENCE:
+        return;
+      }
     }
+    throw newValidationError(id, RESOURCE.notASequence(id.toString()));
   }
 
   public SqlValidatorScope getWithScope(SqlNode withItem) {
@@ -4139,7 +4145,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   SqlValidatorNamespace lookupFieldNamespace(RelDataType rowType, String name) {
-    final RelDataTypeField field = catalogReader.field(rowType, name);
+    final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+    final RelDataTypeField field = nameMatcher.field(rowType, name);
     return new FieldNamespace(this, field.getType());
   }
 
@@ -4480,9 +4487,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         // we could do a better job if they were looked up via
         // resolveColumn.
 
+        final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
         final SqlValidatorScope.ResolvedImpl resolved =
             new SqlValidatorScope.ResolvedImpl();
-        scope.resolve(id.names.subList(0, i), false, resolved);
+        scope.resolve(id.names.subList(0, i), nameMatcher, false, resolved);
         if (resolved.count() == 1) {
           // There's a namespace with the name we seek.
           final SqlValidatorScope.Resolve resolve = resolved.only();
@@ -4522,7 +4530,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
           name = "*";
           field = null;
         } else {
-          field = catalogReader.field(type, name);
+          final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+          field = nameMatcher.field(type, name);
         }
         if (field == null) {
           throw newValidationError(id.getComponent(i),
@@ -4696,8 +4705,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         final SqlValidatorNamespace selectNs = getNamespace(select);
         final RelDataType rowType =
             selectNs.getRowTypeSansSystemColumns();
-        RelDataTypeField field =
-            catalogReader.field(rowType, alias);
+        final SqlNameMatcher nameMatcher = catalogReader.nameMatcher();
+        RelDataTypeField field = nameMatcher.field(rowType, alias);
         if (field != null) {
           return nthSelectItem(
               field.getIndex(),