You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ma...@apache.org on 2016/10/14 22:41:26 UTC

calcite git commit: [CALCITE-1431] RelDataTypeFactoryImpl.copyType() did not copy StructKind

Repository: calcite
Updated Branches:
  refs/heads/master acba7e382 -> 3b4e17142


[CALCITE-1431] RelDataTypeFactoryImpl.copyType() did not copy StructKind


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

Branch: refs/heads/master
Commit: 3b4e17142c98b78eabb611dcdc3e0094a957c9eb
Parents: acba7e3
Author: maryannxue <ma...@gmail.com>
Authored: Fri Oct 14 15:41:18 2016 -0700
Committer: maryannxue <ma...@gmail.com>
Committed: Fri Oct 14 15:41:18 2016 -0700

----------------------------------------------------------------------
 .../rel/type/RelDataTypeFactoryImpl.java        |  21 +-
 .../apache/calcite/test/MockCatalogReader.java  | 262 ++++++++++++-------
 .../apache/calcite/test/SqlValidatorTest.java   | 102 ++++----
 3 files changed, 226 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3b4e1714/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
index 774a728..d739dd9 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
@@ -274,17 +274,9 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
     // For flattening and outer joins, it is desirable to change
     // the nullability of the individual fields.
 
-    return createStructType(
-        new FieldInfo() {
-          public int getFieldCount() {
-            return type.getFieldList().size();
-          }
-
-          public String getFieldName(int index) {
-            return type.getFieldList().get(index).getName();
-          }
-
-          public RelDataType getFieldType(int index) {
+    return createStructType(type.getStructKind(),
+        new AbstractList<RelDataType>() {
+          @Override public RelDataType get(int index) {
             RelDataType fieldType =
                 type.getFieldList().get(index).getType();
             if (ignoreNullable) {
@@ -293,7 +285,12 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
               return createTypeWithNullability(fieldType, nullable);
             }
           }
-        });
+
+          @Override public int size() {
+            return type.getFieldCount();
+          }
+        },
+        type.getFieldNames());
   }
 
   // implement RelDataTypeFactory

http://git-wip-us.apache.org/repos/asf/calcite/blob/3b4e1714/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 23719d4..3912880 100644
--- a/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
+++ b/core/src/test/java/org/apache/calcite/test/MockCatalogReader.java
@@ -39,7 +39,9 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.rel.type.RelDataTypeImpl;
 import org.apache.calcite.rel.type.RelDataTypePrecedenceList;
+import org.apache.calcite.rel.type.RelProtoDataType;
 import org.apache.calcite.rel.type.RelRecordType;
 import org.apache.calcite.rel.type.StructKind;
 import org.apache.calcite.rex.RexBuilder;
