You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by vi...@apache.org on 2018/11/29 20:27:12 UTC

[drill] 03/05: DRILL-6863: Drop table is not working if path within workspace starts with "/"

This is an automated email from the ASF dual-hosted git repository.

vitalii pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git

commit 6e3f0f7df69228bd1a3e95b83cfd8399b4722d51
Author: Bohdan Kazydub <bo...@gmail.com>
AuthorDate: Wed Nov 28 16:51:18 2018 +0200

    DRILL-6863: Drop table is not working if path within workspace starts with "/"
    
    - Made workspace to be honored when table/view name starts with "/" for DROP TABLE, DROP VIEW, CREATE VIEW and SELECT from view queries;
    - Made "/{name}" and "{name}" to be equivalent names (the leading "/" is removed) when creating temporary tables so that SELECT ... FROM "/{name}" ... and SELECT ... FROM "{name}" ... produce the same results and behave as regular tables in the context.
    
    closes #1557
---
 .../drill/exec/planner/sql/SqlConverter.java       |  4 +-
 .../planner/sql/handlers/CreateTableHandler.java   |  3 +-
 .../planner/sql/handlers/DropTableHandler.java     |  3 +-
 .../exec/planner/sql/handlers/ViewHandler.java     |  7 +--
 .../exec/store/dfs/WorkspaceSchemaFactory.java     |  3 +-
 .../test/java/org/apache/drill/TestDropTable.java  | 47 +++++++++++------
 .../java/org/apache/drill/exec/sql/TestCTAS.java   | 32 ++++++++++++
 .../java/org/apache/drill/exec/sql/TestCTTAS.java  | 61 ++++++++++++++++++++++
 .../org/apache/drill/exec/sql/TestViewSupport.java | 49 +++++++++++++++++
 9 files changed, 185 insertions(+), 24 deletions(-)

diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
index 51aad40..154bf8c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SqlConverter.java
@@ -82,6 +82,7 @@ import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PlannerSettings;
 import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.store.dfs.FileSelection;
 import static org.apache.calcite.util.Static.RESOURCE;
 
 import org.apache.drill.shaded.guava.com.google.common.base.Joiner;
