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/15 20:20:31 UTC

[3/3] drill git commit: DRILL-2422: Before creating view make sure no table already exists with given name.

DRILL-2422: Before creating view make sure no table already exists with given name.


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

Branch: refs/heads/master
Commit: 07cca5dba2b19450c35b770aa9da0e9c62fc35f8
Parents: f55e53a
Author: vkorukanti <ve...@gmail.com>
Authored: Wed Apr 15 07:24:31 2015 -0700
Committer: vkorukanti <ve...@gmail.com>
Committed: Wed Apr 15 09:00:24 2015 -0700

----------------------------------------------------------------------
 .../sql/handlers/CreateTableHandler.java        | 15 ++----
 .../planner/sql/handlers/SqlHandlerUtil.java    | 14 ++++++
 .../exec/planner/sql/handlers/ViewHandler.java  | 30 ++++++++----
 .../physical/impl/writer/TestParquetWriter.java | 48 +++++++++++++++++++-
 .../apache/drill/exec/sql/TestViewSupport.java  | 40 +++++++++++++++-
 5 files changed, 126 insertions(+), 21 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/07cca5db/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
index a47cb47..a17a604 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/CreateTableHandler.java
@@ -18,14 +18,11 @@
 package org.apache.drill.exec.planner.sql.handlers;
 
 import java.io.IOException;
-import java.util.List;
 
 import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.tools.Planner;
 import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.ValidationException;
 
-import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.logical.DrillRel;
@@ -36,14 +33,10 @@ import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
-import org.apache.drill.exec.planner.types.DrillFixedRelDataTypeImpl;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.relopt.hep.HepPlanner;
-import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.sql.SqlNode;
 
 public class CreateTableHandler extends DefaultSqlHandler {
@@ -69,9 +62,11 @@ public class CreateTableHandler extends DefaultSqlHandler {
             "Can't create tables in this schema.", drillSchema.getFullSchemaName()));
       }
 