@@ -348,104 +350,33 @@ public class MockCatalogReader implements Prepare.CatalogReader {
     // Register "EMP_20" view.
     // Same columns as "EMP",
     // but "DEPTNO" not visible and set to 20 by default
-    // and "SAL" is visible but must be greater than 1000
-    MockTable emp20View = new MockTable(this, salesSchema.getCatalogName(),
-        salesSchema.name, "EMP_20", false, 600) {
-      private final Table table = empTable.unwrap(Table.class);
-      private final ImmutableIntList mapping =
-          ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8);
-
-      @Override public RelNode toRel(ToRelContext context) {
-        // Expand to the equivalent of:
-        //   SELECT EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, SLACKER
-        //   FROM EMP
-        //   WHERE DEPTNO = 20 AND SAL > 1000
-        RelNode rel = LogicalTableScan.create(context.getCluster(), empTable);
-        final RexBuilder rexBuilder = context.getCluster().getRexBuilder();
-        rel = LogicalFilter.create(rel,
-            rexBuilder.makeCall(
-                SqlStdOperatorTable.AND,
-                rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                    rexBuilder.makeInputRef(rel, 7),
-                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(20))),
-                rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
-                    rexBuilder.makeInputRef(rel, 5),
-                    rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000)))));
-        final List<RelDataTypeField> fieldList =
-            rel.getRowType().getFieldList();
-        final List<Pair<RexNode, String>> projects =
-            new AbstractList<Pair<RexNode, String>>() {
-              @Override public Pair<RexNode, String> get(int index) {
-                return RexInputRef.of2(mapping.get(index), fieldList);
-              }
-
-              @Override public int size() {
-                return mapping.size();
-              }
-            };
-        return LogicalProject.create(rel, Pair.left(projects),
-            Pair.right(projects));
-      }
-
-      @Override public <T> T unwrap(Class<T> clazz) {
-        if (clazz.isAssignableFrom(ModifiableView.class)) {
-          return clazz.cast(
-              new JdbcTest.AbstractModifiableView() {
-                @Override public Table getTable() {
-                  return empTable.unwrap(Table.class);
-                }
-
-                @Override public Path getTablePath() {
-                  final ImmutableList.Builder<Pair<String, Schema>> builder =
-                      ImmutableList.builder();
-                  builder.add(Pair.<String, Schema>of(empTable.names.get(0), null));
-                  builder.add(Pair.<String, Schema>of(empTable.names.get(1), null));
-                  builder.add(Pair.<String, Schema>of(empTable.names.get(2), null));
-                  return Schemas.path(builder.build());
-                }
-
-                @Override public ImmutableIntList getColumnMapping() {
-                  return mapping;
-                }
-
-                @Override public RexNode getConstraint(RexBuilder rexBuilder,
-                    RelDataType tableRowType) {
-                  final RelDataTypeField deptnoField =
-                      tableRowType.getFieldList().get(7);
-                  final RelDataTypeField salField =
-                      tableRowType.getFieldList().get(5);
-                  final List<RexNode> nodes = Arrays.asList(
-                      rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
-                          rexBuilder.makeInputRef(deptnoField.getType(),
-                              deptnoField.getIndex()),
-                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(20L),
-                              deptnoField.getType())),
-                      rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
-                          rexBuilder.makeInputRef(salField.getType(),
-                              salField.getIndex()),
-                          rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000L),
-                              salField.getType())));
-                  return RexUtil.composeConjunction(rexBuilder, nodes, false);
-                }
-
-                @Override public RelDataType
-                getRowType(final RelDataTypeFactory typeFactory) {
-                  return typeFactory.createStructType(
-                      new AbstractList<Map.Entry<String, RelDataType>>() {
-                        @Override public Map.Entry<String, RelDataType>
-                        get(int index) {
-                          return table.getRowType(typeFactory).getFieldList()
-                              .get(mapping.get(index));
-                        }
-
-                        @Override public int size() {
-                          return mapping.size();
-                        }
-                      });
-                }
-              });
-        }
-        return super.unwrap(clazz);
+    // and "SAL" is visible but must be greater than 1000,
+    // which is the equivalent of:
+    //   SELECT EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, SLACKER
+    //   FROM EMP
+    //   WHERE DEPTNO = 20 AND SAL > 1000
+    MockTable emp20View = new MockViewTable(this, salesSchema.getCatalogName(),
+        salesSchema.name, "EMP_20", false, 600, empTable,
+        ImmutableIntList.of(0, 1, 2, 3, 4, 5, 6, 8)) {
+
+      @Override public RexNode getConstraint(RexBuilder rexBuilder,
+          RelDataType tableRowType) {
+        final RelDataTypeField deptnoField =
+            tableRowType.getFieldList().get(7);
+        final RelDataTypeField salField =
+            tableRowType.getFieldList().get(5);
+        final List<RexNode> nodes = Arrays.asList(
+            rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+                rexBuilder.makeInputRef(deptnoField.getType(),
+                    deptnoField.getIndex()),
+                rexBuilder.makeExactLiteral(BigDecimal.valueOf(20L),
+                    deptnoField.getType())),
+            rexBuilder.makeCall(SqlStdOperatorTable.GREATER_THAN,
+                rexBuilder.makeInputRef(salField.getType(),
+                    salField.getIndex()),
+                rexBuilder.makeExactLiteral(BigDecimal.valueOf(1000L),
+                    salField.getType())));
+        return RexUtil.composeConjunction(rexBuilder, nodes, false);
       }
     };
     salesSchema.addTable(Util.last(emp20View.getQualifiedName()));
