Browse Source

HDFS-6625. Remove the Delete Encryption Zone function (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1607507 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 years ago
parent
commit
2a3bccddd9
15 changed files with 30 additions and 407 deletions
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt
  2. 0 11
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
  3. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java
  4. 0 21
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java
  5. 0 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
  6. 0 15
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
  7. 0 12
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  8. 0 23
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
  9. 7 66
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  10. 0 5
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
  11. 0 50
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java
  12. 0 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  13. 0 7
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto
  14. 19 111
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesAPI.java
  15. 2 72
      hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCryptoConf.xml

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES-fs-encryption.txt

@@ -30,6 +30,8 @@ fs-encryption (Unreleased)
 
     HDFS-6605. Client server negotiation of cipher suite. (wang)
 
+    HDFS-6625. Remove the Delete Encryption Zone function (clamb)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -2848,17 +2848,6 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
     }
   }
 
-  public void deleteEncryptionZone(String src) throws IOException {
-    checkOpen();
-    try {
-      namenode.deleteEncryptionZone(src);
-    } catch (RemoteException re) {
-      throw re.unwrapRemoteException(AccessControlException.class,
-                                     SafeModeException.class,
-                                     UnresolvedPathException.class);
-    }
-  }
-
   public List<EncryptionZone> listEncryptionZones() throws IOException {
     checkOpen();
     try {

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java

@@ -1804,11 +1804,6 @@ public class DistributedFileSystem extends FileSystem {
     dfs.createEncryptionZone(getPathName(path), keyId);
   }
 
-  /* HDFS only */
-  public void deleteEncryptionZone(Path path) throws IOException {
-    dfs.deleteEncryptionZone(getPathName(path));
-  }
-
   /* HDFS only */
   public List<EncryptionZone> listEncryptionZones() throws IOException {
     return dfs.listEncryptionZones();

+ 0 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java

@@ -258,27 +258,6 @@ public class HdfsAdmin {
     dfs.createEncryptionZone(path, keyId);
   }
 
-  /**
-   * Delete the encryption zone rooted at path. Path must refer to an existing,
-   * empty directory. Otherwise, an IOException is thrown. This method removes
-   * those extended attributes on the directory which indicate that it is part
-   * of an encryption zone. Following successful completion of this call, any
-   * new files created in the directory (or it's children) will not be
-   * encrypted. The directory is not removed by this method.
-   *
-   * @param path The path of the root of the encryption zone.
-   *
-   * @throws IOException if there was a general IO exception
-   *
-   * @throws AccessControlException if the caller does not have access to path
-   *
-   * @throws FileNotFoundException if the path does not exist
-   */
-  public void deleteEncryptionZone(Path path)
-    throws IOException, AccessControlException, FileNotFoundException {
-    dfs.deleteEncryptionZone(path);
-  }
-
   /**
    * Return a list of all {@EncryptionZone}s in the HDFS hierarchy which are
    * visible to the caller. If the caller is the HDFS admin, then the returned

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java

@@ -1266,13 +1266,6 @@ public interface ClientProtocol {
   public void createEncryptionZone(String src, String keyId)
     throws IOException;
 
-  /**
-   * Delete an encryption zone
-   */
-  @AtMostOnce
-  public void deleteEncryptionZone(String src)
-    throws IOException;
-
   /**
    * Return a list of all {@EncryptionZone}s in the HDFS hierarchy which are
    * visible to the caller. If the caller is the HDFS admin, then the returned

+ 0 - 15
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java

@@ -173,14 +173,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.LocatedBlockProto;
-import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
@@ -1297,18 +1294,6 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
     }
   }
 
-  @Override
-  public DeleteEncryptionZoneResponseProto deleteEncryptionZone(
-    RpcController controller, DeleteEncryptionZoneRequestProto req)
-    throws ServiceException {
-    try {
-      server.deleteEncryptionZone(req.getSrc());
-      return DeleteEncryptionZoneResponseProto.newBuilder().build();
-    } catch (IOException e) {
-      throw new ServiceException(e);
-    }
-  }
-
   @Override
   public ListEncryptionZonesResponseProto listEncryptionZones(
     RpcController controller, ListEncryptionZonesRequestProto req)

+ 0 - 12
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -146,7 +146,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetTim
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.DeleteEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
@@ -1301,17 +1300,6 @@ public class ClientNamenodeProtocolTranslatorPB implements
     }
   }
 
-  @Override
-  public void deleteEncryptionZone(String src) throws IOException {
-    final DeleteEncryptionZoneRequestProto req =
-      DeleteEncryptionZoneRequestProto.newBuilder().setSrc(src).build();
-    try {
-      rpcProxy.deleteEncryptionZone(null, req);
-    } catch (ServiceException e) {
-      throw ProtobufHelper.getRemoteException(e);
-    }
-  }
-
   @Override
   public List<EncryptionZone> listEncryptionZones() throws IOException {
     final ListEncryptionZonesRequestProto req =

+ 0 - 23
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -2716,29 +2716,6 @@ public class FSDirectory implements Closeable {
     }
   }
 
-  List<XAttr> deleteEncryptionZone(String src)
-    throws IOException {
-    writeLock();
-    try {
-      if (isNonEmptyDirectory(src)) {
-        throw new IOException(
-          "Attempt to delete an encryption zone for a non-empty directory.");
-      }
-      final XAttr keyIdXAttr =
-        XAttrHelper.buildXAttr(CRYPTO_XATTR_ENCRYPTION_ZONE, null);
-      List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
-      xattrs.add(keyIdXAttr);
-      final List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(src, xattrs);
-      if (removedXAttrs == null || removedXAttrs.isEmpty()) {
-        throw new IOException(
-          src + " does not appear to be the root of an encryption zone");
-      }
-      return removedXAttrs;
-    } finally {
-      writeUnlock();
-    }
-  }
-
   /**
    * Set the FileEncryptionInfo for an INode.
    */

+ 7 - 66
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -3650,6 +3650,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         checkPermission(pc, src, false, null, FsAction.WRITE, null,
             FsAction.ALL, true, false);
       }
+
+      final EncryptionZone ez = getEncryptionZoneForPath(src);
+      if (ez != null) {
+        encryptionZones.remove(src);
+      }
+
       long mtime = now();
       // Unlink the target directory from directory tree
       long filesRemoved = dir.delete(src, collectedBlocks, removedINodes,
@@ -8393,73 +8399,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     return keyId;
   }
 
-  /**
-   * Delete the encryption zone on directory src.
-   *
-   * @param src the path of a directory which is the root of the encryption
-   * zone. The directory must be empty and must be marked as an encryption
-   * zone.
-   *
-   * @throws AccessControlException if the caller is not the superuser.
-   *
-   * @throws UnresolvedLinkException if the path can't be resolved.
-   *
-   * @throws SafeModeException if the Namenode is in safe mode.
-   */
-  void deleteEncryptionZone(final String src)
-    throws IOException, UnresolvedLinkException,
-      SafeModeException, AccessControlException {
-    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
-    if (cacheEntry != null && cacheEntry.isSuccess()) {
-      return; // Return previous response
-    }
-
-    boolean success = false;
-    try {
-      deleteEncryptionZoneInt(src, cacheEntry != null);
-      encryptionZones.remove(src);
-      success = true;
-    } catch (AccessControlException e) {
-      logAuditEvent(false, "deleteEncryptionZone", src);
-      throw e;
-    } finally {
-      RetryCache.setState(cacheEntry, success);
-    }
-  }
-
-  private void deleteEncryptionZoneInt(final String srcArg,
-    final boolean logRetryCache) throws IOException {
-    String src = srcArg;
-    HdfsFileStatus resultingStat = null;
-    checkSuperuserPrivilege();
-    checkOperation(OperationCategory.WRITE);
-    final byte[][] pathComponents =
-      FSDirectory.getPathComponentsForReservedPath(src);
-    writeLock();
-    try {
-      checkSuperuserPrivilege();
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot delete encryption zone on " + src);
-      src = FSDirectory.resolvePath(src, pathComponents, dir);
-      final EncryptionZone ez = encryptionZones.get(src);
-      if (ez == null) {
-        throw new IOException("Directory " + src +
-          " is not the root of an encryption zone.");
-      }
-      final List<XAttr> removedXAttrs = dir.deleteEncryptionZone(src);
-      if (removedXAttrs != null && !removedXAttrs.isEmpty()) {
-        getEditLog().logRemoveXAttrs(src, removedXAttrs);
-      }
-      encryptionZones.remove(src);
-      resultingStat = getAuditFileInfo(src, false);
-    } finally {
-      writeUnlock();
-    }
-    getEditLog().logSync();
-    logAuditEvent(true, "deleteEncryptionZone", src, null, resultingStat);
-  }
-
   List<EncryptionZone> listEncryptionZones() throws IOException {
+
     boolean success = false;
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);

+ 0 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -1413,11 +1413,6 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.createEncryptionZone(src, keyId);
   }
 
-  @Override
-  public void deleteEncryptionZone(String src) throws IOException {
-    namesystem.deleteEncryptionZone(src);
-  }
-
   @Override
   public List<EncryptionZone> listEncryptionZones() throws IOException {
     return namesystem.listEncryptionZones();

+ 0 - 50
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/CryptoAdmin.java

@@ -32,8 +32,6 @@ import org.apache.hadoop.tools.TableListing;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.Tool;
 
-import com.google.common.base.Joiner;
-
 /**
  * This class implements crypto command-line operations.
  */
@@ -169,53 +167,6 @@ public class CryptoAdmin extends Configured implements Tool {
     }
   }
 
-  private static class DeleteZoneCommand implements Command {
-    @Override
-    public String getName() {
-      return "-deleteZone";
-    }
-
-    @Override
-    public String getShortUsage() {
-      return "[" + getName() + " -path <path> " + "]\n";
-    }
-
-    @Override
-    public String getLongUsage() {
-      final TableListing listing = getOptionDescriptionListing();
-      listing.addRow("<path>", "The path of the encryption zone to delete. " +
-        "It must be an empty directory and an existing encryption zone.");
-      return getShortUsage() + "\n" +
-        "Delete an encryption zone.\n\n" +
-        listing.toString();
-    }
-
-    @Override
-    public int run(Configuration conf, List<String> args) throws IOException {
-      final String path = StringUtils.popOptionWithArgument("-path", args);
-      if (path == null) {
-        System.err.println("You must specify a path with -path.");
-        return 1;
-      }
-
-      if (!args.isEmpty()) {
-        System.err.println("Can't understand argument: " + args.get(0));
-        return 1;
-      }
-
-      final DistributedFileSystem dfs = getDFS(conf);
-      try {
-        dfs.deleteEncryptionZone(new Path(path));
-        System.out.println("Deleted encryption zone " + path);
-      } catch (IOException e) {
-        System.err.println(prettifyException(e));
-        return 2;
-      }
-
-      return 0;
-    }
-  }
-
   private static class ListZonesCommand implements Command {
     @Override
     public String getName() {
@@ -315,7 +266,6 @@ public class CryptoAdmin extends Configured implements Tool {
 
   private static final Command[] COMMANDS = {
     new CreateZoneCommand(),
-    new DeleteZoneCommand(),
     new ListZonesCommand(),
     new HelpCommand(),
   };

+ 0 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -772,8 +772,6 @@ service ClientNamenodeProtocol {
       returns(RemoveXAttrResponseProto);
   rpc createEncryptionZone(CreateEncryptionZoneRequestProto)
       returns(CreateEncryptionZoneResponseProto);
-  rpc deleteEncryptionZone(DeleteEncryptionZoneRequestProto)
-      returns(DeleteEncryptionZoneResponseProto);
   rpc listEncryptionZones(ListEncryptionZonesRequestProto)
       returns(ListEncryptionZonesResponseProto);
 }

+ 0 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/encryption.proto

@@ -41,13 +41,6 @@ message CreateEncryptionZoneRequestProto {
 message CreateEncryptionZoneResponseProto {
 }
 
-message DeleteEncryptionZoneRequestProto {
-  required string src = 1;
-}
-
-message DeleteEncryptionZoneResponseProto {
-}
-
 message ListEncryptionZonesRequestProto {
 }
 

+ 19 - 111
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesAPI.java

@@ -85,7 +85,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test failure of Create EZ on a directory that doesn't exist. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneDirectoryDoesntExist() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -98,7 +98,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test failure of Create EZ on a directory which is already an EZ. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneWhichAlreadyExists()
     throws Exception {
     final HdfsAdmin dfsAdmin =
@@ -114,7 +114,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test success of Create EZ in which a key is created. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneAndGenerateKeyDirectoryEmpty()
     throws Exception {
     final HdfsAdmin dfsAdmin =
@@ -124,7 +124,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test failure of Create EZ operation in an existing EZ. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneInExistingEncryptionZone()
     throws Exception {
     final HdfsAdmin dfsAdmin =
@@ -142,7 +142,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test failure of creating an EZ using a non-empty directory. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneAndGenerateKeyDirectoryNotEmpty()
     throws Exception {
     final HdfsAdmin dfsAdmin =
@@ -159,7 +159,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test failure of creating an EZ passing a key that doesn't exist. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneKeyDoesntExist() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -174,7 +174,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test success of creating an EZ when they key exists. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateEncryptionZoneKeyExist() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -198,9 +198,9 @@ public class TestEncryptionZonesAPI {
     provider.flush();
   }
 
-  /** Test failure of create/delete encryption zones as a non super user. */
-  @Test(timeout = 30000)
-  public void testCreateAndDeleteEncryptionZoneAsNonSuperUser()
+  /** Test failure of create encryption zones as a non super user. */
+  @Test(timeout = 60000)
+  public void testCreateEncryptionZoneAsNonSuperUser()
     throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -225,120 +225,28 @@ public class TestEncryptionZonesAPI {
           return null;
         }
       });
-    dfsAdmin.createEncryptionZone(TEST_PATH, null);
-
-    user.doAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        final HdfsAdmin userAdmin =
-                new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-        try {
-          userAdmin.deleteEncryptionZone(TEST_PATH);
-          fail("deleteEncryptionZone is superuser-only operation");
-        } catch (AccessControlException e) {
-          GenericTestUtils.assertExceptionContains(
-                  "Superuser privilege is required", e);
-        }
-        return null;
-      }
-    });
-  }
-
-  /** Test failure of deleting an EZ passing a directory that doesn't exist. */
-  @Test(timeout = 30000)
-  public void testDeleteEncryptionZoneDirectoryDoesntExist() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    try {
-      dfsAdmin.deleteEncryptionZone(TEST_PATH);
-      fail("Directory doesn't exist");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains(
-        "is not the root of an encryption zone", e);
-    }
-  }
-
-  /** Test failure of deleting an EZ which is not empty. */
-  @Test(timeout = 30000)
-  public void testDeleteEncryptionZoneAndGenerateKeyDirectoryNotEmpty()
-    throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    dfsAdmin.createEncryptionZone(TEST_PATH, null);
-    FileSystem.create(fs, new Path("/test/foo"),
-      new FsPermission((short) 0777));
-    try {
-      dfsAdmin.deleteEncryptionZone(TEST_PATH);
-      fail("Directory not empty");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("non-empty directory", e);
-    }
-  }
-
-  /** Test success of deleting an EZ. */
-  @Test(timeout = 30000)
-  public void testDeleteEncryptionZone()
-    throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    dfsAdmin.createEncryptionZone(TEST_PATH, null);
-    List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
-    Preconditions.checkState(zones.size() == 1, "More than one zone found?");
-    dfsAdmin.deleteEncryptionZone(TEST_PATH);
-    zones = dfsAdmin.listEncryptionZones();
-    Preconditions.checkState(zones.size() == 0, "More than one zone found?");
   }
 
   /**
-   * Test failure of deleting an EZ on a subdir that is not the root of an EZ.
+   * Test success of creating an encryption zone a few levels down.
    */
