You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2021/04/15 04:11:12 UTC

[calcite] 04/08: [CALCITE-4535] ServerDdlExecutor cannot execute DROP commands with qualified object names (Vladimir Ozerov)

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

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

commit 66c0d13a6b2fd4c04f049aace71a64d3d378b0dc
Author: devozerov <pp...@gmail.com>
AuthorDate: Sun Mar 14 11:02:59 2021 +0300

    [CALCITE-4535] ServerDdlExecutor cannot execute DROP commands with qualified object names (Vladimir Ozerov)
    
    Close apache/calcite#2371
---
 .../apache/calcite/server/ServerDdlExecutor.java   | 73 ++++++++++------------
 .../java/org/apache/calcite/test/ServerTest.java   | 23 +++++++
 2 files changed, 55 insertions(+), 41 deletions(-)

diff --git a/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java b/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java
index fd636d9..51c5b5f 100644
--- a/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java
+++ b/server/src/main/java/org/apache/calcite/server/ServerDdlExecutor.java
@@ -47,6 +47,7 @@ import org.apache.calcite.schema.impl.ViewTableMacro;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlDataTypeSpec;
 import org.apache.calcite.sql.SqlIdentifier;
+import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlNodeList;
@@ -62,7 +63,6 @@ import org.apache.calcite.sql.ddl.SqlCreateSchema;
 import org.apache.calcite.sql.ddl.SqlCreateTable;
 import org.apache.calcite.sql.ddl.SqlCreateType;
 import org.apache.calcite.sql.ddl.SqlCreateView;
-import org.apache.calcite.sql.ddl.SqlDropMaterializedView;
 import org.apache.calcite.sql.ddl.SqlDropObject;
 import org.apache.calcite.sql.ddl.SqlDropSchema;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
@@ -281,41 +281,54 @@ public class ServerDdlExecutor extends DdlExecutorImpl {
    * {@code DROP VIEW} commands. */
   public void execute(SqlDropObject drop,
       CalcitePrepare.Context context) {
-    final List<String> path = context.getDefaultSchemaPath();
-    CalciteSchema schema = context.getRootSchema();
-    for (String p : path) {
-      schema = schema.getSubSchema(p, true);
-    }
-    final boolean existed;
+    final Pair<CalciteSchema, String> pair = schema(context, false, drop.name);
+    CalciteSchema schema = pair.left;
+    String objectName = pair.right;
+    assert objectName != null;
+
+    boolean schemaExists = schema != null;
+
+    boolean existed;
     switch (drop.getKind()) {
     case DROP_TABLE:
     case DROP_MATERIALIZED_VIEW:
-      existed = schema.removeTable(drop.name.getSimple());
-      if (!existed && !drop.ifExists) {
+      Table materializedView = schemaExists && drop.getKind() == SqlKind.DROP_MATERIALIZED_VIEW
+          ? schema.plus().getTable(objectName) : null;
+
+      existed = schemaExists && schema.removeTable(objectName);
+      if (existed) {
+        if (materializedView instanceof Wrapper) {
+          ((Wrapper) materializedView).maybeUnwrap(MaterializationKey.class)
+              .ifPresent(materializationKey -> {
+                MaterializationService.instance()
+                    .removeMaterialization(materializationKey);
+              });
+        }
+      } else if (!drop.ifExists) {
         throw SqlUtil.newContextException(drop.name.getParserPosition(),
-            RESOURCE.tableNotFound(drop.name.getSimple()));
+            RESOURCE.tableNotFound(objectName));
       }
       break;
     case DROP_VIEW:
       // Not quite right: removes any other functions with the same name
-      existed = schema.removeFunction(drop.name.getSimple());
+      existed = schemaExists && schema.removeFunction(objectName);
       if (!existed && !drop.ifExists) {
         throw SqlUtil.newContextException(drop.name.getParserPosition(),
-            RESOURCE.viewNotFound(drop.name.getSimple()));
+            RESOURCE.viewNotFound(objectName));
       }
       break;
     case DROP_TYPE:
-      existed = schema.removeType(drop.name.getSimple());
+      existed = schemaExists && schema.removeType(objectName);
       if (!existed && !drop.ifExists) {
         throw SqlUtil.newContextException(drop.name.getParserPosition(),
-            RESOURCE.typeNotFound(drop.name.getSimple()));
+            RESOURCE.typeNotFound(objectName));
       }
       break;
     case DROP_FUNCTION:
-      existed = schema.removeFunction(drop.name.getSimple());
+      existed = schemaExists && schema.removeFunction(objectName);
       if (!existed && !drop.ifExists) {
         throw SqlUtil.newContextException(drop.name.getParserPosition(),
-            RESOURCE.functionNotFound(drop.name.getSimple()));
+            RESOURCE.functionNotFound(objectName));
       }
       break;
     case OTHER_DDL:
@@ -356,24 +369,6 @@ public class ServerDdlExecutor extends DdlExecutorImpl {
             sql, schemaPath, pair.right, true, true);
   }
 
-  /** Executes a {@code DROP MATERIALIZED VIEW} command. */
-  public void execute(SqlDropMaterializedView drop,
-      CalcitePrepare.Context context) {
-    final Pair<CalciteSchema, String> pair = schema(context, true, drop.name);
-    final Table table = pair.left.plus().getTable(pair.right);
-    if (table != null) {
-      // Materialized view exists.
-      execute((SqlDropObject) drop, context);
-      if (table instanceof Wrapper) {
-        ((Wrapper) table).maybeUnwrap(MaterializationKey.class)
-            .ifPresent(materializationKey -> {
-              MaterializationService.instance()
-                  .removeMaterialization(materializationKey);
-            });
-      }
-    }
-  }
-
   /** Executes a {@code CREATE SCHEMA} command. */
   public void execute(SqlCreateSchema create,
       CalcitePrepare.Context context) {
@@ -395,15 +390,11 @@ public class ServerDdlExecutor extends DdlExecutorImpl {
   /** Executes a {@code DROP SCHEMA} command. */
   public void execute(SqlDropSchema drop,
       CalcitePrepare.Context context) {
-    final List<String> path = context.getDefaultSchemaPath();
-    CalciteSchema schema = context.getRootSchema();
-    for (String p : path) {
-      schema = schema.getSubSchema(p, true);
-    }
-    final boolean existed = schema.removeSubSchema(drop.name.getSimple());
+    final Pair<CalciteSchema, String> pair = schema(context, false, drop.name);
+    final boolean existed = pair.left != null && pair.left.removeSubSchema(pair.right);
     if (!existed && !drop.ifExists) {
       throw SqlUtil.newContextException(drop.name.getParserPosition(),
-          RESOURCE.schemaNotFound(drop.name.getSimple()));
+          RESOURCE.schemaNotFound(pair.right));
     }
   }
 
diff --git a/server/src/test/java/org/apache/calcite/test/ServerTest.java b/server/src/test/java/org/apache/calcite/test/ServerTest.java
index 559e949..43fa8c9 100644
--- a/server/src/test/java/org/apache/calcite/test/ServerTest.java
+++ b/server/src/test/java/org/apache/calcite/test/ServerTest.java
@@ -57,6 +57,7 @@ import static org.junit.jupiter.api.Assertions.assertArrayEquals;
 import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -532,4 +533,26 @@ class ServerTest {
       }
     }
   }
+
+  @Test public void testDropWithFullyQualifiedNameWhenSchemaDoesntExist() throws Exception {
+    try (Connection c = connect();
+         Statement s = c.createStatement()) {
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "schema", "Schema");
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "table", "Table");
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "materialized view", "Table");
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "view", "View");
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "type", "Type");
+      checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(s, "function", "Function");
+    }
+  }
+
+  private void checkDropWithFullyQualifiedNameWhenSchemaDoesntExist(
+      Statement statement, String objectType, String objectTypeInErrorMessage) throws Exception {
+    SQLException e = assertThrows(SQLException.class, () ->
+        statement.execute("drop " + objectType + " s.o"),
+        "expected error because the object doesn't exist");
+    assertThat(e.getMessage(), containsString(objectTypeInErrorMessage + " 'O' not found"));
+
+    statement.execute("drop " + objectType + " if exists s.o");
+  }
 }