@@ -461,8 +392,8 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
     MockSchema structTypeSchema = new MockSchema("STRUCT");
     registerSchema(structTypeSchema);
-    MockTable structTypeTable = MockTable.create(this, structTypeSchema, "T",
-        false, 100);
+    final MockTable structTypeTable = MockTable.create(this, structTypeSchema,
+        "T", false, 100);
     structTypeTable.addColumn("K0", varchar20Type);
     structTypeTable.addColumn("C1", varchar20Type);
     final RelDataType f0Type = typeFactory.builder()
@@ -485,6 +416,41 @@ public class MockCatalogReader implements Prepare.CatalogReader {
         .build();
     structTypeTable.addColumn("F2", f2Type);
     registerTable(structTypeTable);
+
+    // Register "STRUCT.T_10" view.
+    // Same columns as "STRUCT.T",
+    // but "F0.C0" is set to 10 by default,
+    // which is the equivalent of:
+    //   SELECT K0, C1, F0, F1, F2
+    //   FROM T
+    //   WHERE F0.C0 = 10
+    MockTable struct10View = new MockViewTable(this,
+        structTypeSchema.getCatalogName(),
+        structTypeSchema.name, "T_10", false, 20,
+        structTypeTable, ImmutableIntList.of(0, 1, 2, 3, 4)) {
+
+      @Override public RexNode getConstraint(RexBuilder rexBuilder,
+          RelDataType tableRowType) {
+        final RelDataTypeField f0Field =
+            tableRowType.getFieldList().get(2);
+        final RelDataTypeField c0Field =
+            f0Field.getType().getFieldList().get(0);
+        return rexBuilder.makeCall(SqlStdOperatorTable.EQUALS,
+            rexBuilder.makeFieldAccess(
+                rexBuilder.makeInputRef(f0Field.getType(),
+                    f0Field.getIndex()),
+                c0Field.getIndex()),
+            rexBuilder.makeExactLiteral(BigDecimal.valueOf(10L),
+                c0Field.getType()));
+      }
+    };
+    structTypeSchema.addTable(Util.last(struct10View.getQualifiedName()));
+    struct10View.addColumn("K0", varchar20Type);
+    struct10View.addColumn("C1", varchar20Type);
+    struct10View.addColumn("F0", f0Type);
+    struct10View.addColumn("F1", f1Type);
+    struct10View.addColumn("F2", f2Type);
+    registerTable(struct10View);
     return this;
   }
 
