Skip to content

Iceberg does not support drop namespace cascade #3541

@pan3793

Description

@pan3793

The semantic of drop namespace cascade is different between Iceberg and Spark, then run Spark SQL DROP DATABASE xxx CASCADE cause failure if the database is not empty.

Iceberg interface SupportsNamespaces

  /**
   * Drop a namespace. If the namespace exists and was dropped, this will return true.
   *
   * @param namespace a namespace. {@link Namespace}
   * @return true if the namespace was dropped, false otherwise.
   * @throws NamespaceNotEmptyException If the namespace does not empty
   */
  boolean dropNamespace(Namespace namespace) throws NamespaceNotEmptyException;

Spark interface SupportsNamespaces extends CatalogPlugin

  /**
   * Drop a namespace from the catalog, recursively dropping all objects within the namespace.
   * <p>
   * If the catalog implementation does not support this operation, it may throw
   * {@link UnsupportedOperationException}.
   *
   * @param namespace a multi-part namespace
   * @return true if the namespace was dropped
   * @throws NoSuchNamespaceException If the namespace does not exist (optional)
   * @throws UnsupportedOperationException If drop is not a supported operation
   */
  boolean dropNamespace(String[] namespace) throws NoSuchNamespaceException;

The CASCADE logic is handled by DropNamespaceExec

    if (nsCatalog.namespaceExists(ns)) {
      // The default behavior of `SupportsNamespace.dropNamespace()` is cascading,
      // so make sure the namespace to drop is empty.
      if (!cascade) {
        if (catalog.asTableCatalog.listTables(ns).nonEmpty
          || nsCatalog.listNamespaces(ns).nonEmpty) {
          throw QueryExecutionErrors.cannotDropNonemptyNamespaceError(namespace)
        }
      }

      if (!nsCatalog.dropNamespace(ns)) {
        throw QueryExecutionErrors.cannotDropNonemptyNamespaceError(namespace)
      }

Metadata

Metadata

Assignees

No one assigned

    Labels

    Type

    No type

    Projects

    No projects

    Milestone

    No milestone

    Relationships

    None yet

    Development

    No branches or pull requests

    Issue actions