ソースを参照

HDFS-6387. HDFS CLI admin tool for creating & deleting an encryption zone. (clamb)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1605518 13f79535-47bb-0310-9956-ffa450edef68
Charles Lamb 11 年 前
コミット
2676365799

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

@@ -10,6 +10,9 @@ fs-encryption (Unreleased)
 
   IMPROVEMENTS
 
+    HDFS-6387. HDFS CLI admin tool for creating & deleting an
+    encryption zone. (clamb)
+
     HDFS-6386. HDFS Encryption Zones (clamb)
 
     HDFS-6473. Protocol and API for Encryption Zones (clamb)

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs

@@ -61,6 +61,7 @@ function print_usage(){
   echo "  portmap              run a portmap service"
   echo "  nfs3                 run an NFS version 3 gateway"
   echo "  cacheadmin           configure the HDFS cache"
+  echo "  crypto               configure HDFS encryption zones"
   echo ""
   echo "Most commands print help when invoked w/o parameters."
 }
@@ -184,6 +185,8 @@ elif [ "$COMMAND" = "nfs3" ] ; then
   HADOOP_OPTS="$HADOOP_OPTS $HADOOP_NFS3_OPTS"
 elif [ "$COMMAND" = "cacheadmin" ] ; then
   CLASS=org.apache.hadoop.hdfs.tools.CacheAdmin
+elif [ "$COMMAND" = "crypto" ] ; then
+  CLASS=org.apache.hadoop.hdfs.tools.CryptoAdmin
 else
   CLASS="$COMMAND"
 fi

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