-  @Test(timeout = 30000)
-  public void testDeleteEncryptionZoneInExistingEncryptionZone()
-    throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    dfsAdmin.createEncryptionZone(TEST_PATH, null);
-    FileSystem.mkdirs(fs, TEST_PATH_WITH_CHILD, new FsPermission((short) 0777));
-    try {
-      dfsAdmin.deleteEncryptionZone(TEST_PATH_WITH_CHILD);
-      fail("EZ in an EZ");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains(
-        "is not the root of an encryption zone", e);
-    }
-  }
-
-  /**
-   * Test success of creating and deleting an encryption zone a few levels down.
-   */
-  @Test(timeout = 30000)
-  public void testCreateAndDeleteEncryptionZoneDownAFewLevels()
+  @Test(timeout = 60000)
+  public void testCreateEncryptionZoneDownAFewLevels()
     throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    dfsAdmin.createEncryptionZone(TEST_PATH, null);
     FileSystem.mkdirs(fs, TEST_PATH_WITH_MULTIPLE_CHILDREN,
       new FsPermission((short) 0777));
-    try {
-      dfsAdmin.deleteEncryptionZone(TEST_PATH_WITH_MULTIPLE_CHILDREN);
-      fail("EZ in an EZ");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains(
-        "is not the root of an encryption zone", e);
-    }
+    dfsAdmin.createEncryptionZone(TEST_PATH_WITH_MULTIPLE_CHILDREN, null);
     final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
     Preconditions.checkState(zones.size() == 1, "More than one zone found?");
     final EncryptionZone ez = zones.get(0);
       GenericTestUtils.assertMatches(ez.toString(),
-         "EncryptionZone \\[path=/test, keyId=");
+         "EncryptionZone \\[path=/test/foo/baz, keyId=");
   }
 
   /** Test failure of creating an EZ using a non-empty directory. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testCreateFileInEncryptionZone() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -353,7 +261,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test listing encryption zones. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testListEncryptionZones() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -377,7 +285,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test listing encryption zones as a non super user. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testListEncryptionZonesAsNonSuperUser() throws Exception {
     final HdfsAdmin dfsAdmin =
       new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
@@ -412,7 +320,7 @@ public class TestEncryptionZonesAPI {
   }
 
   /** Test success of Rename EZ on a directory which is already an EZ. */
