You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ve...@apache.org on 2015/04/02 19:33:16 UTC

[2/6] drill git commit: DRILL-2640: Move view tests out of 'exec/jdbc' module into appropriate module.

DRILL-2640: Move view tests out of 'exec/jdbc' module into appropriate module.


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

Branch: refs/heads/master
Commit: f956b7278a9775d3366e2aab37c7a98e893e8b33
Parents: 0b6cddf
Author: vkorukanti <ve...@gmail.com>
Authored: Tue Mar 31 14:08:30 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Thu Apr 2 08:29:18 2015 -0700

----------------------------------------------------------------------
 .../sql/hive/TestViewSupportOnHiveTables.java   | 115 ++++
 .../drill/exec/sql/TestBaseViewSupport.java     | 168 +++++
 .../apache/drill/exec/sql/TestViewSupport.java  | 344 +++++++++-
 .../src/test/resources/nation/nation.tbl        |  25 +
 .../org/apache/drill/jdbc/test/TestViews.java   | 644 -------------------
 exec/jdbc/src/test/resources/nation/nation.tbl  |  25 -
 6 files changed, 648 insertions(+), 673 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
new file mode 100644
index 0000000..14ab506
--- /dev/null
+++ b/contrib/storage-hive/core/src/test/java/org/apache/drill/exec/sql/hive/TestViewSupportOnHiveTables.java
@@ -0,0 +1,115 @@
+/**
+ * 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.drill.exec.sql.hive;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.drill.exec.sql.TestBaseViewSupport;
+import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestViewSupportOnHiveTables extends TestBaseViewSupport {
+  private static final String TEMP_SCHEMA = "dfs_test.tmp";
+
+  protected static HiveTestDataGenerator hiveTest;
+
+  @BeforeClass
+  public static void generateHive() throws Exception{
+    hiveTest = new HiveTestDataGenerator(getDrillbitContext().getStorage());
+    hiveTest.createAndAddHiveTestPlugin();
+  }
+
+  @Test
+  public void viewWithStarInDef_StarInQuery() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT * FROM hive.kv",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "key", "value"},
+        ImmutableList.of(new Object[] { 1, " key_1" })
+    );
+  }
+
+  @Test
+  public void viewWithStarInDef_SelectFieldsInQuery1() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT * FROM hive.kv",
+        "SELECT key, `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "key", "value" },
+        ImmutableList.of(new Object[] { 1, " key_1" })
+    );
+  }
+
+  @Test
+  public void viewWithStarInDef_SelectFieldsInQuery2() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT * FROM hive.kv",
+        "SELECT `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "value" },
+        ImmutableList.of(new Object[] { " key_1" })
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_StarInQuery() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT key, `value` FROM hive.kv",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "key", "value" },
+        ImmutableList.of(new Object[] { 1, " key_1" })
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInQuery() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT key, `value` FROM hive.kv",
+        "SELECT key, `value` FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "key", "value" },
+        ImmutableList.of(new Object[] { 1, " key_1" })
+    );
+  }
+
+  @Test
+  public void testInfoSchemaWithHiveView() throws Exception {
+    testBuilder()
+        .optionSettingQueriesForTestQuery("USE hive.`default`")
+        .sqlQuery("SELECT * FROM INFORMATION_SCHEMA.VIEWS WHERE TABLE_NAME = 'hiveview'")
+        .unOrdered()
+        .baselineColumns("TABLE_CATALOG", "TABLE_SCHEMA", "TABLE_NAME", "VIEW_DEFINITION")
+        .baselineValues("DRILL", "hive.default", "hiveview", "SELECT `kv`.`key`, `kv`.`value` FROM `default`.`kv`")
+        .go();
+  }
+
+  @AfterClass
+  public static void cleanupHiveTestData() throws Exception{
+    if (hiveTest != null) {
+      hiveTest.deleteHiveTestPlugin();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
new file mode 100644
index 0000000..bfe113b
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestBaseViewSupport.java
@@ -0,0 +1,168 @@
+/**
+ * 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.drill.exec.sql;
+
+import com.google.common.base.Strings;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.TestBuilder;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Base class for view tests. It has utility methods which can be used when writing tests for views on tables
+ * in different storage engines such as Hive, HBase etc.
+ */
+public class TestBaseViewSupport extends BaseTestQuery {
+  private static AtomicInteger viewSeqNum = new AtomicInteger(0);
+
+  /**
+   * Create view with given parameters.
+   *
+   * Current default schema "dfs_test"
+   *
+   * CREATE VIEW tmp.viewName(f1, f2) AS SELECT * FROM cp.`region.json`
+   *
+   * For the above CREATE VIEW query, function parameters are:
+   *   viewSchema = "tmp"
+   *   viewName = "viewName"
+   *   finalSchema = "dfs_test.tmp"
+   *   viewFields = "(f1, f2)"
+   *   viewDef = "SELECT * FROM cp.`region.json`"
+   *
+   * @param viewSchema Schema name to prefix when referring to the view.
+   * @param viewName Name of the view.
+   * @param finalSchema Absolute schema path where the view is created. Pameter <i>viewSchema</i> may refer the schema
+   *                    with respect the current default schema. Combining <i>viewSchema</i> with default schema
+   *                    gives the final schema.
+   * @param viewFields If the created view needs to specify the fields, otherwise null
+   * @param viewDef Definition of the view.
+   * @throws Exception
+   */
+  protected static void createViewHelper(final String viewSchema, final String viewName,
+      final String finalSchema, final String viewFields, final String viewDef) throws Exception {
+
+    String viewFullName = "`" + viewName + "`";
+    if (!Strings.isNullOrEmpty(viewSchema)) {
+      viewFullName = viewSchema + "." + viewFullName;
+    }
+
+    final String createViewSql = String.format("CREATE VIEW %s %s AS %s", viewFullName,
+        viewFields == null ? "" : viewFields, viewDef);
+
+    testBuilder()
+        .sqlQuery(createViewSql)
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(true, String.format("View '%s' created successfully in '%s' schema", viewName, finalSchema))
+        .go();
+  }
+
+  /**
+   * Drop view with given parameters.
+   *
+   * Current schema "dfs_test"
+   * DROP VIEW tmp.viewName
+   *
+   * For the above DROP VIEW query, function parameters values are:
+   *  viewSchema = "tmp"
+   *  "viewName" = "viewName"
+   *  "finalSchema" = "dfs_test.tmp"
+   *
+   * @param viewSchema
+   * @param viewName
+   * @param finalSchema
+   * @throws Exception
+   */
+  protected static void dropViewHelper(final String viewSchema, final String viewName, final String finalSchema) throws
+      Exception{
+    String viewFullName = "`" + viewName + "`";
+    if (!Strings.isNullOrEmpty(viewSchema)) {
+      viewFullName = viewSchema + "." + viewFullName;
+    }
+
+    testBuilder()
+        .sqlQuery(String.format("DROP VIEW %s", viewFullName))
+        .unOrdered()
+        .baselineColumns("ok", "summary")
+        .baselineValues(true, String.format("View '%s' deleted successfully from '%s' schema", viewName, finalSchema))
+        .go();
+  }
+
+  /**
+   * Execute the given query and check against the given baseline.
+   *
+   * @param query
+   * @param baselineColumns
+   * @param baselineValues
+   * @throws Exception
+   */
+  protected static void queryViewHelper(final String query, final String[] baselineColumns,
+      final List<Object[]> baselineValues) throws Exception {
+    TestBuilder testBuilder = testBuilder()
+        .sqlQuery(query)
+        .unOrdered()
+        .baselineColumns(baselineColumns);
+
+    for(Object[] values : baselineValues) {
+      testBuilder = testBuilder.baselineValues(values);
+    }
+
+    testBuilder.go();
+  }
+
+  /**
+   * Generate a view name by appending the test class with a increasing counter.
+   *
+   * @return
+   */
+  protected static String generateViewName() {
+    return TestBaseViewSupport.class.getSimpleName() + "_" + viewSeqNum.incrementAndGet();
+  }
+
+  /**
+   * Tests creating a view with given parameters, query the view and check against the provided baselines and finally
+   * drop the view.
+   *
+   * @param finalSchema Absolute schema path where the view is going to be created.
+   * @param viewFields If view has any field list
+   * @param viewDef Definition of the view.
+   * @param queryOnView Query to run on created view. Refer to test view using "TEST_VIEW_NAME". Similarly schema
+   *                    using "TEST_SCHEMA".
+   * @param expectedBaselineColumns Expected columns from querying the view.
+   * @param expectedBaselineValues Expected row values from querying the view.
+   * @throws Exception
+   */
+  protected static void testViewHelper(final String finalSchema, final String viewFields, final String viewDef,
+      String queryOnView, final String[] expectedBaselineColumns, final List<Object[]> expectedBaselineValues)
+      throws Exception {
+    final String viewName = generateViewName();
+
+    try {
+      createViewHelper(finalSchema, viewName, finalSchema, viewFields, viewDef);
+
+      queryOnView = queryOnView
+          .replace("TEST_VIEW_NAME", "`" + viewName + "`")
+          .replace("TEST_SCHEMA", finalSchema);
+
+      queryViewHelper(queryOnView, expectedBaselineColumns, expectedBaselineValues);
+    } finally {
+      dropViewHelper(finalSchema, viewName, finalSchema);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
index 3b55b99..578eace 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestViewSupport.java
@@ -17,11 +17,14 @@
  */
 package org.apache.drill.exec.sql;
 
-import org.apache.drill.BaseTestQuery;
+import com.google.common.collect.ImmutableList;
+import org.junit.Ignore;
 import org.junit.Test;
 
-public class TestViewSupport extends BaseTestQuery{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestViewSupport.class);
+import java.util.List;
+
+public class TestViewSupport extends TestBaseViewSupport {
+  private static final String TEMP_SCHEMA = "dfs_test.tmp";
 
   @Test
   public void referToSchemaInsideAndOutsideView() throws Exception {
@@ -45,7 +48,7 @@ public class TestViewSupport extends BaseTestQuery{
    * where columns are required.
    */
   @Test
-  public void testNullabilityPropertyInViewPersistence() throws Exception {
+  public void nullabilityPropertyInViewPersistence() throws Exception {
     final String viewName = "testNullabilityPropertyInViewPersistence";
     try {
 
@@ -77,4 +80,337 @@ public class TestViewSupport extends BaseTestQuery{
       test("drop view " + viewName + ";");
     }
   }
+
+  @Test
+  public void viewWithStarInDef_StarInQuery() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT * FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "region_id", "sales_city", "sales_state_province", "sales_district", "sales_region",
+            "sales_country", "sales_district_id" },
+        ImmutableList.of(new Object[] { 0L, "None", "None", "No District", "No Region", "No Country", 0L })
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_StarInQuery() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "region_id", "sales_city" },
+        ImmutableList.of(
+            new Object[] { 0L, "None" },
+            new Object[] { 1L, "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInView_StarInQuery() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        "(regionid, salescity)",
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "regionid", "salescity" },
+        ImmutableList.of(
+            new Object[] { 0L, "None" },
+            new Object[] { 1L, "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithStarInDef_SelectFieldsInQuery() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT * FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT region_id, sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "region_id", "sales_city" },
+        ImmutableList.of(
+            new Object[] { 0L, "None" },
+            new Object[] { 1L, "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInQuery1() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT region_id, sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "region_id", "sales_city" },
+        ImmutableList.of(
+            new Object[] { 0L, "None" },
+            new Object[] { 1L, "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInQuery2() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`",
+        "SELECT sales_city FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "sales_city" },
+        ImmutableList.of(
+            new Object[] { "None" },
+            new Object[] { "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInView_SelectFieldsInQuery1() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        "(regionid, salescity)",
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id` LIMIT 2",
+        "SELECT regionid, salescity FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "regionid", "salescity" },
+        ImmutableList.of(
+            new Object[] { 0L, "None" },
+            new Object[] { 1L, "San Francisco" }
+        )
+    );
+  }
+
+  @Test
+  public void viewWithSelectFieldsInDef_SelectFieldsInView_SelectFieldsInQuery2() throws Exception {
+    testViewHelper(
+        TEMP_SCHEMA,
+        "(regionid, salescity)",
+        "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id` DESC",
+        "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "regionid" },
+        ImmutableList.of(
+            new Object[] { 109L },
+            new Object[] { 108L }
+        )
+    );
+  }
+
+  @Test
+  @Ignore("DRILL-1921")
+  public void viewWithUnionWithSelectFieldsInDef_StarInQuery() throws Exception{
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        "SELECT region_id FROM cp.`region.json` UNION SELECT employee_id FROM cp.`employee.json`",
+        "SELECT regionid FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 2",
+        new String[] { "regionid" },
+        ImmutableList.of(
+            new Object[] { 110L },
+            new Object[] { 108L }
+        )
+    );
+  }
+
+  @Test
+  public void viewCreatedFromAnotherView() throws Exception {
+    final String innerView = generateViewName();
+    final String outerView = generateViewName();
+
+    try {
+      createViewHelper(TEMP_SCHEMA, innerView, TEMP_SCHEMA, null,
+          "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY `region_id`");
+
+      createViewHelper(TEMP_SCHEMA, outerView, TEMP_SCHEMA, null,
+          String.format("SELECT region_id FROM %s.`%s`", TEMP_SCHEMA, innerView));
+
+      queryViewHelper(
+          String.format("SELECT region_id FROM %s.`%s` LIMIT 1", TEMP_SCHEMA, outerView),
+          new String[] { "region_id" },
+          ImmutableList.of(new Object[] { 0L })
+      );
+    } finally {
+      dropViewHelper(TEMP_SCHEMA, outerView, TEMP_SCHEMA);
+      dropViewHelper(TEMP_SCHEMA, innerView, TEMP_SCHEMA);
+    }
+  }
+
+  @Test // DRILL-1015
+  public void viewWithCompoundIdentifiersInDef() throws Exception{
+    final String viewDef = "SELECT " +
+        "cast(columns[0] AS int) n_nationkey, " +
+        "cast(columns[1] AS CHAR(25)) n_name, " +
+        "cast(columns[2] AS INT) n_regionkey, " +
+        "cast(columns[3] AS VARCHAR(152)) n_comment " +
+        "FROM dfs_test.`[WORKING_PATH]/src/test/resources/nation`";
+
+    testViewHelper(
+        TEMP_SCHEMA,
+        null,
+        viewDef,
+        "SELECT * FROM TEST_SCHEMA.TEST_VIEW_NAME LIMIT 1",
+        new String[] { "n_nationkey", "n_name", "n_regionkey", "n_comment" },
+        ImmutableList.of(
+            new Object[] { 0, "ALGERIA", 0, " haggle. carefully final deposits detect slyly agai" }
+        )
+    );
+  }
+
+  @Test
+  public void createViewWhenViewAlreadyExists() throws Exception {
+    final String viewName = generateViewName();
+
+    try {
+      final String viewDef1 = "SELECT region_id, sales_city FROM cp.`region.json`";
+
+      // Create the view
+      createViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA, null, viewDef1);
+
+      // Try to create the view with same name in same schema.
+      final String createViewSql = String.format("CREATE VIEW %s.`%s` AS %s", TEMP_SCHEMA, viewName, viewDef1);
+      testBuilder()
+          .sqlQuery(createViewSql)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(false, "View with given name already exists in current schema")
+          .go();
+
+      // Try creating the view with same name in same schema, but with CREATE OR REPLACE VIEW clause
+      final String viewDef2 = "SELECT sales_state_province FROM cp.`region.json` ORDER BY `region_id`";
+      final String createOrReplaceViewSql = String.format("CREATE OR REPLACE VIEW %s.`%s` AS %s", TEMP_SCHEMA,
+          viewName, viewDef2);
+
+      testBuilder()
+          .sqlQuery(createOrReplaceViewSql)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true,
+              String.format("View '%s' replaced successfully in '%s' schema", viewName, TEMP_SCHEMA))
+          .go();
+
+      // Make sure the new view created returns the data expected.
+      queryViewHelper(String.format("SELECT * FROM %s.`%s` LIMIT 1", TEMP_SCHEMA, viewName),
+          new String[] { "sales_state_province" },
+          ImmutableList.of(new Object[] { "None" })
+      );
+    } finally {
+      dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
+    }
+  }
+
+  @Test
+  public void infoSchemaWithView() throws Exception {
+    final String viewName = generateViewName();
+
+    try {
+      test("USE " + TEMP_SCHEMA);
+      createViewHelper(null /*pass no schema*/, viewName, TEMP_SCHEMA, null,
+          "SELECT cast(`employee_id` as integer) employeeid FROM cp.`employee.json`");
+
+      // Test SHOW TABLES on view
+      testBuilder()
+          .sqlQuery(String.format("SHOW TABLES like '%s'", viewName))
+          .unOrdered()
+          .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
+          .baselineValues(TEMP_SCHEMA, viewName)
+          .go();
+
+      // Test record in INFORMATION_SCHEMA.VIEWS
+      testBuilder()
+          .sqlQuery(String.format("SELECT * FROM INFORMATION_SCHEMA.VIEWS WHERE TABLE_NAME = '%s'", viewName))
+          .unOrdered()
+          .baselineColumns("TABLE_CATALOG", "TABLE_SCHEMA", "TABLE_NAME", "VIEW_DEFINITION")
+          .baselineValues("DRILL", TEMP_SCHEMA, viewName,
+              "SELECT CAST(`employee_id` AS INTEGER) AS `employeeid`\n" + "FROM `cp`.`employee.json`")
+          .go();
+
+      // Test record in INFORMATION_SCHEMA.TABLES
+      testBuilder()
+          .sqlQuery(String.format("SELECT * FROM INFORMATION_SCHEMA.`TABLES` WHERE TABLE_NAME = '%s'", viewName))
+          .unOrdered()
+          .baselineColumns("TABLE_CATALOG", "TABLE_SCHEMA", "TABLE_NAME", "TABLE_TYPE")
+          .baselineValues("DRILL", TEMP_SCHEMA, viewName, "VIEW")
+          .go();
+
+      // Test DESCRIBE view
+      testBuilder()
+          .sqlQuery(String.format("DESCRIBE `%s`", viewName))
+          .unOrdered()
+          .baselineColumns("COLUMN_NAME", "DATA_TYPE", "IS_NULLABLE")
+          .baselineValues("employeeid", "INTEGER", "YES")
+          .go();
+    } finally {
+      dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
+    }
+  }
+
+  @Test
+  public void viewWithPartialSchemaIdentifier() throws Exception {
+    final String viewName = generateViewName();
+
+    try {
+      // Change default schema to just "dfs_test". View is actually created in "dfs_test.tmp" schema.
+      test("USE dfs_test");
+
+      // Create a view with with "tmp" schema identifier
+      createViewHelper("tmp", viewName, TEMP_SCHEMA, null,
+          "SELECT CAST(`employee_id` AS INTEGER) AS `employeeid`\n" + "FROM `cp`.`employee.json`");
+
+      final String[] baselineColumns = new String[] { "employeeid" };
+      final List<Object[]> baselineValues = ImmutableList.of(new Object[] { 1156 });
+
+      // Query view from current schema "dfs_test" by referring to the view using "tmp.viewName"
+      queryViewHelper(
+          String.format("SELECT * FROM %s.`%s` ORDER BY `employeeid` DESC LIMIT 1", "tmp", viewName),
+          baselineColumns, baselineValues);
+
+      // Change the default schema to "dfs_test.tmp" and query view by referring to it using "viewName"
+      test("USE dfs_test.tmp");
+      queryViewHelper(
+          String.format("SELECT * FROM `%s` ORDER BY `employeeid` DESC LIMIT 1", viewName),
+          baselineColumns, baselineValues);
+
+      // Change the default schema to "cp" and query view by referring to it using "dfs_test.tmp.viewName";
+      test("USE cp");
+      queryViewHelper(
+          String.format("SELECT * FROM %s.`%s` ORDER BY `employeeid` DESC LIMIT 1", "dfs_test.tmp", viewName),
+          baselineColumns, baselineValues);
+
+    } finally {
+      dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
+    }
+  }
+
+  @Test // DRILL-1114
+  public void viewResolvingTablesInWorkspaceSchema() throws Exception {
+    final String viewName = generateViewName();
+
+    try {
+      // Change default schema to "cp"
+      test("USE cp");
+
+      // Create a view with full schema identifier and refer the "region.json" as without schema.
+      createViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA, null, "SELECT region_id, sales_city FROM `region.json`");
+
+      final String[] baselineColumns = new String[] { "region_id", "sales_city" };
+      final List<Object[]> baselineValues = ImmutableList.of(new Object[] { 109L, "Santa Fe"});
+
+      // Query the view
+      queryViewHelper(
+          String.format("SELECT * FROM %s.`%s` ORDER BY region_id DESC LIMIT 1", "dfs_test.tmp", viewName),
+          baselineColumns, baselineValues);
+
+      // Change default schema to "dfs_test" and query by referring to the view using "tmp.viewName"
+      test("USE dfs_test");
+      queryViewHelper(
+          String.format("SELECT * FROM %s.`%s` ORDER BY region_id DESC LIMIT 1", "tmp", viewName),
+          baselineColumns,baselineValues);
+
+    } finally {
+      dropViewHelper(TEMP_SCHEMA, viewName, TEMP_SCHEMA);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/exec/java-exec/src/test/resources/nation/nation.tbl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/nation/nation.tbl b/exec/java-exec/src/test/resources/nation/nation.tbl
new file mode 100644
index 0000000..ed3fd5b
--- /dev/null
+++ b/exec/java-exec/src/test/resources/nation/nation.tbl
@@ -0,0 +1,25 @@
+0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|
+1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|
+2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |
+3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|
+4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|
+5|ETHIOPIA|0|ven packages wake quickly. regu|
+6|FRANCE|3|refully final requests. regular, ironi|
+7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|
+8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|
+9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|
+10|IRAN|4|efully alongside of the slyly final dependencies. |
+11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|
+12|JAPAN|2|ously. final, express gifts cajole a|
+13|JORDAN|4|ic deposits are blithely about the carefully regular pa|
+14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|
+15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|
+16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|
+17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|
+18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|
+19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|
+20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|
+21|VIETNAM|2|hely enticingly express accounts. even, final |
+22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|
+23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|
+24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|

http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
deleted file mode 100644
index 3fe8e2f..0000000
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
+++ /dev/null
@@ -1,644 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.jdbc.test;
-
-import static org.junit.Assert.assertTrue;
-
-import java.io.File;
-import java.sql.Connection;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Statement;
-
-import org.apache.commons.io.FileUtils;
-import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
-import org.apache.drill.test.DrillAssert;
-import org.junit.BeforeClass;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import com.google.common.base.Function;
-
-/** Contains tests for creating/dropping and using views in Drill. */
-public class TestViews extends JdbcTestQueryBase {
-
-  @BeforeClass
-  public static void generateHive() throws Exception{
-    new HiveTestDataGenerator().generateTestData();
-
-    // delete tmp workspace directory
-    File f = new File("/tmp/drilltest");
-    if(f.exists()){
-      FileUtils.cleanDirectory(f);
-      FileUtils.forceDelete(f);
-    }
-  }
-
-  // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment race
-  // conditions are fixed (not just DRILL-2245 fixes).
-  ///**
-  // * Calls {@link ResultSet#next} on given {@code ResultSet} until it returns
-  // * false.  (For TEMPORARY workaround for query cancelation race condition.)
-  // */
-  //private void nextUntilEnd(final ResultSet resultSet) throws SQLException {
-  //  while (resultSet.next()) {
-  //  }
-  //}
-
-  /** Helper test method for view tests */
-  private void testViewHelper(final String viewCreate, final String viewName,
-                              final String viewQuery, final String queryResult) throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE dfs_test.tmp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create view
-          resultSet = statement.executeQuery(viewCreate);
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String viewCreateResult = "ok=true; summary=View '" + viewName + "' created successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, viewCreateResult),
-              viewCreateResult.equals(result));
-
-          // query from view
-          resultSet = statement.executeQuery(viewQuery);
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, queryResult),
-              queryResult.equals(result));
-
-          resultSet = statement.executeQuery("drop view " + viewName);
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testView1() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview1 AS SELECT * FROM cp.`region.json`",
-        "testview1",
-        "SELECT * FROM testview1 LIMIT 1",
-        "region_id=0; sales_city=None; sales_state_province=None; sales_district=No District; " +
-            "sales_region=No Region; sales_country=No Country; sales_district_id=0");
-  }
-
-  @Test
-  public void testView2() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview2 AS SELECT region_id, sales_city FROM cp.`region.json`",
-        "testview2",
-        "SELECT * FROM testview2 LIMIT 2",
-        "region_id=0; sales_city=None\nregion_id=1; sales_city=San Francisco");
-  }
-
-  @Test
-  public void testView3() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview3(regionid, salescity) AS SELECT region_id, sales_city FROM cp.`region.json`",
-        "testview3",
-        "SELECT * FROM testview3 LIMIT 2",
-        "regionid=0; salescity=None\nregionid=1; salescity=San Francisco");
-  }
-
-  @Test
-  @Ignore // See DRILL-595 - can't project columns from inner query.
-  public void testView4() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview1 AS SELECT * FROM cp.`region.json`",
-        "testview1",
-        "SELECT region_id, sales_city FROM testview1 LIMIT 2",
-        "");
-  }
-
-  @Test
-  public void testView5() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview2 AS SELECT region_id, sales_city FROM cp.`region.json`",
-        "testview2",
-        "SELECT region_id, sales_city FROM testview2 LIMIT 2",
-        "region_id=0; sales_city=None\nregion_id=1; sales_city=San Francisco");
-  }
-
-  @Test
-  public void testView6() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview2 AS SELECT region_id, sales_city FROM cp.`region.json`",
-        "testview2",
-        "SELECT sales_city FROM testview2 LIMIT 2",
-        "sales_city=None\nsales_city=San Francisco");
-  }
-
-  @Test
-  public void testView7() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview3(regionid, salescity) AS SELECT region_id, sales_city FROM cp.`region.json` LIMIT 2",
-        "testview3",
-        "SELECT regionid, salescity FROM testview3",
-        "regionid=0; salescity=None\nregionid=1; salescity=San Francisco");
-  }
-
-  @Test
-  public void testView8() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview3(regionid, salescity) AS " +
-            "SELECT region_id, sales_city FROM cp.`region.json` ORDER BY region_id DESC",
-        "testview3",
-        "SELECT regionid FROM testview3 LIMIT 2",
-        "regionid=109\nregionid=108");
-  }
-
-  @Test
-  @Ignore // Query on testview2 fails with CannotPlanException. Seems to be an issue with Union.
-  public void testView9() throws Exception{
-    testViewHelper(
-        "CREATE VIEW testview2 AS " +
-            "SELECT region_id FROM cp.`region.json` " +
-            "UNION " +
-            "SELECT employee_id FROM cp.`employee.json`",
-        "testview2",
-        "SELECT sales_city FROM testview2 LIMIT 2",
-        "sales_city=None\nsales_city=San Francisco");
-  }
-
-  @Test
-  public void testViewOnHiveTable1() throws Exception{
-    testViewHelper(
-        "CREATE VIEW hiveview AS SELECT * FROM hive_test.kv",
-        "hiveview",
-        "SELECT * FROM hiveview LIMIT 1",
-        "key=1; value= key_1");
-  }
-
-  @Test
-  public void testViewOnHiveTable2() throws Exception{
-    testViewHelper(
-        "CREATE VIEW hiveview AS SELECT * FROM hive_test.kv",
-        "hiveview",
-        "SELECT key, `value` FROM hiveview LIMIT 1",
-        "key=1; value= key_1");
-  }
-
-  @Test
-  public void testViewOnHiveTable3() throws Exception{
-    testViewHelper(
-        "CREATE VIEW hiveview AS SELECT * FROM hive_test.kv",
-        "hiveview",
-        "SELECT `value` FROM hiveview LIMIT 1",
-        "value= key_1");
-  }
-
-  @Test
-  public void testViewOnHiveTable4() throws Exception{
-    testViewHelper(
-        "CREATE VIEW hiveview AS SELECT key, `value` FROM hive_test.kv",
-        "hiveview",
-        "SELECT * FROM hiveview LIMIT 1",
-        "key=1; value= key_1");
-  }
-
-  @Test
-  public void testViewOnHiveTable5() throws Exception{
-    testViewHelper(
-        "CREATE VIEW hiveview AS SELECT key, `value` FROM hive_test.kv",
-        "hiveview",
-        "SELECT key, `value` FROM hiveview LIMIT 1",
-        "key=1; value= key_1");
-  }
-
-  @Test
-  public void testViewWithCompoundIdentifiersInSchema() throws Exception{
-    String query = String.format("CREATE VIEW nationview AS SELECT " +
-        "cast(columns[0] AS int) n_nationkey, " +
-        "cast(columns[1] AS CHAR(25)) n_name, " +
-        "cast(columns[2] AS INT) n_regionkey, " +
-        "cast(columns[3] AS VARCHAR(152)) n_comment " +
-        "FROM dfs_test.`%s/src/test/resources/nation`", WORKING_PATH);
-
-    testViewHelper(
-        query,
-        "nationview",
-        "SELECT * FROM nationview LIMIT 1",
-        "n_nationkey=0; n_name=ALGERIA; n_regionkey=0; n_comment= haggle. carefully final deposits detect slyly agai");
-  }
-
-  @Test
-  public void testDropView() throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE dfs_test.tmp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create view
-          resultSet = statement.executeQuery(
-              "CREATE VIEW testview3(regionid) AS SELECT region_id FROM cp.`region.json`");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // query from view
-          resultSet = statement.executeQuery("SELECT regionid FROM testview3 LIMIT 1");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "regionid=0";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("DROP VIEW testview3");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "ok=true; summary=View 'testview3' deleted successfully from 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testInfoSchemaWithView() throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE dfs_test.tmp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create view
-          resultSet = statement.executeQuery(
-              "CREATE VIEW testview3 AS SELECT * FROM hive_test.kv");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // show tables on view
-          resultSet = statement.executeQuery("SHOW TABLES like 'testview3'");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "TABLE_SCHEMA=dfs_test.tmp; TABLE_NAME=testview3";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // test record in INFORMATION_SCHEMA.VIEWS
-          resultSet = statement.executeQuery("SELECT * FROM INFORMATION_SCHEMA.VIEWS " +
-              "WHERE TABLE_NAME = 'testview3'");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "TABLE_CATALOG=DRILL; TABLE_SCHEMA=dfs_test.tmp; TABLE_NAME=testview3; VIEW_DEFINITION=SELECT *\nFROM `hive_test`.`kv`";
-          DrillAssert.assertMultiLineStringEquals(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected, result);
-
-          // test record in INFORMATION_SCHEMA.TABLES
-          resultSet = statement.executeQuery("SELECT * FROM INFORMATION_SCHEMA.`TABLES` " +
-              "WHERE TABLE_NAME = 'testview3'");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "TABLE_CATALOG=DRILL; TABLE_SCHEMA=dfs_test.tmp; TABLE_NAME=testview3; TABLE_TYPE=VIEW";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // describe a view
-          resultSet = statement.executeQuery("DESCRIBE dfs_test.tmp.testview3");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected =
-              "COLUMN_NAME=key; DATA_TYPE=INTEGER; IS_NULLABLE=YES\n" +
-              "COLUMN_NAME=value; DATA_TYPE=VARCHAR; IS_NULLABLE=YES";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("drop view testview3");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-          resultSet.close();
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testInfoSchemaWithHiveView() throws Exception {
-    JdbcAssert.withFull("hive_test.`default`")
-        .sql("SELECT * FROM INFORMATION_SCHEMA.VIEWS WHERE TABLE_NAME = 'hiveview'")
-        .returns("TABLE_CATALOG=DRILL; TABLE_SCHEMA=hive_test.default; TABLE_NAME=hiveview; " +
-            "VIEW_DEFINITION=SELECT `kv`.`key`, `kv`.`value` FROM `default`.`kv`");
-  }
-
-  @Test
-  public void testViewWithFullSchemaIdentifier() throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE cp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create a view with full schema identifier
-          resultSet =  statement.executeQuery("CREATE VIEW dfs_test.tmp.testview AS SELECT * FROM hive_test.kv");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "ok=true; summary=View 'testview' created successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // query from view
-          resultSet = statement.executeQuery("SELECT key FROM dfs_test.tmp.testview LIMIT 1");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "key=1";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("drop view dfs_test.tmp.testview");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-          resultSet.close();
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testViewWithPartialSchemaIdentifier() throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE dfs_test");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create a view with partial schema identifier
-          resultSet = statement.executeQuery("CREATE VIEW tmp.testview AS SELECT * FROM hive_test.kv");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "ok=true; summary=View 'testview' created successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // query from view
-          resultSet = statement.executeQuery("SELECT key FROM tmp.testview LIMIT 1");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "key=1";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // change the default schema and query
-          resultSet = statement.executeQuery("USE dfs_test.tmp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          resultSet = statement.executeQuery("SELECT key FROM testview LIMIT 1");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("drop view tmp.testview");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-          resultSet.close();
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testViewResolvingTablesInWorkspaceSchema() throws Exception{
-    JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-          ResultSet resultSet;
-
-          // change default schema
-          resultSet = statement.executeQuery("USE cp");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-
-          // create a view with full schema identifier
-          resultSet = statement.executeQuery(
-              "CREATE VIEW dfs_test.tmp.testViewResolvingTablesInWorkspaceSchema AS " +
-              "SELECT region_id, sales_city FROM `region.json`");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "ok=true; summary=View 'testViewResolvingTablesInWorkspaceSchema' " +
-              "created successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // query from view
-          resultSet = statement.executeQuery(
-              "SELECT region_id FROM dfs_test.tmp.testViewResolvingTablesInWorkspaceSchema LIMIT 1");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "region_id=0";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("drop view dfs_test.tmp.testViewResolvingTablesInWorkspaceSchema");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-          resultSet.close();
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  @Test
-  public void testCreateViewWhenViewAlreadyExists() throws Exception{
-    JdbcAssert.withFull("dfs_test.tmp").withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-
-          // create a view
-          ResultSet resultSet = statement.executeQuery(
-              "CREATE VIEW testCreateViewWhenViewAlreadyExists AS SELECT region_id, sales_city FROM cp.`region.json`");
-          String result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          String expected = "ok=true; summary=View 'testCreateViewWhenViewAlreadyExists' " +
-              "created successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // try to create the view with same name
-          resultSet = statement.executeQuery(
-              "CREATE VIEW testCreateViewWhenViewAlreadyExists AS SELECT region_id FROM cp.`region.json`");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "ok=false; summary=View with given name already exists in current schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          // try creating the view with same name but with a OR REPLACE clause
-          resultSet = statement.executeQuery(
-              "CREATE OR REPLACE VIEW testCreateViewWhenViewAlreadyExists AS SELECT region_id FROM cp.`region.json`");
-          result = JdbcAssert.toString(resultSet).trim();
-          resultSet.close();
-          expected = "ok=true; summary=View 'testCreateViewWhenViewAlreadyExists' " +
-              "replaced successfully in 'dfs_test.tmp' schema";
-          assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected),
-              expected.equals(result));
-
-          resultSet = statement.executeQuery("drop view dfs_test.tmp.testCreateViewWhenViewAlreadyExists");
-          // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-          // race conditions are fixed (not just DRILL-2245 fixes).
-          // nextUntilEnd(resultSet);
-          resultSet.close();
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-
-  private void createViewHelper(Statement statement, String schema, String viewName, String query) throws Exception {
-    ResultSet resultSet = statement.executeQuery(query);
-    String result = JdbcAssert.toString(resultSet).trim();
-    resultSet.close();
-    String expected = String.format("ok=true; summary=View '%s' created successfully in '%s' schema", viewName, schema);
-    assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", result, expected), expected.equals(result));
-  }
-
-  private void queryView(Statement statement, String query, String expectedResult) throws Exception {
-    ResultSet resultSet = statement.executeQuery(query);
-    String actualResult = JdbcAssert.toString(resultSet).trim();
-    resultSet.close();
-    assertTrue(String.format("Generated string:\n%s\ndoes not match:\n%s", actualResult, expectedResult),
-        expectedResult.equals(actualResult));
-  }
-
-  private void dropView(Statement statement, String schema, String viewName) throws Exception {
-    if (schema != null && !schema.isEmpty()) {
-      viewName = schema + "." + viewName;
-    }
-    ResultSet resultSet = statement.executeQuery("drop view innerView");
-    // TODO:  Purge nextUntilEnd(...) and calls when remaining fragment
-    // race conditions are fixed (not just DRILL-2245 fixes).
-    // nextUntilEnd(resultSet);
-    resultSet.close();
-  }
-
-  @Test
-  public void testViewCreatedFromView() throws Exception {
-    final String schema = "dfs_test.tmp";
-    JdbcAssert.withFull(schema).withConnection(new Function<Connection, Void>() {
-      public Void apply(Connection connection) {
-        try {
-          Statement statement = connection.createStatement();
-
-          // create a view
-          String createInnerView = "CREATE VIEW innerView AS SELECT region_id, sales_city FROM cp.`region.json`";
-          String innerViewName = "innerView";
-          createViewHelper(statement, schema, "innerView", createInnerView);
-
-          // create another view from above created view
-          String createOuterView = "CREATE VIEW outerView AS SELECT region_id FROM innerView";
-          String outerViewName = "outerView";
-          createViewHelper(statement, schema, outerViewName, createOuterView);
-
-          // query on outer view
-          String queryView = "SELECT region_id FROM outerView LIMIT 1";
-          String expectedResult = "region_id=0";
-          queryView(statement, queryView, expectedResult);
-
-          dropView(statement, schema, outerViewName);
-          dropView(statement, schema, innerViewName);
-
-          statement.close();
-          return null;
-        } catch (Exception e) {
-          throw new RuntimeException(e);
-        }
-      }
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/f956b727/exec/jdbc/src/test/resources/nation/nation.tbl
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/resources/nation/nation.tbl b/exec/jdbc/src/test/resources/nation/nation.tbl
deleted file mode 100644
index ed3fd5b..0000000
--- a/exec/jdbc/src/test/resources/nation/nation.tbl
+++ /dev/null
@@ -1,25 +0,0 @@
-0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|
-1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|
-2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |
-3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|
-4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|
-5|ETHIOPIA|0|ven packages wake quickly. regu|
-6|FRANCE|3|refully final requests. regular, ironi|
-7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|
-8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|
-9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|
-10|IRAN|4|efully alongside of the slyly final dependencies. |
-11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|
-12|JAPAN|2|ously. final, express gifts cajole a|
-13|JORDAN|4|ic deposits are blithely about the carefully regular pa|
-14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|
-15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|
-16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|
-17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|
-18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|
-19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|
-20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|
-21|VIETNAM|2|hely enticingly express accounts. even, final |
-22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|
-23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|
-24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|