You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ma...@apache.org on 2015/04/27 19:14:45 UTC

phoenix git commit: Implement PhoenixSchema; Add testJoinHsqldb

Repository: phoenix
Updated Branches:
  refs/heads/calcite 086462884 -> d555ee62f


Implement PhoenixSchema; Add testJoinHsqldb


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

Branch: refs/heads/calcite
Commit: d555ee62f6c4b308acf4c8bc86bb6a5b3542dd14
Parents: 0864628
Author: maryannxue <we...@intel.com>
Authored: Mon Apr 27 13:14:34 2015 -0400
Committer: maryannxue <we...@intel.com>
Committed: Mon Apr 27 13:14:34 2015 -0400

----------------------------------------------------------------------
 phoenix-core/pom.xml                            |  8 ++
 .../org/apache/phoenix/calcite/CalciteTest.java | 92 +++++++++++++++-----
 .../apache/phoenix/calcite/PhoenixSchema.java   | 33 +++++--
 .../apache/phoenix/calcite/PhoenixTable.java    |  2 +-
 .../java/org/apache/phoenix/util/TestUtil.java  | 16 ++--
 pom.xml                                         | 12 +++
 6 files changed, 124 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/phoenix-core/pom.xml
----------------------------------------------------------------------
diff --git a/phoenix-core/pom.xml b/phoenix-core/pom.xml
index 89adf1f..3fe2c7a 100644
--- a/phoenix-core/pom.xml
+++ b/phoenix-core/pom.xml
@@ -430,5 +430,13 @@
         <artifactId>joni</artifactId>
         <version>${joni.version}</version>
     </dependency>
+    <dependency>
+      <groupId>net.hydromatic</groupId>
+      <artifactId>foodmart-data-hsqldb</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.hsqldb</groupId>
+      <artifactId>hsqldb</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
index 67c764c..f7c8b21 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/calcite/CalciteTest.java
@@ -2,7 +2,9 @@ package org.apache.phoenix.calcite;
 
 import com.google.common.collect.Lists;
 
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
 import org.apache.calcite.jdbc.CalciteConnection;
+import org.apache.calcite.schema.SchemaPlus;
 import org.apache.phoenix.end2end.BaseClientManagedTimeIT;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.junit.Before;
@@ -14,6 +16,8 @@ import java.io.PrintWriter;
 import java.sql.*;
 import java.util.List;
 
+import javax.sql.DataSource;
+
 import static org.apache.phoenix.util.TestUtil.JOIN_ITEM_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_ORDER_TABLE_FULL_NAME;
 import static org.apache.phoenix.util.TestUtil.JOIN_SUPPLIER_TABLE_FULL_NAME;
@@ -151,10 +155,23 @@ public class CalciteTest extends BaseClientManagedTimeIT {
         final PhoenixConnection phoenixConnection =
             DriverManager.getConnection(url).unwrap(PhoenixConnection.class);
         calciteConnection.getRootSchema().add("phoenix",
-            new PhoenixSchema(phoenixConnection));
+            new PhoenixSchema(null, phoenixConnection));
         calciteConnection.setSchema("phoenix");
         return connection;
     }
+    
+    private static Connection createConnectionWithHsqldb() throws SQLException {
+        final Connection connection = createConnection();
+        final CalciteConnection calciteConnection =
+                connection.unwrap(CalciteConnection.class);
+        DataSource dataSource =
+                JdbcSchema.dataSource("jdbc:hsqldb:res:foodmart", "org.hsqldb.jdbcDriver", "FOODMART", "FOODMART");
+        SchemaPlus rootSchema = calciteConnection.getRootSchema();
+        rootSchema.add("foodmart",
+                JdbcSchema.create(rootSchema, "foodmart", dataSource, null,
+                    "foodmart"));
+        return connection;
+    }
 
     private static Connection connectUsingModel() throws Exception {
         final File file = File.createTempFile("model", ".json");
@@ -237,9 +254,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "  PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
                            "    PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
                            "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
-                           "        PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "        PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "        PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"0000000001", "T1", "0000000001", "S1"}, 
                           {"0000000002", "T2", "0000000001", "S1"}, 
@@ -253,9 +270,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                 .explainIs("PhoenixToEnumerableConverter\n" +
                            "  PhoenixServerProject(item_id=[$0], NAME=[$1], PRICE=[$2], DISCOUNT1=[$3], DISCOUNT2=[$4], supplier_id=[$5], DESCRIPTION=[$6], supplier_id0=[$7], NAME0=[$8], PHONE=[$9], ADDRESS=[$10], LOC_ID=[$11])\n" +
                            "    PhoenixServerJoin(condition=[=($5, $7)], joinType=[inner])\n" +
-                           "      PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "      PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "      PhoenixServerProject(supplier_id=[$0], NAME=[$1], PHONE=[$2], ADDRESS=[$3], LOC_ID=[$4], $f5=[CAST($1):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL])\n" +
-                           "        PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]], filter=[=(CAST($1):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL, 'S5')])\n")
+                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]], filter=[=(CAST($1):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\" NOT NULL, 'S5')])\n")
                 .resultIs(new Object[][] {
                           {"0000000005", "T5", 500, 8, 15, "0000000005", "Item T5", "0000000005", "S5", "888-888-5555", "505 YYY Street", "10005"}})
                 .close();
@@ -268,9 +285,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "    PhoenixClientJoin(condition=[=($2, $3)], joinType=[full])\n" +
                            "      PhoenixServerSort(sort0=[$2], dir0=[ASC])\n" +
                            "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "      PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "        PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "        PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .close();        
     }
     