-  @Test(timeout = 30000)
+  @Test(timeout = 60000)
   public void testRenameEncryptionZone()
           throws Exception {
     final HdfsAdmin dfsAdmin =

+ 2 - 72
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCryptoConf.xml

@@ -71,7 +71,6 @@
         <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
       </test-commands>
       <cleanup-commands>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /foo</command>
       </cleanup-commands>
       <comparators>
@@ -90,7 +89,6 @@
         <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
       </test-commands>
       <cleanup-commands>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /foo</command>
       </cleanup-commands>
       <comparators>
@@ -112,7 +110,6 @@
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /foo/bar</command>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /foo</command>
       </cleanup-commands>
       <comparators>
@@ -169,7 +166,6 @@
         <crypto-admin-command>-createZone -path /foo -keyId mykey</crypto-admin-command>
       </test-commands>
       <cleanup-commands>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /foo</command>
       </cleanup-commands>
       <comparators>
@@ -181,74 +177,13 @@
     </test>
 
     <test>
-      <description>Test failure of deleting an EZ passing a directory that doesn't exist.</description>
-      <test-commands>
-        <command>-fs NAMENODE -ls /</command>-
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
-      </test-commands>
-      <cleanup-commands>
-      </cleanup-commands>
-      <comparators>
-        <comparator>
-          <type>SubstringComparator</type>
-          <expected-output>Directory /foo is not the root of an encryption zone.</expected-output>
-        </comparator>
-      </comparators>
-    </test>
-
-    <test>
-      <description>Test failure of deleting an EZ which is not empty.</description>
-      <test-commands>
-        <command>-fs NAMENODE -mkdir /foo</command>
-        <command>-fs NAMENODE -ls /</command>-
-        <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
-        <command>-fs NAMENODE -touchz /foo/bar</command>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
-      </test-commands>
-      <cleanup-commands>
-        <command>-fs NAMENODE -rm /foo/bar</command>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
-        <command>-fs NAMENODE -rmdir /foo</command>
-      </cleanup-commands>
-      <comparators>
-        <comparator>
-          <type>SubstringComparator</type>
-          <expected-output>Attempt to delete an encryption zone for a non-empty directory.</expected-output>
-        </comparator>
-      </comparators>
-    </test>
-
-    <test>
-      <description>Test failure of deleting an EZ on a subdir that is not the root of an EZ.</description>
-      <test-commands>
-        <command>-fs NAMENODE -mkdir /foo</command>
-        <command>-fs NAMENODE -ls /</command>-
-        <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
-        <command>-fs NAMENODE -mkdir /foo/bar</command>
-        <crypto-admin-command>-deleteZone -path /foo/bar</crypto-admin-command>
-      </test-commands>
-      <cleanup-commands>
-        <command>-fs NAMENODE -rmdir /foo/bar</command>
-        <crypto-admin-command>-deleteZone -path /foo</crypto-admin-command>
-        <command>-fs NAMENODE -rmdir /foo</command>
-      </cleanup-commands>
-      <comparators>
-        <comparator>
-          <type>SubstringComparator</type>
-          <expected-output>Directory /foo/bar is not the root of an encryption zone.</expected-output>
-        </comparator>
-      </comparators>
-    </test>
-
-    <test>
-      <description>Test success of creating and deleting an encryption zone a few levels down.</description>
+      <description>Test success of creating an encryption zone a few levels down.</description>
       <test-commands>
         <command>-fs NAMENODE -mkdir /foo</command>
         <command>-fs NAMENODE -mkdir /foo/bar</command>
         <command>-fs NAMENODE -mkdir /foo/bar/baz</command>
         <command>-fs NAMENODE -ls /</command>-
         <crypto-admin-command>-createZone -path /foo/bar/baz</crypto-admin-command>
-        <crypto-admin-command>-deleteZone -path /foo/bar/baz</crypto-admin-command>
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /foo/bar/baz</command>
@@ -258,7 +193,7 @@
       <comparators>
         <comparator>
           <type>SubstringComparator</type>
-          <expected-output>Deleted encryption zone /foo/bar/baz</expected-output>
+          <expected-output>Added encryption zone /foo/bar/baz</expected-output>
         </comparator>
       </comparators>
     </test>
@@ -276,8 +211,6 @@
       </test-commands>
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /src/subdir</command>
-        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
-        <crypto-admin-command>-deleteZone -path /dst</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /src</command>
         <command>-fs NAMENODE -rmdir /dst</command>
       </cleanup-commands>
@@ -299,7 +232,6 @@
         <command>-fs NAMENODE -mv /src /dst</command>-
       </test-commands>
       <cleanup-commands>
-        <crypto-admin-command>-deleteZone -path /dst</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /src</command>
         <command>-fs NAMENODE -rmdir /dst</command>
       </cleanup-commands>
@@ -321,7 +253,6 @@
         <command>-fs NAMENODE -mv /src /dst</command>-
       </test-commands>
       <cleanup-commands>
-        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /src</command>
         <command>-fs NAMENODE -rmdir /dst</command>
       </cleanup-commands>
@@ -345,7 +276,6 @@
       <cleanup-commands>
         <command>-fs NAMENODE -rmdir /src/subdir2/subdir1</command>
         <command>-fs NAMENODE -rmdir /src/subdir2</command>
-        <crypto-admin-command>-deleteZone -path /src</crypto-admin-command>
         <command>-fs NAMENODE -rmdir /src</command>
       </cleanup-commands>
       <comparators>