@@ -808,6 +774,102 @@ public class MockCatalogReader implements Prepare.CatalogReader {
 
   /**
    * Mock implementation of
+   * {@link org.apache.calcite.prepare.Prepare.PreparingTable} for views.
+   */
+  public abstract static class MockViewTable extends MockTable {
+    private final MockTable fromTable;
+    private final Table table;
+    private final ImmutableIntList mapping;
+
+    MockViewTable(MockCatalogReader catalogReader, String catalogName,
+        String schemaName, String name, boolean stream, double rowCount,
+        MockTable fromTable, ImmutableIntList mapping) {
+      super(catalogReader, catalogName, schemaName, name, stream, rowCount);
+      this.fromTable = fromTable;
+      this.table = fromTable.unwrap(Table.class);
+      this.mapping = mapping;
+    }
+
+    protected abstract RexNode getConstraint(RexBuilder rexBuilder,
+        RelDataType tableRowType);
+
+    @Override public void onRegister(RelDataTypeFactory typeFactory) {
+      super.onRegister(typeFactory);
+      // To simulate getRowType() behavior in ViewTable.
+      final RelProtoDataType protoRowType = RelDataTypeImpl.proto(rowType);
+      rowType = protoRowType.apply(typeFactory);
+    }
+
+    @Override public RelNode toRel(ToRelContext context) {
+      RelNode rel = LogicalTableScan.create(context.getCluster(), fromTable);
+      final RexBuilder rexBuilder = context.getCluster().getRexBuilder();
+      rel = LogicalFilter.create(
+          rel, getConstraint(rexBuilder, rel.getRowType()));
+      final List<RelDataTypeField> fieldList =
+          rel.getRowType().getFieldList();
+      final List<Pair<RexNode, String>> projects =
+          new AbstractList<Pair<RexNode, String>>() {
+            @Override public Pair<RexNode, String> get(int index) {
+              return RexInputRef.of2(mapping.get(index), fieldList);
+            }
+
+            @Override public int size() {
+              return mapping.size();
+            }
+          };
+      return LogicalProject.create(rel, Pair.left(projects),
+          Pair.right(projects));
+    }
+
+    @Override public <T> T unwrap(Class<T> clazz) {
+      if (clazz.isAssignableFrom(ModifiableView.class)) {
+        return clazz.cast(
+            new JdbcTest.AbstractModifiableView() {
+              @Override public Table getTable() {
+                return fromTable.unwrap(Table.class);
+              }
+
+              @Override public Path getTablePath() {
+                final ImmutableList.Builder<Pair<String, Schema>> builder =
+                    ImmutableList.builder();
+                for (String name : fromTable.names) {
+                  builder.add(Pair.<String, Schema>of(name, null));
+                }
+                return Schemas.path(builder.build());
+              }
+
+              @Override public ImmutableIntList getColumnMapping() {
+                return mapping;
+              }
+
+              @Override public RexNode getConstraint(RexBuilder rexBuilder,
+                  RelDataType tableRowType) {
+                return MockViewTable.this.getConstraint(rexBuilder, tableRowType);
+              }
+
+              @Override public RelDataType
+              getRowType(final RelDataTypeFactory typeFactory) {
+                return typeFactory.createStructType(
+                    new AbstractList<Map.Entry<String, RelDataType>>() {
+                      @Override public Map.Entry<String, RelDataType>
+                      get(int index) {
+                        return table.getRowType(typeFactory).getFieldList()
+                            .get(mapping.get(index));
+                      }
+
+                      @Override public int size() {
+                        return mapping.size();
+                      }
+                    });
+              }
+            });
+      }
+      return super.unwrap(clazz);
+    }
+  }
+
+  /**
+   * Mock implementation of
    * {@link org.apache.calcite.prepare.Prepare.PreparingTable} with dynamic record type.
    */
   public static class MockDynamicTable extends MockTable {

http://git-wip-us.apache.org/repos/asf/calcite/blob/3b4e1714/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 7f77f7d..fc3010b 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -8166,6 +8166,14 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
   }
 
   @Test public void testStructType() {
+    checkStructType("T");
+  }
+
+  @Test public void testStructTypeWithView() {
+    checkStructType("T_10");
+  }
+
+  private void checkStructType(String table) {
     // Table STRUCT.T is defined as: (
     //   K0 VARCHAR(20) NOT NULL,
     //   C1 VARCHAR(20) NOT NULL,
@@ -8184,120 +8192,120 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // F2 can all be transparent. F0 has default struct priority; F1 and F2 have
     // lower priority.
 
-    sql("select * from struct.t").ok();
+    sql("select * from struct." + table).ok();
 
     // Resolve K0 as top-level column K0.
-    sql("select k0 from struct.t")
+    sql("select k0 from struct." + table)
         .type("RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
 
     // Resolve C2 as secondary-level column F1.C2.
-    sql("select c2 from struct.t")
+    sql("select c2 from struct." + table)
         .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve F1.C2 as fully qualified column F1.C2.
-    sql("select f1.c2 from struct.t")
+    sql("select f1.c2 from struct." + table)
         .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve C1 as top-level column C1 as opposed to F0.C1.
-    sql("select c1 from struct.t")
+    sql("select c1 from struct." + table)
         .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
 
     // Resolve C0 as secondary-level column F0.C0 as opposed to F1.C0, since F0
     // has the default priority.
-    sql("select c0 from struct.t")
+    sql("select c0 from struct." + table)
         .type("RecordType(INTEGER NOT NULL C0) NOT NULL");
 
     // Resolve F1.C0 as fully qualified column F1.C0 (as evidenced by "INTEGER"
     // rather than "INTEGER NOT NULL")
-    sql("select f1.c0 from struct.t")
+    sql("select f1.c0 from struct." + table)
         .type("RecordType(INTEGER C0) NOT NULL");
 
     // Fail ambiguous column reference A0, since F1.A0 and F2.A0 both exist with
     // the same resolving priority.
-    sql("select ^a0^ from struct.t")
+    sql("select ^a0^ from struct." + table)
         .fails("Column 'A0' is ambiguous");
 
     // Resolve F2.A0 as fully qualified column F2.A0.
-    sql("select f2.a0 from struct.t")
+    sql("select f2.a0 from struct." + table)
         .type("RecordType(BOOLEAN NOT NULL A0) NOT NULL");
 
     // Resolve T0.K0 as top-level column K0, since T0 is recognized as the table
     // alias.
-    sql("select t0.k0 from struct.t t0")
+    sql("select t0.k0 from struct." + table + " t0")
         .type("RecordType(VARCHAR(20) NOT NULL K0) NOT NULL");
 
     // Resolve T0.C2 as secondary-level column F1.C2, since T0 is recognized as
     // the table alias here.
-    sql("select t0.c2 from struct.t t0")
+    sql("select t0.c2 from struct." + table + " t0")
         .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve F0.C2 as secondary-level column F1.C2, since F0 is recognized as
     // the table alias here.
-    sql("select f0.c2 from struct.t f0")
+    sql("select f0.c2 from struct." + table + " f0")
         .type("RecordType(INTEGER NOT NULL C2) NOT NULL");
 
     // Resolve F0.C1 as top-level column C1 as opposed to F0.C1, since F0 is
     // recognized as the table alias here.
-    sql("select f0.c1 from struct.t f0")
+    sql("select f0.c1 from struct." + table + " f0")
         .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
 
     // Resolve C1 as inner INTEGER column not top-level VARCHAR column.
-    sql("select f0.f0.c1 from struct.t f0")
+    sql("select f0.f0.c1 from struct." + table + " f0")
         .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
-    // Resolve T.C1 as top-level column C1 as opposed to F0.C1, since T is
+    // Resolve <table>.C1 as top-level column C1 as opposed to F0.C1, since <table> is
     // recognized as the table name.
-    sql("select t.c1 from struct.t")
+    sql("select " + table + ".c1 from struct." + table)
         .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
 
-    // Alias "f0" obscures table name "t"
-    sql("select ^t^.c1 from struct.t f0")
-        .fails("Table 'T' not found");
+    // Alias "f0" obscures table name "<table>"
+    sql("select ^" + table + "^.c1 from struct." + table + " f0")
+        .fails("Table '" + table + "' not found");
 
-    // Resolve STRUCT.T.C1 as top-level column C1 as opposed to F0.C1, since
-    // STRUCT.T is recognized as the schema and table name.
-    sql("select struct.t.c1 from struct.t")
+    // Resolve STRUCT.<table>.C1 as top-level column C1 as opposed to F0.C1, since
+    // STRUCT.<table> is recognized as the schema and table name.
+    sql("select struct." + table + ".c1 from struct." + table)
         .type("RecordType(VARCHAR(20) NOT NULL C1) NOT NULL");
 
-    // Table alias "f0" obscures table name "struct.t"
-    sql("select ^struct.t^.c1 from struct.t f0")
-        .fails("Table 'STRUCT.T' not found");
+    // Table alias "f0" obscures table name "STRUCT.<table>"
+    sql("select ^struct." + table + "^.c1 from struct." + table + " f0")
+        .fails("Table 'STRUCT." + table + "' not found");
 
     // Resolve F0.F0.C1 as secondary-level column F0.C1, since the first F0 is
     // recognized as the table alias here.
-    sql("select f0.f0.c1 from struct.t f0")
+    sql("select f0.f0.c1 from struct." + table + " f0")
         .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
-    // Resolve T.F0.C1 as secondary-level column F0.C1, since T is recognized as
-    // the table name.
-    sql("select t.f0.c1 from struct.t")
+    // Resolve <table>.F0.C1 as secondary-level column F0.C1, since <table> is
+    // recognized as the table name.
+    sql("select " + table + ".f0.c1 from struct." + table)
         .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
     // Table alias obscures
-    sql("select ^t.f0^.c1 from struct.t f0")
-        .fails("Table 'T.F0' not found");
+    sql("select ^" + table + ".f0^.c1 from struct." + table + " f0")
+        .fails("Table '" + table + ".F0' not found");
 
-    // Resolve STRUCT.T.F0.C1 as secondary-level column F0.C1, since STRUCT.T is
-    // recognized as the schema and table name.
-    sql("select struct.t.f0.c1 from struct.t")
+    // Resolve STRUCT.<table>.F0.C1 as secondary-level column F0.C1, since
+    // STRUCT.<table> is recognized as the schema and table name.
+    sql("select struct." + table + ".f0.c1 from struct." + table)
         .type("RecordType(INTEGER NOT NULL C1) NOT NULL");
 
-    // Table alias "f0" obscures table name "struct.t"
-    sql("select ^struct.t.f0^.c1 from struct.t f0")
-        .fails("Table 'STRUCT.T.F0' not found");
+    // Table alias "f0" obscures table name "STRUCT.<table>"
+    sql("select ^struct." + table + ".f0^.c1 from struct." + table + " f0")
+        .fails("Table 'STRUCT." + table + ".F0' not found");
 
     // Resolve struct type F1 with wildcard.
-    sql("select f1.* from struct.t")
+    sql("select f1.* from struct." + table)
         .type("RecordType(INTEGER C0, INTEGER NOT NULL C2,"
             + " INTEGER NOT NULL A0) NOT NULL");
 
     // Resolve struct type F1 with wildcard.
-    sql("select t.f1.* from struct.t")
+    sql("select " + table + ".f1.* from struct." + table)
         .type("RecordType(INTEGER C0, INTEGER NOT NULL C2,"
             + " INTEGER NOT NULL A0) NOT NULL");
 
     // Fail non-existent column B0.
-    sql("select ^b0^ from struct.t")
+    sql("select ^b0^ from struct." + table)
         .fails("Column 'B0' not found in any table");
 
     // It's OK to reference a record type.
@@ -8305,18 +8313,18 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     // This is admittedly a bit strange for Phoenix users. We model a column
     // family as a column whose type is a record, but Phoenix users would
     // rarely if ever want to use a column family as a record.
-    sql("select f1 from struct.t")
+    sql("select f1 from struct." + table)
         .type("RecordType(RecordType:peek(INTEGER C0, INTEGER NOT NULL C2,"
             + " INTEGER NOT NULL A0) NOT NULL F1) NOT NULL");
 
     // If we fail to find a column, give an error based on the shortest prefix
     // that fails.
-    sql("select t.^f0.notFound^.a.b.c.d from struct.t")
-        .fails("Column 'F0\\.NOTFOUND' not found in table 'T'");
-    sql("select t.^f0.notFound^ from struct.t")
-        .fails("Column 'F0\\.NOTFOUND' not found in table 'T'");
-    sql("select t.^f0.c1.notFound^ from struct.t")
-        .fails("Column 'F0\\.C1\\.NOTFOUND' not found in table 'T'");
+    sql("select " + table + ".^f0.notFound^.a.b.c.d from struct." + table)
+        .fails("Column 'F0\\.NOTFOUND' not found in table '" + table + "'");
+    sql("select " + table + ".^f0.notFound^ from struct." + table)
+        .fails("Column 'F0\\.NOTFOUND' not found in table '" + table + "'");
+    sql("select " + table + ".^f0.c1.notFound^ from struct." + table)
+        .fails("Column 'F0\\.C1\\.NOTFOUND' not found in table '" + table + "'");
   }
 
   /** Test case for