@@ -342,9 +359,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "    PhoenixServerProject(NAME=[$2])\n" +
                            "      PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
                            "        PhoenixServerProject(supplier_id=[$5])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"S1", 2L},
                           {"S2", 2L},
@@ -420,9 +437,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "      PhoenixServerProject(NAME=[$2])\n" +
                            "        PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
                            "          PhoenixServerProject(supplier_id=[$5])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "          PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "            PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"S6", 1L},
                           {"S5", 1L},
@@ -436,9 +453,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "    PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
                            "      PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
                            "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"0000000006", "T6", "0000000006", "S6"}, 
                           {"0000000005", "T5", "0000000005", "S5"}, 
@@ -498,9 +515,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "        PhoenixServerProject(NAME=[$2])\n" +
                            "          PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
                            "            PhoenixServerProject(supplier_id=[$5])\n" +
-                           "              PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "              PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "            PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "              PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "              PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"S6", 1L},
                           {"S5", 1L},
@@ -514,9 +531,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "      PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$3], NAME0=[$4])\n" +
                            "        PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
                            "          PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "          PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "            PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"0000000006", "T6", "0000000006", "S6"}, 
                           {"0000000005", "T5", "0000000005", "S5"}, 
@@ -557,9 +574,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "      PhoenixServerProject(NAME=[$2])\n" +
                            "        PhoenixServerJoin(condition=[=($1, $0)], joinType=[inner])\n" +
                            "          PhoenixServerProject(supplier_id=[$5])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "            PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "          PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "            PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "            PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"S1", 2L},
                           {"S2", 2L},
@@ -572,9 +589,9 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                            "    PhoenixLimit(fetch=[3])\n" +
                            "      PhoenixServerJoin(condition=[=($2, $3)], joinType=[inner])\n" +
                            "        PhoenixServerProject(item_id=[$0], NAME=[$1], supplier_id=[$5])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, ITEMTABLE]])\n" +
+                           "          PhoenixTableScan(table=[[phoenix, Join, ItemTable]])\n" +
                            "        PhoenixServerProject(supplier_id=[$0], NAME=[$1])\n" +