@@ -624,7 +625,8 @@ public class SqlConverter {
 
     private List<String> getTemporaryNames(List<String> names) {
       if (mightBeTemporaryTable(names, session.getDefaultSchemaPath(), drillConfig)) {
-        String temporaryTableName = session.resolveTemporaryTableName(names.get(names.size() - 1));
+        String tableName = FileSelection.removeLeadingSlash(names.get(names.size() - 1));
+        String temporaryTableName = session.resolveTemporaryTableName(tableName);
         if (temporaryTableName != null) {
           List<String> temporaryNames = new ArrayList<>(SchemaUtilites.getSchemaPathAsList(temporarySchema));
           temporaryNames.add(temporaryTableName);
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 928a849..82a111a 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
@@ -55,6 +55,7 @@ import org.apache.drill.exec.planner.sql.DrillSqlOperator;
 import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateTable;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
@@ -72,7 +73,7 @@ public class CreateTableHandler extends DefaultSqlHandler {
   @Override
   public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException, ForemanSetupException {
     final SqlCreateTable sqlCreateTable = unwrap(sqlNode, SqlCreateTable.class);
-    final String originalTableName = sqlCreateTable.getName();
+    final String originalTableName = FileSelection.removeLeadingSlash(sqlCreateTable.getName());
 
     final ConvertedRelNode convertedRelNode = validateAndConvert(sqlCreateTable.getQuery());
     final RelDataType validatedRowType = convertedRelNode.getValidatedRowType();
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
index c17ac20..4f1a759 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DropTableHandler.java
@@ -34,6 +34,7 @@ import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.parser.SqlDropTable;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.dfs.FileSelection;
 
 // SqlHandler for dropping a table.
 public class DropTableHandler extends DefaultSqlHandler {
@@ -56,7 +57,7 @@ public class DropTableHandler extends DefaultSqlHandler {
   @Override
   public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException {
     SqlDropTable dropTableNode = ((SqlDropTable) sqlNode);
-    String originalTableName = dropTableNode.getName();
+    String originalTableName = FileSelection.removeLeadingSlash(dropTableNode.getName());
     SchemaPlus defaultSchema = config.getConverter().getDefaultSchema();
     List<String> tableSchema = dropTableNode.getSchema();
     DrillConfig drillConfig = context.getConfig();
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 e86b90a..012315f 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
@@ -35,6 +35,7 @@ import org.apache.drill.exec.planner.sql.SchemaUtilites;
 import org.apache.drill.exec.planner.sql.parser.SqlCreateView;
 import org.apache.drill.exec.planner.sql.parser.SqlDropView;
 import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.dfs.FileSelection;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.sql.SqlNode;
@@ -60,7 +61,7 @@ public abstract class ViewHandler extends DefaultSqlHandler {
     public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException, ForemanSetupException {
       SqlCreateView createView = unwrap(sqlNode, SqlCreateView.class);
 
-      final String newViewName = createView.getName();
+      final String newViewName = FileSelection.removeLeadingSlash(createView.getName());
 
       // Disallow temporary tables usage in view definition
       config.getConverter().disallowTemporaryTables();
@@ -87,7 +88,7 @@ public abstract class ViewHandler extends DefaultSqlHandler {
 
       final boolean replaced = drillSchema.createView(view);
       final String summary = String.format("View '%s' %s successfully in '%s' schema",
-          createView.getName(), replaced ? "replaced" : "created", drillSchema.getFullSchemaName());
+          newViewName, replaced ? "replaced" : "created", drillSchema.getFullSchemaName());
 
       return DirectPlan.createDirectPlan(context, true, summary);
     }
@@ -155,7 +156,7 @@ public abstract class ViewHandler extends DefaultSqlHandler {
     @Override
     public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException, ForemanSetupException {
       SqlDropView dropView = unwrap(sqlNode, SqlDropView.class);
-      final String viewName = dropView.getName();
+      final String viewName = FileSelection.removeLeadingSlash(dropView.getName());
       final AbstractSchema drillSchema =
           SchemaUtilites.resolveToMutableDrillSchema(context.getNewDefaultSchema(), dropView.getSchemaPath());
 
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index 63b5497..493278c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -518,7 +518,8 @@ public class WorkspaceSchemaFactory {
       List<DotDrillFile> files = Collections.emptyList();
       try {
         try {
-          files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()), tableName, DotDrillType.VIEW);
+          files = DotDrillUtil.getDotDrills(getFS(), new Path(config.getLocation()),
+              FileSelection.removeLeadingSlash(tableName), DotDrillType.VIEW);
         } catch (AccessControlException e) {
           if (!schemaConfig.getIgnoreAuthErrors()) {
             logger.debug(e.getMessage());
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
index 6313d74..0c9067c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestDropTable.java
@@ -21,6 +21,7 @@ import org.apache.drill.categories.SqlTest;
 import org.apache.drill.categories.UnlikelyTest;
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.hadoop.fs.Path;
+import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Assert;
 import org.junit.experimental.categories.Category;
@@ -28,16 +29,19 @@ import org.junit.experimental.categories.Category;
 @Category(SqlTest.class)
 public class TestDropTable extends PlanTestBase {
 
-  private static final String CREATE_SIMPLE_TABLE = "create table %s as select 1 from cp.`employee.json`";
-  private static final String CREATE_SIMPLE_VIEW = "create view %s as select 1 from cp.`employee.json`";
-  private static final String DROP_TABLE = "drop table %s";
-  private static final String DROP_TABLE_IF_EXISTS = "drop table if exists %s";
-  private static final String DROP_VIEW_IF_EXISTS = "drop view if exists %s";
-  private static final String BACK_TICK = "`";
+  private static final String CREATE_SIMPLE_TABLE = "create table `%s` as select 1 from cp.`employee.json`";
+  private static final String CREATE_SIMPLE_VIEW = "create view `%s` as select 1 from cp.`employee.json`";
+  private static final String DROP_TABLE = "drop table `%s`";
+  private static final String DROP_TABLE_IF_EXISTS = "drop table if exists `%s`";
+  private static final String DROP_VIEW_IF_EXISTS = "drop view if exists `%s`";
+
+  @BeforeClass
+  public static void setup() throws Exception {
+    test("use dfs.tmp");
+  }
 
   @Test
   public void testDropJsonTable() throws Exception {
-    test("use dfs.tmp");
     test("alter session set `store.format` = 'json'");
 
     final String tableName = "simple_json";
@@ -55,7 +59,6 @@ public class TestDropTable extends PlanTestBase {
 
   @Test
   public void testDropParquetTable() throws Exception {
-    test("use dfs.tmp");
     final String tableName = "simple_json";
 
     // create a parquet table
@@ -72,7 +75,6 @@ public class TestDropTable extends PlanTestBase {
 
   @Test
   public void testDropTextTable() throws Exception {
-    test("use dfs.tmp");
     test("alter session set `store.format` = 'csv'");
     final String csvTable = "simple_csv";
 
@@ -118,7 +120,6 @@ public class TestDropTable extends PlanTestBase {
 
   @Test
   public void testNonHomogenousDrop() throws Exception {
-    test("use dfs.tmp");
     final String tableName = "homogenous_table";
 
     // create a parquet table
@@ -127,7 +128,7 @@ public class TestDropTable extends PlanTestBase {
     // create a json table within the same directory
     test("alter session set `store.format` = 'json'");
     final String nestedJsonTable = tableName + Path.SEPARATOR + "json_table";
-    test(CREATE_SIMPLE_TABLE, BACK_TICK + nestedJsonTable + BACK_TICK);
+    test(CREATE_SIMPLE_TABLE, nestedJsonTable);
 
     boolean dropFailed = false;
     // this should fail, because the directory contains non-homogenous files
@@ -142,7 +143,7 @@ public class TestDropTable extends PlanTestBase {
 
     // drop the individual json table
     testBuilder()
-        .sqlQuery(DROP_TABLE, BACK_TICK + nestedJsonTable + BACK_TICK)
+        .sqlQuery(DROP_TABLE, nestedJsonTable)
         .unOrdered()
         .baselineColumns("ok", "summary")
         .baselineValues(true, String.format("Table [%s] dropped", nestedJsonTable))
@@ -174,7 +175,6 @@ public class TestDropTable extends PlanTestBase {
   @Category(UnlikelyTest.class)
   public void testDropTableIfExistsWhileTableExists() throws Exception {
     final String existentTableName = "test_table_exists";
-    test("use dfs.tmp");
 
     // successful dropping of existent table
     test(CREATE_SIMPLE_TABLE, existentTableName);
@@ -190,7 +190,6 @@ public class TestDropTable extends PlanTestBase {
   @Category(UnlikelyTest.class)
   public void testDropTableIfExistsWhileTableDoesNotExist() throws Exception {
     final String nonExistentTableName = "test_table_not_exists";
-    test("use dfs.tmp");
 
     // dropping of non existent table without error
     testBuilder()
@@ -205,9 +204,7 @@ public class TestDropTable extends PlanTestBase {
   @Category(UnlikelyTest.class)
   public void testDropTableIfExistsWhileItIsAView() throws Exception {
     final String viewName = "test_view";
-    try{
-      test("use dfs.tmp");
-
+    try {
       // dropping of non existent table without error if the view with such name is existed
       test(CREATE_SIMPLE_VIEW, viewName);
       testBuilder()
@@ -220,4 +217,20 @@ public class TestDropTable extends PlanTestBase {
       test(DROP_VIEW_IF_EXISTS, viewName);
     }
   }
+
+  @Test
+  public void testDropTableNameStartsWithSlash() throws Exception {
+    String tableName = "test_table_starts_with_slash_drop";
+    try {
+      test(CREATE_SIMPLE_TABLE, tableName);
+      testBuilder()
+          .sqlQuery(DROP_TABLE, "/" + tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Table [%s] dropped", tableName))
+          .go();
+    } finally {
+      test(DROP_TABLE_IF_EXISTS, tableName);
+    }
+  }
 }
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
index 49ec6d8..6e9f456 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTAS.java
@@ -325,6 +325,38 @@ public class TestCTAS extends BaseTestQuery {
     }
   }
 
+  @Test
+  public void testTableIsCreatedWithinWorkspace() throws Exception {
+    String tableName = "table_created_within_workspace";
+    try {
+      test("CREATE TABLE `%s`.`%s` AS SELECT * FROM cp.`region.json`", DFS_TMP_SCHEMA, "/" + tableName);
+      testBuilder()
+          .sqlQuery("SELECT region_id FROM `%s`.`%s` LIMIT 1", DFS_TMP_SCHEMA, tableName)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+    } finally {
+      test("DROP TABLE IF EXISTS `%s`.`%s`", DFS_TMP_SCHEMA, tableName);
+    }
+  }
+
+  @Test
+  public void testTableIsFoundWithinWorkspaceWhenNameStartsWithSlash() throws Exception {
+    String tableName = "table_found_within_workspace";
+    try {
+      test("CREATE TABLE `%s`.`%s` AS SELECT * FROM cp.`region.json`", DFS_TMP_SCHEMA, tableName);
+      testBuilder()
+          .sqlQuery("SELECT region_id FROM `%s`.`%s` LIMIT 1", DFS_TMP_SCHEMA, "/" + tableName)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+    } finally {
+      test("DROP TABLE IF EXISTS `%s`.`%s`", DFS_TMP_SCHEMA, tableName);
+    }
+  }
+
   private static void ctasErrorTestHelper(final String ctasSql, final String expErrorMsg) throws Exception {
     final String createTableSql = String.format(ctasSql, "testTableName");
     errorMsgTestHelper(createTableSql, expErrorMsg);
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
index 55e0b59..ba8bece 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestCTTAS.java
@@ -446,6 +446,67 @@ public class TestCTTAS extends BaseTestQuery {
         .go();
   }
 
+  @Test
+  public void testTemporaryTableWithAndWithoutLeadingSlashAreTheSame() throws Exception {
+    String tablename = "table_with_and_without_slash_create";
+
+    try {
+      test("CREATE TEMPORARY TABLE %s AS SELECT * FROM cp.`region.json`", tablename);
+
+      expectUserRemoteExceptionWithMessage(
+          String.format("VALIDATION ERROR: A table or view with given name [%s] already exists in schema [%s]",
+          tablename, DFS_TMP_SCHEMA));
+
+      test(String.format("CREATE TEMPORARY TABLE `%s` AS SELECT * FROM cp.`employee.json`", "/" + tablename));
+    } finally {
+      test("DROP TABLE IF EXISTS %s", tablename);
+    }
+  }
+
+  @Test
+  public void testSelectFromTemporaryTableWithAndWithoutLeadingSlash() throws Exception {
+    String tablename = "select_from_table_with_and_without_slash";
+
+    try {
+      test("CREATE TEMPORARY TABLE %s AS SELECT * FROM cp.`region.json`", tablename);
+
+      String query = "SELECT region_id FROM `%s` LIMIT 1";
+
+      testBuilder()
+          .sqlQuery(query, tablename)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+
+      testBuilder()
+          .sqlQuery(query, "/" + tablename)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+    } finally {
+      test("DROP TABLE IF EXISTS %s", tablename);
+    }
+  }
+
+  @Test
+  public void testDropTemporaryTableNameStartsWithSlash() throws Exception {
+    String tableName = "table_starts_with_slash_drop";
+
+    try {
+      test("CREATE TEMPORARY TABLE `%s` AS SELECT 1 FROM cp.`employee.json`", tableName);
+      testBuilder()
+          .sqlQuery("DROP TABLE `%s`", "/" + tableName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true, String.format("Temporary table [%s] dropped", tableName))
+          .go();
+    } finally {
+      test("DROP TABLE IF EXISTS %s", tableName);
+    }
+  }
+
   private void expectUserRemoteExceptionWithMessage(String message) {
     thrown.expect(UserRemoteException.class);
     thrown.expectMessage(containsString(message));
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 e7de7ab..a0773bc 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
@@ -751,4 +751,53 @@ public class TestViewSupport extends TestBaseViewSupport {
         .baselineValues("HeadQuarters")
         .go();
   }
+
+  @Test
+  public void testDropViewNameStartsWithSlash() throws Exception {
+    String viewName = "view_name_starts_with_slash_drop";
+    try {
+      test("CREATE VIEW `%s`.`%s` AS SELECT * FROM cp.`region.json`", DFS_TMP_SCHEMA, viewName);
+      testBuilder()
+          .sqlQuery("DROP VIEW `%s`.`%s`", DFS_TMP_SCHEMA, "/" + viewName)
+          .unOrdered()
+          .baselineColumns("ok", "summary")
+          .baselineValues(true,
+              String.format("View [%s] deleted successfully from schema [%s].", viewName, DFS_TMP_SCHEMA))
+          .go();
+    } finally {
+      test("DROP VIEW IF EXISTS `%s`.`%s`", DFS_TMP_SCHEMA, viewName);
+    }
+  }
+
+  @Test
+  public void testViewIsCreatedWithinWorkspace() throws Exception {
+    String viewName = "view_created_within_workspace";
+    try {
+      test("CREATE VIEW `%s`.`%s` AS SELECT * FROM cp.`region.json`", DFS_TMP_SCHEMA, "/" + viewName);
+      testBuilder()
+          .sqlQuery("SELECT region_id FROM `%s`.`%s` LIMIT 1", DFS_TMP_SCHEMA, viewName)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+    } finally {
+      test("DROP VIEW IF EXISTS `%s`.`%s`", DFS_TMP_SCHEMA, viewName);
+    }
+  }
+
+  @Test
+  public void testViewIsFoundWithinWorkspaceWhenNameStartsWithSlash() throws Exception {
+    String viewName = "view_found_within_workspace";
+    try {
+      test("CREATE VIEW `%s`.`%s` AS SELECT * FROM cp.`region.json`", DFS_TMP_SCHEMA, viewName);
+      testBuilder()
+          .sqlQuery("SELECT region_id FROM `%s`.`%s` LIMIT 1", DFS_TMP_SCHEMA, "/" + viewName)
+          .unOrdered()
+          .baselineColumns("region_id")
+          .baselineValues(0L)
+          .go();
+    } finally {
+      test("DROP VIEW IF EXISTS `%s`.`%s`", DFS_TMP_SCHEMA, viewName);
+    }
+  }
 }