You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by GitBox <gi...@apache.org> on 2018/11/29 20:27:30 UTC

[GitHub] asfgit closed pull request #1557: DRILL-6863: Drop table is not working if path within workspace starts…

asfgit closed pull request #1557: DRILL-6863: Drop table is not working if path within workspace starts…
URL: https://github.com/apache/drill/pull/1557
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 51aad406c48..154bf8cbb41 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.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 @@ void disallowTemporaryTables() {
 
     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 928a849884a..82a111a1d97 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.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 CreateTableHandler(SqlHandlerConfig config, Pointer<String> textPlan) {
   @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 c17ac20840a..4f1a7596e3b 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.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 DropTableHandler(SqlHandlerConfig config) {
   @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 e86b90a091a..012315f0559 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.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 CreateView(SqlHandlerConfig config) {
     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 PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConv
 
       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 DropView(SqlHandlerConfig config) {
     @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 63b54975550..493278c27b1 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 Table getTable(String tableName) {
       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 6313d74a048..0c9067c5ffb 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.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 @@
 @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 void testDropJsonTable() throws Exception {
 
   @Test
   public void testDropParquetTable() throws Exception {
-    test("use dfs.tmp");
     final String tableName = "simple_json";
 
     // create a parquet table
@@ -72,7 +75,6 @@ public void testDropParquetTable() throws Exception {
 
   @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 void testDropTextTable() throws Exception {
 
   @Test
   public void testNonHomogenousDrop() throws Exception {
-    test("use dfs.tmp");
     final String tableName = "homogenous_table";
 
     // create a parquet table
@@ -127,7 +128,7 @@ public void testNonHomogenousDrop() throws Exception {
     // 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 void testNonHomogenousDrop() throws Exception {
 
     // 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 void testDropOnImmutableSchema() throws Exception {
   @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 void testDropTableIfExistsWhileTableExists() throws Exception {
   @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 void testDropTableIfExistsWhileTableDoesNotExist() throws Exception {
   @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 void testDropTableIfExistsWhileItIsAView() throws Exception {
       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 49ec6d85881..6e9f4567d04 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 void testCTASWithEmptyJson() throws Exception {
     }
   }
 
+  @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 55e0b59eb9f..ba8beceaa65 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 void testJoinTemporaryWithPersistentTable() throws Exception {
         .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 e7de7abb1f3..a0773bc95ce 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 void selectFromViewCreatedOnCalcite1_4() throws Exception {
         .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);
+    }
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services