-                           "          PhoenixTableScan(table=[[phoenix, SUPPLIERTABLE]])\n")
+                           "          PhoenixTableScan(table=[[phoenix, Join, SupplierTable]])\n")
                 .resultIs(new Object[][] {
                           {"0000000001", "T1", "0000000001", "S1"}, 
                           {"0000000002", "T2", "0000000001", "S1"}, 
@@ -587,14 +604,14 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                .explainIs("PhoenixToEnumerableConverter\n" +
                           "  PhoenixServerProject(order_id=[$0], QUANTITY=[$4])\n" +
                           "    PhoenixServerJoin(condition=[AND(=($2, $6), =($4, $7))], joinType=[inner])\n" +
-                          "      PhoenixTableScan(table=[[phoenix, ORDERTABLE]])\n" +
+                          "      PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
                           "      PhoenixServerAggregate(group=[{0}], EXPR$0=[MAX($1)])\n" +
                           "        PhoenixServerProject(item_id0=[$6], QUANTITY=[$4])\n" +
                           "          PhoenixServerJoin(condition=[=($6, $2)], joinType=[inner])\n" +
-                          "            PhoenixTableScan(table=[[phoenix, ORDERTABLE]])\n" +
+                          "            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n" +
                           "            PhoenixServerAggregate(group=[{0}])\n" +
                           "              PhoenixServerProject(item_id=[$2])\n" +
-                          "                PhoenixTableScan(table=[[phoenix, ORDERTABLE]])\n")
+                          "                PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n")
                .resultIs(new Object[][]{
                          {"000000000000001", 1000},
                          {"000000000000003", 3000},
@@ -602,6 +619,33 @@ public class CalciteTest extends BaseClientManagedTimeIT {
                          {"000000000000005", 5000}})
                .close();
     }
+    
+    @Test public void testConnectJoinHsqldb() {
+        final Start start = new Start() {
+            @Override
+            Connection createConnection() throws Exception {
+                return createConnectionWithHsqldb();
+            }
+        };
+        start.sql("select \"the_year\", count(*) as c, min(\"the_month\") as m\n"
+            + "from \"foodmart\".\"time_by_day\" t\n"
+            + "join " + JOIN_ORDER_TABLE_FULL_NAME + " c on t.\"the_year\" = c.quantity\n" 
+            + "group by \"the_year\"\n"
+            + "order by 1, 2")
+            .explainIs("EnumerableSort(sort0=[$0], sort1=[$1], dir0=[ASC], dir1=[ASC])\n" +
+"  EnumerableAggregate(group=[{0}], C=[COUNT()], M=[MIN($1)])\n" +
+"    EnumerableCalc(expr#0..3=[{inputs}], the_year=[$t1], the_month=[$t0])\n" +
+"      EnumerableJoin(condition=[=($2, $3)], joinType=[inner])\n" +
+"        JdbcToEnumerableConverter\n" +
+"          JdbcProject(the_month=[$3], the_year=[$4], $f10=[CAST($4):INTEGER])\n" +
+"            JdbcTableScan(table=[[foodmart, time_by_day]])\n" +
+"        PhoenixToEnumerableConverter\n" +
+"          PhoenixServerProject(QUANTITY=[$4])\n" +
+"            PhoenixTableScan(table=[[phoenix, Join, OrderTable]])\n")
+            .resultIs(new Object[][] {})
+            //.resultIs(new Object[][] { new Object[] {(short) 1997, 365L, "April"}, new Object[] {(short) 1998, 365L, "April"}})
+            .close();;
+    }
 
     @Test public void testConnectUsingModel() throws Exception {
         final Start start = new Start() {

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
index c51308e..14c0ee9 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixSchema.java
@@ -1,7 +1,10 @@
 package org.apache.phoenix.calcite;
 
+import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.SetMultimap;
+
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.schema.*;
 import org.apache.phoenix.compile.ColumnResolver;
@@ -22,11 +25,23 @@ import java.util.*;
  * Implementation of Calcite's {@link Schema} SPI for Phoenix.
  */
 public class PhoenixSchema implements Schema {
-    private final String schemaName = null;
+    private final String schemaName;
     private final PhoenixConnection pc;
     protected final MetaDataClient client;
+    
+    // TODO to be removed after PHOENIX-1878.
+    private static final SetMultimap<String, String> tableCache;
+    static {
+        tableCache = HashMultimap.<String, String> create();
+        tableCache.put("", "ATABLE");
+        tableCache.put("Join", "ItemTable");
+        tableCache.put("Join", "SupplierTable");
+        tableCache.put("Join", "CustomerTable");
+        tableCache.put("Join", "OrderTable");
+    }
 
-    PhoenixSchema(PhoenixConnection pc) {
+    PhoenixSchema(String name, PhoenixConnection pc) {
+        this.schemaName = name;
         this.pc = pc;
         this.client = new MetaDataClient(pc);
     }
@@ -42,7 +57,7 @@ public class PhoenixSchema implements Schema {
                 DriverManager.getConnection(url, properties);
             final PhoenixConnection phoenixConnection =
                 connection.unwrap(PhoenixConnection.class);
-            return new PhoenixSchema(phoenixConnection);
+            return new PhoenixSchema(null, phoenixConnection);
         } catch (SQLException e) {
             throw new RuntimeException(e);
         }
@@ -66,7 +81,7 @@ public class PhoenixSchema implements Schema {
 
     @Override
     public Set<String> getTableNames() {
-        return ImmutableSet.of("ATABLE", "ITEMTABLE", "SUPPLIERTABLE", "ORDERTABLE", "CUSTOMERTABLE");
+        return tableCache.get(schemaName == null ? "" : schemaName);
     }
 
     @Override
@@ -81,12 +96,18 @@ public class PhoenixSchema implements Schema {
 
     @Override
     public Schema getSubSchema(String name) {
-        return null;
+        if (schemaName != null || !tableCache.containsKey(name))
+            return null;
+        
+        return new PhoenixSchema(name, pc);
     }
 
     @Override
     public Set<String> getSubSchemaNames() {
-        return ImmutableSet.of();
+        if (schemaName != null)
+            return Collections.emptySet();
+        
+        return tableCache.keySet();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
index 9938a30..e9378af 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/calcite/PhoenixTable.java
@@ -95,7 +95,7 @@ public class PhoenixTable extends AbstractTable implements TranslatableTable {
             public Double getRowCount() {
                 // TODO
                 String tableName = pTable.getTableName().getString();
-                return tableName.equals("ITEMTABLE") ? 70d : tableName.equals("SUPPLIERTABLE") ? 60d : 100d;
+                return tableName.equals("ItemTable") ? 70d : tableName.equals("SupplierTable") ? 60d : 100d;
             }
 
             @Override

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
index 04ae727..66695f8 100644
--- a/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
+++ b/phoenix-core/src/test/java/org/apache/phoenix/util/TestUtil.java
@@ -197,15 +197,15 @@ public class TestUtil {
     public static final String JOIN_ITEM_TABLE = "ItemTable";
     public static final String JOIN_SUPPLIER_TABLE = "SupplierTable";
     public static final String JOIN_COITEM_TABLE = "CoitemTable";
-    public static final String JOIN_ORDER_TABLE_FULL_NAME = JOIN_ORDER_TABLE; // '"' + JOIN_SCHEMA + "\".\"" + JOIN_ORDER_TABLE + '"';
-    public static final String JOIN_CUSTOMER_TABLE_FULL_NAME = JOIN_CUSTOMER_TABLE; // '"' + JOIN_SCHEMA + "\".\"" + JOIN_CUSTOMER_TABLE + '"';
-    public static final String JOIN_ITEM_TABLE_FULL_NAME = JOIN_ITEM_TABLE; //'"' + JOIN_SCHEMA + "\".\"" + JOIN_ITEM_TABLE + '"';
-    public static final String JOIN_SUPPLIER_TABLE_FULL_NAME = JOIN_SUPPLIER_TABLE; //'"' + JOIN_SCHEMA + "\".\"" + JOIN_SUPPLIER_TABLE + '"';
+    public static final String JOIN_ORDER_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_ORDER_TABLE + '"';
+    public static final String JOIN_CUSTOMER_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_CUSTOMER_TABLE + '"';
+    public static final String JOIN_ITEM_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_ITEM_TABLE + '"';
+    public static final String JOIN_SUPPLIER_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_SUPPLIER_TABLE + '"';
     public static final String JOIN_COITEM_TABLE_FULL_NAME = '"' + JOIN_SCHEMA + "\".\"" + JOIN_COITEM_TABLE + '"';
-    public static final String JOIN_ORDER_TABLE_DISPLAY_NAME = JOIN_ORDER_TABLE.toUpperCase();
-    public static final String JOIN_CUSTOMER_TABLE_DISPLAY_NAME = JOIN_CUSTOMER_TABLE.toUpperCase();
-    public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_ITEM_TABLE.toUpperCase();
-    public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SUPPLIER_TABLE.toUpperCase();
+    public static final String JOIN_ORDER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ORDER_TABLE;
+    public static final String JOIN_CUSTOMER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_CUSTOMER_TABLE;
+    public static final String JOIN_ITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_ITEM_TABLE;
+    public static final String JOIN_SUPPLIER_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_SUPPLIER_TABLE;
     public static final String JOIN_COITEM_TABLE_DISPLAY_NAME = JOIN_SCHEMA + "." + JOIN_COITEM_TABLE;
 
     /**

http://git-wip-us.apache.org/repos/asf/phoenix/blob/d555ee62/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index d89899c..984b3b9 100644
--- a/pom.xml
+++ b/pom.xml
@@ -677,6 +677,18 @@
         <artifactId>calcite-core</artifactId>
         <version>${calcite.version}</version>
       </dependency>
+      <dependency>
+        <groupId>net.hydromatic</groupId>
+        <artifactId>foodmart-data-hsqldb</artifactId>
+        <version>0.3</version>
+        <scope>test</scope>
+      </dependency>
+      <dependency>
+        <groupId>org.hsqldb</groupId>
+        <artifactId>hsqldb</artifactId>
+        <version>2.3.1</version>
+        <scope>test</scope>
+      </dependency>
     </dependencies>
   </dependencyManagement>