-      String newTblName = sqlCreateTable.getName();
-      if (schema.getTable(newTblName) != null) {
-        return DirectPlan.createDirectPlan(context, false, String.format("Table '%s' already exists.", newTblName));
+      final String newTblName = sqlCreateTable.getName();
+      if (SqlHandlerUtil.getTableFromSchema(drillSchema, newTblName) != null) {
+        throw new ValidationException(
+            String.format("A table or view with given name [%s] already exists in schema [%s]",
+                newTblName, drillSchema.getFullSchemaName()));
       }
 
       log("Optiq Logical", newTblRelNode);

http://git-wip-us.apache.org/repos/asf/drill/blob/07cca5db/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
index 2f6a56d..c347bef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandlerUtil.java
@@ -17,12 +17,15 @@
  */
 package org.apache.drill.exec.planner.sql.handlers;
 
+import net.hydromatic.optiq.Table;
 import net.hydromatic.optiq.tools.Planner;
 import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.ValidationException;
+import org.apache.drill.common.exceptions.DrillException;
 import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.types.DrillFixedRelDataTypeImpl;
+import org.apache.drill.exec.store.AbstractSchema;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptUtil;
 import org.eigenbase.reltype.RelDataType;
@@ -83,4 +86,15 @@ public class SqlHandlerUtil {
 
     return validatedQueryRelNode;
   }
+
+  public static Table getTableFromSchema(AbstractSchema drillSchema, String tblName) throws DrillException {
+    try {
+      return drillSchema.getTable(tblName);
+    } catch (Exception e) {
+      // TODO: Move to better exception types.
+      throw new DrillException(
+          String.format("Failure while trying to check if a table or view with given name [%s] already exists " +
+              "in schema [%s]: %s", tblName, drillSchema.getFullSchemaName(), e.getMessage()), e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/07cca5db/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
index 5c0c2f2..00fc522 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ViewHandler.java
@@ -20,7 +20,9 @@ package org.apache.drill.exec.planner.sql.handlers;
 import java.io.IOException;
 import java.util.List;
 
+import net.hydromatic.optiq.Schema.TableType;
 import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.Table;
 import net.hydromatic.optiq.tools.Planner;
 import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.ValidationException;
@@ -28,18 +30,13 @@ import net.hydromatic.optiq.tools.ValidationException;
 import org.apache.drill.exec.dotdrill.View;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.planner.common.DrillRelOptUtil;
 import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateView;
 import org.apache.drill.exec.planner.sql.parser.SqlDropView;
-import org.apache.drill.exec.planner.types.DrillFixedRelDataTypeImpl;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
-import org.eigenbase.rel.CalcRel;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptUtil;
-import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.sql.SqlNode;
 
 import com.google.common.collect.ImmutableList;
@@ -91,12 +88,27 @@ public abstract class ViewHandler extends AbstractSqlHandler {
 
         View view = new View(createView.getName(), viewSql, newViewRelNode.getRowType(), workspaceSchemaPath);
 
+        final String viewName = view.getName();
+        final Table existingTable = SqlHandlerUtil.getTableFromSchema(drillSchema, viewName);
+
+        if (existingTable != null) {
+          if (existingTable.getJdbcTableType() != TableType.VIEW) {
+            // existing table is not a view
+            throw new ValidationException(
+                String.format("A non-view table with given name [%s] already exists in schema [%s]",
+                    viewName, schemaPath));
+          }
+
+          if (existingTable.getJdbcTableType() == TableType.VIEW && !createView.getReplace()) {
+            // existing table is a view and create view has no "REPLACE" clause
+            throw new ValidationException(
+                String.format("A view with given name [%s] already exists in schema [%s]",
+                    view.getName(), schemaPath));
+          }
+        }
+
         boolean replaced;
         if (drillSchema instanceof WorkspaceSchema) {
-          WorkspaceSchema workspaceSchema = (WorkspaceSchema) drillSchema;
-          if (!createView.getReplace() && workspaceSchema.viewExists(view.getName())) {
-            return DirectPlan.createDirectPlan(context, false, "View with given name already exists in current schema");
-          }
           replaced = ((WorkspaceSchema) drillSchema).createView(view);
         } else {
           return DirectPlan.createDirectPlan(context, false, "Schema provided was not a workspace schema.");

http://git-wip-us.apache.org/repos/asf/drill/blob/07cca5db/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
index df6e43d..89837e7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/writer/TestParquetWriter.java
@@ -342,7 +342,7 @@ public class TestParquetWriter extends BaseTestQuery {
   @Ignore
   @Test
   public void test958_sql_all_columns() throws Exception {
-    compareParquetReadersHyperVector("*",  "dfs.`/tmp/store_sales`");
+    compareParquetReadersHyperVector("*", "dfs.`/tmp/store_sales`");
     compareParquetReadersHyperVector("ss_addr_sk, ss_hdemo_sk", "dfs.`/tmp/store_sales`");
     // TODO - Drill 1388 - this currently fails, but it is an issue with project, not the reader, pulled out the physical plan
     // removed the unneeded project in the plan and ran it against both readers, they outputs matched
@@ -447,6 +447,52 @@ public class TestParquetWriter extends BaseTestQuery {
     }
   }
 
+  @Test // DRILL-2422
+  public void createTableWhenATableWithSameNameAlreadyExists() throws Exception{
+    final String newTblName = "createTableWhenTableAlreadyExists";
+
+    try {
+      test("USE dfs_test.tmp");
+      final String ctas = String.format("CREATE TABLE %s AS SELECT * from cp.`region.json`", newTblName);
+
+      test(ctas);
+
+      testBuilder()
+          .unOrdered()
+          .sqlQuery(ctas)
+          .baselineColumns("ok", "summary")
+          .baselineValues(false,
+              String.format("Error: A table or view with given name [%s] already exists in schema [%s]",
+                  newTblName, "dfs_test.tmp"))
+          .go();
+    } finally {
+      deleteTableIfExists(newTblName);
+    }
+  }
+
+  @Test // DRILL-2422
+  public void createTableWhenAViewWithSameNameAlreadyExists() throws Exception{
+    final String newTblName = "createTableWhenAViewWithSameNameAlreadyExists";
+
+    try {
+      test("USE dfs_test.tmp");
+      final String createView = String.format("CREATE VIEW %s AS SELECT * from cp.`region.json`", newTblName);
+
+      test(createView);
+
+      testBuilder()
+          .unOrdered()
+          .sqlQuery(String.format("CREATE TABLE %s AS SELECT * FROM cp.`employee.json`", newTblName))
+          .baselineColumns("ok", "summary")
+          .baselineValues(false,
+              String.format("Error: A table or view with given name [%s] already exists in schema [%s]",
+                  newTblName, "dfs_test.tmp"))
+          .go();
+    } finally {
+      test("DROP VIEW " + newTblName);
+    }
+  }
+
   @Test // see DRILL-2408
   public void testWriteEmptyFileAfterFlush() throws Exception {
     String outputFile = "testparquetwriter_test_write_empty_file_after_flush";

http://git-wip-us.apache.org/repos/asf/drill/blob/07cca5db/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 b0b2bb8..2ae6991 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
@@ -18,9 +18,11 @@
 package org.apache.drill.exec.sql;
 
 import com.google.common.collect.ImmutableList;
+import org.apache.commons.io.FileUtils;
 import org.junit.Ignore;
 import org.junit.Test;
 
+import java.io.File;
 import java.util.List;
 
 public class TestViewSupport extends TestBaseViewSupport {
@@ -275,7 +277,8 @@ public class TestViewSupport extends TestBaseViewSupport {
           .sqlQuery(createViewSql)
           .unOrdered()
           .baselineColumns("ok", "summary")
-          .baselineValues(false, "View with given name already exists in current schema")
+          .baselineValues(false,
+              String.format("Error: A view with given name [%s] already exists in schema [%s]", viewName, TEMP_SCHEMA))
           .go();
 
       // Try creating the view with same name in same schema, but with CREATE OR REPLACE VIEW clause
@@ -301,6 +304,41 @@ public class TestViewSupport extends TestBaseViewSupport {
     }
   }
 
+  @Test // DRILL-2422
+  public void createViewWhenATableWithSameNameAlreadyExists() throws Exception {
+    final String tableName = generateViewName();
+
+    try {
+      final String tableDef1 = "SELECT region_id, sales_city FROM cp.`region.json`";
+
+      test(String.format("CREATE TABLE %s.%s as %s", TEMP_SCHEMA, tableName, tableDef1));
+
+      // Try to create the view with same name in same schema.
+      final String createViewSql = String.format("CREATE VIEW %s.`%s` AS %s", TEMP_SCHEMA, tableName, tableDef1);
+      testBuilder()
+          .sqlQuery(createViewSql)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(false,
+              String.format("Error: A non-view table with given name [%s] already exists in schema [%s]",
+                  tableName, TEMP_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`";
+      testBuilder()
+          .sqlQuery(String.format("CREATE OR REPLACE VIEW %s.`%s` AS %s", TEMP_SCHEMA, tableName, viewDef2))
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(false,
+              String.format("Error: A non-view table with given name [%s] already exists in schema [%s]",
+                  tableName, TEMP_SCHEMA))
+          .go();
+    } finally {
+      FileUtils.deleteQuietly(new File(getDfsTestTmpSchemaLocation(), tableName));
+    }
+  }
+
   @Test
   public void infoSchemaWithView() throws Exception {
     final String viewName = generateViewName();