@@ -0,0 +1,344 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.tools;
+
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+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.
+ */
+@InterfaceAudience.Private
+public class CryptoAdmin extends Configured implements Tool {
+
+  /**
+   * Maximum length for printed lines
+   */
+  private static final int MAX_LINE_WIDTH = 80;
+
+  public CryptoAdmin() {
+    this(null);
+  }
+
+  public CryptoAdmin(Configuration conf) {
+    super(conf);
+  }
+
+  @Override
+  public int run(String[] args) throws IOException {
+    if (args.length == 0) {
+      printUsage(false);
+      return 1;
+    }
+    final Command command = determineCommand(args[0]);
+    if (command == null) {
+      System.err.println("Can't understand command '" + args[0] + "'");
+      if (!args[0].startsWith("-")) {
+        System.err.println("Command names must start with dashes.");
+      }
+      printUsage(false);
+      return 1;
+    }
+    final List<String> argsList = new LinkedList<String>();
+    for (int j = 1; j < args.length; j++) {
+      argsList.add(args[j]);
+    }
+    try {
+      return command.run(getConf(), argsList);
+    } catch (IllegalArgumentException e) {
+      System.err.println(prettifyException(e));
+      return -1;
+    }
+  }
+
+  public static void main(String[] argsArray) throws IOException {
+    final CryptoAdmin cryptoAdmin = new CryptoAdmin(new Configuration());
+    System.exit(cryptoAdmin.run(argsArray));
+  }
+
+  private static DistributedFileSystem getDFS(Configuration conf)
+      throws IOException {
+    final FileSystem fs = FileSystem.get(conf);
+    if (!(fs instanceof DistributedFileSystem)) {
+      throw new IllegalArgumentException("FileSystem " + fs.getUri() +
+      " is not an HDFS file system");
+    }
+    return (DistributedFileSystem) fs;
+  }
+
+  /**
+   * NN exceptions contain the stack trace as part of the exception message.
+   * When it's a known error, pretty-print the error and squish the stack trace.
+   */
+  private static String prettifyException(Exception e) {
+    return e.getClass().getSimpleName() + ": " +
+      e.getLocalizedMessage().split("\n")[0];
+  }
+
+  private static TableListing getOptionDescriptionListing() {
+    final TableListing listing = new TableListing.Builder()
+      .addField("").addField("", true)
+      .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
+    return listing;
+  }
+
+  interface Command {
+    String getName();
+    String getShortUsage();
+    String getLongUsage();
+    int run(Configuration conf, List<String> args) throws IOException;
+  }
+
+  private static class CreateZoneCommand implements Command {
+    @Override
+    public String getName() {
+      return "-createZone";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName() + " [-keyId <keyId>] -path <path> " + "]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      final TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<path>", "The path of the encryption zone to create. " +
+        "It must be an empty directory.");
+      listing.addRow("<keyId>", "The keyId of the new encryption zone.");
+      return getShortUsage() + "\n" +
+        "Create a new 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;
+      }
+
+      final String keyId =
+          StringUtils.popOptionWithArgument("-keyId", args);
+
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        return 1;
+      }
+
+      final DistributedFileSystem dfs = getDFS(conf);
+      try {
+        dfs.createEncryptionZone(new Path(path), keyId);
+        System.out.println("Added encryption zone " + path);
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+
+      return 0;
+    }
+  }
+
+  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() {
+      return "-listZones";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[" + getName()+ "]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      return getShortUsage() + "\n" +
+        "List all encryption zones.\n\n";
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (!args.isEmpty()) {
+        System.err.println("Can't understand argument: " + args.get(0));
+        return 1;
+      }
+
+      final DistributedFileSystem dfs = getDFS(conf);
+      try {
+        final TableListing listing = new TableListing.Builder()
+          .addField("").addField("", true)
+          .wrapWidth(MAX_LINE_WIDTH).hideHeaders().build();
+        final List<EncryptionZone> ezs = dfs.listEncryptionZones();
+        for (EncryptionZone ez : ezs) {
+          listing.addRow(ez.getPath(), ez.getKeyId());
+        }
+        System.out.println(listing.toString());
+      } catch (IOException e) {
+        System.err.println(prettifyException(e));
+        return 2;
+      }
+
+      return 0;
+    }
+  }
+
+  private static class HelpCommand implements Command {
+    @Override
+    public String getName() {
+      return "-help";
+    }
+
+    @Override
+    public String getShortUsage() {
+      return "[-help <command-name>]\n";
+    }
+
+    @Override
+    public String getLongUsage() {
+      final TableListing listing = getOptionDescriptionListing();
+      listing.addRow("<command-name>", "The command for which to get " +
+          "detailed help. If no command is specified, print detailed help for " +
+          "all commands");
+      return getShortUsage() + "\n" +
+        "Get detailed help about a command.\n\n" +
+        listing.toString();
+    }
+
+    @Override
+    public int run(Configuration conf, List<String> args) throws IOException {
+      if (args.size() == 0) {
+        for (Command command : COMMANDS) {
+          System.err.println(command.getLongUsage());
+        }
+        return 0;
+      }
+      if (args.size() != 1) {
+        System.out.println("You must give exactly one argument to -help.");
+        return 0;
+      }
+      final String commandName = args.get(0);
+      // prepend a dash to match against the command names
+      final Command command = determineCommand("-"+commandName);
+      if (command == null) {
+        System.err.print("Sorry, I don't know the command '" +
+          commandName + "'.\n");
+        System.err.print("Valid help command names are:\n");
+        String separator = "";
+        for (Command c : COMMANDS) {
+          System.err.print(separator + c.getName().substring(1));
+          separator = ", ";
+        }
+        System.err.print("\n");
+        return 1;
+      }
+      System.err.print(command.getLongUsage());
+      return 0;
+    }
+  }
+
+  private static final Command[] COMMANDS = {
+    new CreateZoneCommand(),
+    new DeleteZoneCommand(),
+    new ListZonesCommand(),
+    new HelpCommand(),
+  };
+
+  private static void printUsage(boolean longUsage) {
+    System.err.println(
+        "Usage: bin/hdfs crypto [COMMAND]");
+    for (Command command : COMMANDS) {
+      if (longUsage) {
+        System.err.print(command.getLongUsage());
+      } else {
+        System.err.print("          " + command.getShortUsage());
+      }
+    }
+    System.err.println();
+  }
+
+  private static Command determineCommand(String commandName) {
+    for (int i = 0; i < COMMANDS.length; i++) {
+      if (COMMANDS[i].getName().equals(commandName)) {
+        return COMMANDS[i];
+      }
+    }
+    return null;
+  }
+}

+ 169 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestCryptoCLI.java

@@ -0,0 +1,169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.cli;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.util.UUID;
+
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.cli.util.CLICommand;
+import org.apache.hadoop.cli.util.CLICommandCryptoAdmin;
+import org.apache.hadoop.cli.util.CLICommandTypes;
+import org.apache.hadoop.cli.util.CLITestCmd;
+import org.apache.hadoop.cli.util.CryptoAdminCmdExecutor;
+import org.apache.hadoop.cli.util.CommandExecutor;
+import org.apache.hadoop.cli.util.CommandExecutor.Result;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.crypto.key.KeyProvider;
+import org.apache.hadoop.crypto.key.KeyProviderFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HDFSPolicyProvider;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.tools.CryptoAdmin;
+import org.apache.hadoop.security.authorize.PolicyProvider;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.xml.sax.SAXException;
+
+public class TestCryptoCLI  extends CLITestHelperDFS {
+  protected MiniDFSCluster dfsCluster = null;
+  protected FileSystem fs = null;
+  protected String namenode = null;
+  private static File tmpDir;
+
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    conf.setClass(PolicyProvider.POLICY_PROVIDER_CONFIG,
+        HDFSPolicyProvider.class, PolicyProvider.class);
+    conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1);
+
+    tmpDir = new File(System.getProperty("test.build.data", "target"),
+        UUID.randomUUID().toString()).getAbsoluteFile();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
+
+    dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    dfsCluster.waitClusterUp();
+    createAKey("mykey", conf);
+    namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///");
+
+    username = System.getProperty("user.name");
+
+    fs = dfsCluster.getFileSystem();
+    assertTrue("Not an HDFS: " + fs.getUri(),
+        fs instanceof DistributedFileSystem);
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    if (fs != null) {
+      fs.close();
+    }
+    if (dfsCluster != null) {
+      dfsCluster.shutdown();
+    }
+    Thread.sleep(2000);
+    super.tearDown();
+  }
+
+  /* Helper function to create a key in the Key Provider. */
+  private void createAKey(String keyId, Configuration conf)
+    throws NoSuchAlgorithmException, IOException {
+    final KeyProvider provider =
+        dfsCluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey(keyId, options);
+    provider.flush();
+    }
+
+  @Override
+  protected String getTestFile() {
+    return "testCryptoConf.xml";
+  }
+
+  @Override
+  protected String expandCommand(final String cmd) {
+    String expCmd = cmd;
+    expCmd = expCmd.replaceAll("NAMENODE", namenode);
+    expCmd = expCmd.replaceAll("#LF#",
+        System.getProperty("line.separator"));
+    expCmd = super.expandCommand(expCmd);
+    return expCmd;
+  }
+
+  @Override
+  protected TestConfigFileParser getConfigParser() {
+    return new TestConfigFileParserCryptoAdmin();
+  }
+
+  private class TestConfigFileParserCryptoAdmin extends
+      CLITestHelper.TestConfigFileParser {
+    @Override
+    public void endElement(String uri, String localName, String qName)
+        throws SAXException {
+      if (qName.equals("crypto-admin-command")) {
+        if (testCommands != null) {
+          testCommands.add(new CLITestCmdCryptoAdmin(charString,
+              new CLICommandCryptoAdmin()));
+        } else if (cleanupCommands != null) {
+          cleanupCommands.add(new CLITestCmdCryptoAdmin(charString,
+              new CLICommandCryptoAdmin()));
+        }
+      } else {
+        super.endElement(uri, localName, qName);
+      }
+    }
+  }
+
+  private class CLITestCmdCryptoAdmin extends CLITestCmd {
+    public CLITestCmdCryptoAdmin(String str, CLICommandTypes type) {
+      super(str, type);
+    }
+
+    @Override
+    public CommandExecutor getExecutor(String tag)
+        throws IllegalArgumentException {
+      if (getType() instanceof CLICommandCryptoAdmin) {
+        return new CryptoAdminCmdExecutor(tag, new CryptoAdmin(conf));
+      }
+      return super.getExecutor(tag);
+    }
+  }
+
+  @Override
+  protected Result execute(CLICommand cmd) throws Exception {
+    return cmd.getExecutor(namenode).executeCommand(cmd.getCmd());
+  }
+
+  @Test
+  @Override
+  public void testAll () {
+    super.testAll();
+  }
+}

+ 21 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandCryptoAdmin.java

@@ -0,0 +1,21 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.cli.util;
+
+public class CLICommandCryptoAdmin implements CLICommandTypes {
+}

+ 37 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CryptoAdminCmdExecutor.java

@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.cli.util;
+
+import org.apache.hadoop.hdfs.tools.CryptoAdmin;
+import org.apache.hadoop.util.ToolRunner;
+
+public class CryptoAdminCmdExecutor extends CommandExecutor {
+  protected String namenode = null;
+  protected CryptoAdmin admin = null;
+
+  public CryptoAdminCmdExecutor(String namenode, CryptoAdmin admin) {
+    this.namenode = namenode;
+    this.admin = admin;
+  }
+
+  @Override
+  protected void execute(final String cmd) throws Exception {
+    String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode);
+    ToolRunner.run(admin, args);
+  }
+}

+ 266 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testCryptoConf.xml

@@ -0,0 +1,266 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="testConf.xsl"?>
+
+<!--
+   Licensed to the Apache Software Foundation (ASF) under one or more
+   contributor license agreements.  See the NOTICE file distributed with
+   this work for additional information regarding copyright ownership.
+   The ASF licenses this file to You under the Apache License, Version 2.0
+   (the "License"); you may not use this file except in compliance with
+   the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
+-->
+
+<configuration>
+  <!-- Normal mode is test. To run just the commands and dump the output
+       to the log, set it to nocompare -->
+  <mode>test</mode>
+
+  <!--  Comparator types:
+           ExactComparator
+           SubstringComparator
+           RegexpComparator
+           TokenComparator
+           -->
+  <tests>
+
+    <test>
+      <description>Test basic usage</description>
+      <test-commands>
+        <crypto-admin-command></crypto-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+	  <expected-output>Usage: bin/hdfs crypto [COMMAND]</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test create ez, dir doesn't exist</description>
+      <test-commands>
+        <command>-fs NAMENODE -ls /test</command>-
+        <crypto-admin-command>-createZone -path /test</crypto-admin-command>
+      </test-commands>
+      <cleanup-commands>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>cannot find /test</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of create ez on an existing ez</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /foo</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
+        <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>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Directory /foo is already in an encryption zone</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test success of create ez in which a key is created</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /foo</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <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>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Added encryption zone /foo</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of Create EZ operation in an existing 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>-createZone -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 already in an encryption zone. (/foo)</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of creating an EZ using a non-empty directory.</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /foo</command>
+        <command>-fs NAMENODE -touchz /foo/bar</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /foo</crypto-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rm /foo/bar</command>
+        <command>-fs NAMENODE -rmdir /foo</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Attempt to create an encryption zone for a non-empty directory.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test failure of creating an EZ passing a key that doesn't exist.</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /foo</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <crypto-admin-command>-createZone -path /foo -keyId doesntexist</crypto-admin-command>
+      </test-commands>
+      <cleanup-commands>
+        <command>-fs NAMENODE -rmdir /foo</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Key doesntexist doesn't exist.</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+
+    <test>
+      <description>Test success of creating an EZ when the key exists.</description>
+      <test-commands>
+        <command>-fs NAMENODE -mkdir /foo</command>
+        <command>-fs NAMENODE -ls /</command>-
+        <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>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Added encryption zone /foo</expected-output>
+        </comparator>
+      </comparators>
+    </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>
+      <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>
+        <command>-fs NAMENODE -rmdir /foo/bar</command>
+        <command>-fs NAMENODE -rmdir /foo/</command>
+      </cleanup-commands>
+      <comparators>
+        <comparator>
+          <type>SubstringComparator</type>
+          <expected-output>Deleted encryption zone /foo/bar/baz</expected-output>
+        </comparator>
+      </comparators>
+    </test>
+  </tests>
+</configuration>