瀏覽代碼

HDFS-6619. Clean up encryption-related tests. (wang)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1610849 13f79535-47bb-0310-9956-ffa450edef68
Andrew Wang 11 年之前
父節點
當前提交
b066be8115

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

@@ -42,6 +42,8 @@ fs-encryption (Unreleased)
     HDFS-6474. Namenode needs to get the actual keys and iv from the
     KeyProvider. (wang)
 
+    HDFS-6619. Clean up encryption-related tests. (wang)
+
   OPTIMIZATIONS
 
   BUG FIXES

+ 0 - 352
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/TestHDFSEncryption.java

@@ -1,352 +0,0 @@
-/**
- * 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.fs;
-
-import static org.apache.hadoop.fs.CreateFlag.CREATE;
-import static org.apache.hadoop.fs.FileContextTestHelper.getDefaultBlockSize;
-import static org.apache.hadoop.fs.FileContextTestHelper.getFileData;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.Arrays;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.UUID;
-
-import javax.security.auth.login.LoginException;
-
-import org.apache.commons.lang.RandomStringUtils;
-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.KeyProvider.KeyVersion;
-import org.apache.hadoop.crypto.key.KeyProviderFactory;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.ipc.RemoteException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class TestHDFSEncryption {
-  private static MiniDFSCluster cluster;
-  private static Path defaultWorkingDirectory;
-  private static final HdfsConfiguration CONF = new HdfsConfiguration();
-  private static FileContext fc;
-  private Path localFsRootPath;
-  private Path src1;
-  /* The KeyProvider, if any. */
-  private static KeyProvider provider = null;
-
-  private static File tmpDir;
-
-  @BeforeClass
-  public static void clusterSetupAtBegining() throws IOException,
-      LoginException, URISyntaxException {
-    tmpDir = new File(System.getProperty("test.build.data", "target"),
-        UUID.randomUUID().toString()).getAbsoluteFile();
-    tmpDir.mkdirs();
-
-    CONF.set(KeyProviderFactory.KEY_PROVIDER_PATH,
-            JavaKeyStoreProvider.SCHEME_NAME + "://file" + tmpDir + "/test.jks");
-    initializeKeyProvider(CONF);
-    try {
-      createOneKey();
-      KeyVersion blort = provider.getCurrentKey("blort");
-    } catch (java.security.NoSuchAlgorithmException e) {
-      throw new RuntimeException(e);
-    }
-
-    cluster = new MiniDFSCluster.Builder(CONF).numDataNodes(1).build();
-    cluster.waitClusterUp();
-
-    URI uri0 = cluster.getURI(0);
-    fc = FileContext.getFileContext(uri0, CONF);
-    defaultWorkingDirectory = fc.makeQualified(new Path("/user/" +
-        UserGroupInformation.getCurrentUser().getShortUserName()));
-    fc.mkdir(defaultWorkingDirectory, FileContext.DEFAULT_PERM, true);
-  }
-
-  private static void initializeKeyProvider(final Configuration conf)
-    throws IOException {
-    final List<KeyProvider> providers = KeyProviderFactory.getProviders(conf);
-    if (providers == null) {
-      return;
-    }
-
-    if (providers.size() == 0) {
-      return;
-    }
-
-    if (providers.size() > 1) {
-      final String err =
-        "Multiple KeyProviders found. Only one is permitted.";
-      throw new RuntimeException(err);
-    }
-    provider = providers.get(0);
-    if (provider.isTransient()) {
-      final String err =
-        "A KeyProvider was found but it is a transient provider.";
-      throw new RuntimeException(err);
-    }
-  }
-
-  private static void createOneKey()
-    throws java.security.NoSuchAlgorithmException, IOException {
-    final org.apache.hadoop.crypto.key.KeyProvider.Options options =
-      KeyProvider.options(CONF);
-    provider.createKey("blort", options);
-    provider.flush();
-  }
-
-  @AfterClass
-  public static void ClusterShutdownAtEnd() throws Exception {
-    if (cluster != null) {
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @Before
-  public void setUp() throws Exception {
-    File testBuildData = new File(System.getProperty("test.build.data",
-            "build/test/data"), RandomStringUtils.randomAlphanumeric(10));
-    Path rootPath = new Path(testBuildData.getAbsolutePath(),
-            "root-uri");
-    localFsRootPath = rootPath.makeQualified(LocalFileSystem.NAME, null);
-    fc.mkdir(getTestRootPath(fc, "test"), FileContext.DEFAULT_PERM, true);
-    src1 = getTestRootPath(fc, "testfile");
-  }
-
-  @After
-  public void tearDown() throws Exception {
-    final boolean del =
-      fc.delete(new Path(fileContextTestHelper.getAbsoluteTestRootPath(fc), new Path("test")), true);
-    assertTrue(del);
-    fc.delete(localFsRootPath, true);
-  }
-
-  protected final FileContextTestHelper fileContextTestHelper =
-    createFileContextHelper();
-
-  protected FileContextTestHelper createFileContextHelper() {
-    return new FileContextTestHelper();
-  }
-
-  protected Path getDefaultWorkingDirectory() {
-    return defaultWorkingDirectory;
-  }
-
-  private Path getTestRootPath(FileContext fc, String path) {
-    return fileContextTestHelper.getTestRootPath(fc, path);
-  }
-
-  protected IOException unwrapException(IOException e) {
-    if (e instanceof RemoteException) {
-      return ((RemoteException) e).unwrapRemoteException();
-    }
-    return e;
-  }
-
-  private static final int NUM_BLOCKS = 3;
-
-  private static final byte[] data = getFileData(NUM_BLOCKS,
-      getDefaultBlockSize());
-
-  private void writeSomeData() throws Exception {
-    writeSomeData(false, false);
-  }
-
-  private void writeSomeData(boolean doHFlush, boolean doHSync) throws Exception {
-    final FSDataOutputStream out =
-      fc.create(src1, EnumSet.of(CREATE), Options.CreateOpts.createParent());
-    out.write(data, 0, data.length);
-    if (doHFlush) {
-      out.hflush();
-    }
-
-    if (doHSync) {
-      out.hsync();
-    }
-
-    out.close();
-  }
-
-  private void writeAndVerify(boolean doHFlush, boolean doHSync) throws Exception {
-    writeSomeData(doHFlush, doHSync);
-
-    final FSDataInputStream in = fc.open(src1);
-    try {
-      final byte[] readBuf = new byte[getDefaultBlockSize() * NUM_BLOCKS];
-
-      in.readFully(readBuf);
-      assertTrue("Expected read-back data to be equal (hflush=" + doHFlush
-        + " hfsync=" + doHSync + ")", Arrays.equals(data, readBuf));
-    } finally {
-      in.close();
-    }
-  }
-
-  @Test
-  public void testBasicEncryptionStreamNoFlushNoSync() throws Exception {
-    writeAndVerify(false, false);
-  }
-
-  @Test
-  public void testBasicEncryptionStreamFlushSync() throws Exception {
-    writeAndVerify(true, true);
-  }
-
-  @Test
-  public void testBasicEncryptionStreamNoFlushSync() throws Exception {
-    writeAndVerify(false, true);
-  }
-
-  @Test
-  public void testBasicEncryptionStreamFlushNoSync() throws Exception {
-    writeAndVerify(true, false);
-  }
-
-  @Test
-  public void testGetPos() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-
-    int expectedGetPos = 0;
-    while (in.read() != -1) {
-      assertTrue(++expectedGetPos == in.getPos());
-    }
-  }
-
-  @Test
-  public void testDoubleClose() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-    in.close();
-    try {
-      in.close();
-    } catch (Exception e) {
-      fail("Caught unexpected exception on double-close: " + e);
-    }
-  }
-
-  @Test
-  public void testHFlush() throws Exception {
-    final DistributedFileSystem fs = cluster.getFileSystem();
-    final FSDataOutputStream out =
-      fc.create(src1, EnumSet.of(CREATE), Options.CreateOpts.createParent());
-    out.write(data, 0, data.length);
-    out.hflush();
-    out.close();
-  }
-
-  @Test
-  public void testSeekBogusArgs() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-    try {
-      in.seek(-1);
-      fail("Expected IOException");
-    } catch (Exception e) {
-      GenericTestUtils.assertExceptionContains("Cannot seek to negative offset", e);
-    }
-
-    try {
-      in.seek(1 << 20);
-      fail("Expected IOException");
-    } catch (Exception e) {
-      GenericTestUtils.assertExceptionContains("Cannot seek after EOF", e);
-    }
-    in.close();
-  }
-
-  @Test
-  public void testSeekForward() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-
-    for (int seekInc = 1; seekInc < 1024; seekInc += 32) {
-      long seekTo = 0;
-      while (seekTo < data.length) {
-        in.seek(seekTo);
-        int b = in.read();
-        byte expected = data[(int) seekTo];
-        assertTrue("seek(" + seekTo + ") Expected: " + expected + ", but got: " + b,
-          b == expected);
-        seekTo += seekInc;
-      }
-    }
-    in.close();
-  }
-
-  @Test
-  public void testSeekBackwards() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-
-    for (int seekInc = 1; seekInc < 1024; seekInc += 32) {
-      long seekTo = data.length - 1;
-      while (seekTo >= 0) {
-        in.seek(seekTo);
-        int b = in.read();
-        byte expected = data[(int) seekTo];
-        assertTrue("seek(" + seekTo + ") Expected: " + expected + ", but got: " + b,
-          b == expected);
-        seekTo -= seekInc;
-      }
-    }
-    in.close();
-  }
-
-  @Test
-  public void testPostionedReadable() throws Exception {
-    writeSomeData();
-
-    final FSDataInputStream in = fc.open(src1);
-
-    try {
-      final byte[] oneByteToRead = new byte[1];
-      for (int i = 0; i < data.length; i++) {
-        int nread = in.read(i, oneByteToRead, 0, 1);
-        final byte b = oneByteToRead[0];
-        byte expected = data[(int) i];
-        assertTrue("read() expected only one byte to be read, but got " + nread, nread == 1);
-        assertTrue("read() expected: " + expected + ", but got: " + b,
-          b == expected);
-      }
-    } finally {
-      in.close();
-    }
-  }
-}

+ 433 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java

@@ -0,0 +1,433 @@
+/**
+ * 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;
+
+import java.io.File;
+import java.io.IOException;
+import java.security.NoSuchAlgorithmException;
+import java.security.PrivilegedExceptionAction;
+import java.util.Arrays;
+import java.util.List;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.CipherSuite;
+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.FSDataInputStream;
+import org.apache.hadoop.fs.FSTestWrapper;
+import org.apache.hadoop.fs.FileContext;
+import org.apache.hadoop.fs.FileContextTestWrapper;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.FileSystemTestWrapper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.EncryptionZone;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
+import org.apache.hadoop.security.AccessControlException;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestEncryptionZones {
+
+  private Configuration conf;
+  private FileSystemTestHelper fsHelper;
+
+  private MiniDFSCluster cluster;
+  private HdfsAdmin dfsAdmin;
+  private DistributedFileSystem fs;
+
+  protected FileSystemTestWrapper fsWrapper;
+  protected FileContextTestWrapper fcWrapper;
+
+  @Before
+  public void setup() throws IOException {
+    conf = new HdfsConfiguration();
+    fsHelper = new FileSystemTestHelper();
+    // Set up java key store
+    String testRoot = fsHelper.getTestRootDir();
+    File testRootDir = new File(testRoot).getAbsoluteFile();
+    conf.set(KeyProviderFactory.KEY_PROVIDER_PATH,
+        JavaKeyStoreProvider.SCHEME_NAME + "://file" + testRootDir + "/test.jks"
+    );
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    Logger.getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);
+    fs = cluster.getFileSystem();
+    fsWrapper = new FileSystemTestWrapper(cluster.getFileSystem());
+    fcWrapper = new FileContextTestWrapper(
+        FileContext.getFileContext(cluster.getURI(), conf));
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+  }
+
+  @After
+  public void teardown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  public void assertNumZones(final int numZones) throws IOException {
+    final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
+    assertEquals("Unexpected number of encryption zones!", numZones,
+        zones.size());
+  }
+
+  /**
+   * Checks that an encryption zone with the specified keyId and path (if not
+   * null) is present.
+   *
+   * @throws IOException if a matching zone could not be found
+   */
+  public void assertZonePresent(String keyId, String path) throws IOException {
+    final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
+    boolean match = false;
+    for (EncryptionZone zone : zones) {
+      boolean matchKey = (keyId == null);
+      boolean matchPath = (path == null);
+      if (keyId != null && zone.getKeyId().equals(keyId)) {
+        matchKey = true;
+      }
+      if (path != null && zone.getPath().equals(path)) {
+        matchPath = true;
+      }
+      if (matchKey && matchPath) {
+        match = true;
+        break;
+      }
+    }
+    assertTrue("Did not find expected encryption zone with keyId " + keyId +
+            " path " + path, match
+    );
+  }
+
+  /**
+   * Helper function to create a key in the Key Provider.
+   */
+  private void createKey(String keyId)
+      throws NoSuchAlgorithmException, IOException {
+    KeyProvider provider = cluster.getNameNode().getNamesystem().getProvider();
+    final KeyProvider.Options options = KeyProvider.options(conf);
+    provider.createKey(keyId, options);
+    provider.flush();
+  }
+
+  @Test(timeout = 60000)
+  public void testBasicOperations() throws Exception {
+
+    int numZones = 0;
+
+    /* Test failure of create EZ on a directory that doesn't exist. */
+    final Path zone1 = new Path("/zone1");
+    try {
+      dfsAdmin.createEncryptionZone(zone1, null);
+      fail("expected /test doesn't exist");
+    } catch (IOException e) {
+      assertExceptionContains("cannot find", e);
+    }
+
+    /* Normal creation of an EZ */
+    fsWrapper.mkdir(zone1, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(zone1, null);
+    assertNumZones(++numZones);
+    assertZonePresent(null, zone1.toString());
+
+    /* Test failure of create EZ on a directory which is already an EZ. */
+    try {
+      dfsAdmin.createEncryptionZone(zone1, null);
+    } catch (IOException e) {
+      assertExceptionContains("already in an encryption zone", e);
+    }
+
+    /* Test failure of create EZ operation in an existing EZ. */
+    final Path zone1Child = new Path(zone1, "child");
+    fsWrapper.mkdir(zone1Child, FsPermission.getDirDefault(), false);
+    try {
+      dfsAdmin.createEncryptionZone(zone1Child, null);
+      fail("EZ in an EZ");
+    } catch (IOException e) {
+      assertExceptionContains("already in an encryption zone", e);
+    }
+
+    /* create EZ on a folder with a folder fails */
+    final Path notEmpty = new Path("/notEmpty");
+    final Path notEmptyChild = new Path(notEmpty, "child");
+    fsWrapper.mkdir(notEmptyChild, FsPermission.getDirDefault(), true);
+    try {
+      dfsAdmin.createEncryptionZone(notEmpty, null);
+      fail("Created EZ on an non-empty directory with folder");
+    } catch (IOException e) {
+      assertExceptionContains("create an encryption zone", e);
+    }
+    fsWrapper.delete(notEmptyChild, false);
+
+    /* create EZ on a folder with a file fails */
+    fsWrapper.createFile(notEmptyChild);
+    try {
+      dfsAdmin.createEncryptionZone(notEmpty, null);
+      fail("Created EZ on an non-empty directory with file");
+    } catch (IOException e) {
+      assertExceptionContains("create an encryption zone", e);
+    }
+
+    /* Test failure of creating an EZ passing a key that doesn't exist. */
+    final Path zone2 = new Path("/zone2");
+    fsWrapper.mkdir(zone2, FsPermission.getDirDefault(), false);
+    final String myKeyId = "mykeyid";
+    try {
+      dfsAdmin.createEncryptionZone(zone2, myKeyId);
+      fail("expected key doesn't exist");
+    } catch (IOException e) {
+      assertExceptionContains("doesn't exist.", e);
+    }
+    assertNumZones(1);
+
+    /* Test success of creating an EZ when they key exists. */
+    createKey(myKeyId);
+    dfsAdmin.createEncryptionZone(zone2, myKeyId);
+    assertNumZones(++numZones);
+    assertZonePresent(myKeyId, zone2.toString());
+
+    /* Test failure of create encryption zones as a non super user. */
+    final UserGroupInformation user = UserGroupInformation.
+        createUserForTesting("user", new String[] { "mygroup" });
+    final Path nonSuper = new Path("/nonSuper");
+    fsWrapper.mkdir(nonSuper, FsPermission.getDirDefault(), false);
+
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        final HdfsAdmin userAdmin =
+            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+        try {
+          userAdmin.createEncryptionZone(nonSuper, null);
+          fail("createEncryptionZone is superuser-only operation");
+        } catch (AccessControlException e) {
+          assertExceptionContains("Superuser privilege is required", e);
+        }
+        return null;
+      }
+    });
+
+    // Test success of creating an encryption zone a few levels down.
+    Path deepZone = new Path("/d/e/e/p/zone");
+    fsWrapper.mkdir(deepZone, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(deepZone, null);
+    assertNumZones(++numZones);
+    assertZonePresent(null, deepZone.toString());
+  }
+
+  /**
+   * Test listing encryption zones as a non super user.
+   */
+  @Test(timeout = 60000)
+  public void testListEncryptionZonesAsNonSuperUser() throws Exception {
+
+    final UserGroupInformation user = UserGroupInformation.
+        createUserForTesting("user", new String[] { "mygroup" });
+
+    final Path testRoot = new Path(fsHelper.getTestRootDir());
+    final Path superPath = new Path(testRoot, "superuseronly");
+    final Path allPath = new Path(testRoot, "accessall");
+
+    fsWrapper.mkdir(superPath, new FsPermission((short) 0700), true);
+    dfsAdmin.createEncryptionZone(superPath, null);
+
+    fsWrapper.mkdir(allPath, new FsPermission((short) 0707), true);
+    dfsAdmin.createEncryptionZone(allPath, null);
+
+    user.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        final HdfsAdmin userAdmin =
+            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+        try {
+          userAdmin.listEncryptionZones();
+        } catch (AccessControlException e) {
+          assertExceptionContains("Superuser privilege is required", e);
+        }
+        return null;
+      }
+    });
+  }
+
+  /**
+   * Test success of Rename EZ on a directory which is already an EZ.
+   */
+  private void doRenameEncryptionZone(FSTestWrapper wrapper) throws Exception {
+    final Path testRoot = new Path(fsHelper.getTestRootDir());
+    final Path pathFoo = new Path(testRoot, "foo");
+    final Path pathFooBaz = new Path(pathFoo, "baz");
+    wrapper.mkdir(pathFoo, FsPermission.getDirDefault(), true);
+    dfsAdmin.createEncryptionZone(pathFoo, null);
+    wrapper.mkdir(pathFooBaz, FsPermission.getDirDefault(), true);
+    try {
+      wrapper.rename(pathFooBaz, testRoot);
+    } catch (IOException e) {
+      assertExceptionContains(pathFooBaz.toString() + " can't be moved from" +
+              " an encryption zone.", e
+      );
+    }
+  }
+
+  @Test(timeout = 60000)
+  public void testRenameFileSystem() throws Exception {
+    doRenameEncryptionZone(fsWrapper);
+  }
+
+  @Test(timeout = 60000)
+  public void testRenameFileContext() throws Exception {
+    doRenameEncryptionZone(fcWrapper);
+  }
+
+  private void validateFiles(Path p1, Path p2, int len) throws Exception {
+    FSDataInputStream in1 = fs.open(p1);
+    FSDataInputStream in2 = fs.open(p2);
+    for (int i = 0; i < len; i++) {
+      assertEquals("Mismatch at byte " + i, in1.read(), in2.read());
+    }
+    in1.close();
+    in2.close();
+  }
+
+  private FileEncryptionInfo getFileEncryptionInfo(Path path) throws Exception {
+    LocatedBlocks blocks = fs.getClient().getLocatedBlocks(path.toString(), 0);
+    return blocks.getFileEncryptionInfo();
+  }
+
+  @Test(timeout = 120000)
+  public void testReadWrite() throws Exception {
+    final HdfsAdmin dfsAdmin =
+        new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    // Create a base file for comparison
+    final Path baseFile = new Path("/base");
+    final int len = 8192;
+    DFSTestUtil.createFile(fs, baseFile, len, (short) 1, 0xFEED);
+    // Create the first enc file
+    final Path zone = new Path("/zone");
+    fs.mkdirs(zone);
+    dfsAdmin.createEncryptionZone(zone, null);
+    final Path encFile1 = new Path(zone, "myfile");
+    DFSTestUtil.createFile(fs, encFile1, len, (short) 1, 0xFEED);
+    // Read them back in and compare byte-by-byte
+    validateFiles(baseFile, encFile1, len);
+    // Roll the key of the encryption zone
+    List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
+    assertEquals("Expected 1 EZ", 1, zones.size());
+    String keyId = zones.get(0).getKeyId();
+    cluster.getNamesystem().getProvider().rollNewVersion(keyId);
+    cluster.getNamesystem().getFSDirectory().ezManager.kickMonitor();
+    // Read them back in and compare byte-by-byte
+    validateFiles(baseFile, encFile1, len);
+    // Write a new enc file and validate
+    final Path encFile2 = new Path(zone, "myfile2");
+    DFSTestUtil.createFile(fs, encFile2, len, (short) 1, 0xFEED);
+    // FEInfos should be different
+    FileEncryptionInfo feInfo1 = getFileEncryptionInfo(encFile1);
+    FileEncryptionInfo feInfo2 = getFileEncryptionInfo(encFile2);
+    assertFalse("EDEKs should be different", Arrays
+        .equals(feInfo1.getEncryptedDataEncryptionKey(),
+            feInfo2.getEncryptedDataEncryptionKey()));
+    assertNotEquals("Key was rolled, versions should be different",
+        feInfo1.getEzKeyVersionName(), feInfo2.getEzKeyVersionName());
+    // Contents still equal
+    validateFiles(encFile1, encFile2, len);
+  }
+
+  @Test(timeout = 60000)
+  public void testCipherSuiteNegotiation() throws Exception {
+    final HdfsAdmin dfsAdmin =
+        new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
+    final Path zone = new Path("/zone");
+    fs.mkdirs(zone);
+    dfsAdmin.createEncryptionZone(zone, null);
+    // Create a file in an EZ, which should succeed
+    DFSTestUtil
+        .createFile(fs, new Path(zone, "success1"), 0, (short) 1, 0xFEED);
+    // Pass no cipherSuites, fail
+    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(0);
+    try {
+      DFSTestUtil.createFile(fs, new Path(zone, "fail"), 0, (short) 1, 0xFEED);
+      fail("Created a file without specifying a CipherSuite!");
+    } catch (UnknownCipherSuiteException e) {
+      assertExceptionContains("No cipher suites", e);
+    }
+    // Pass some unknown cipherSuites, fail
+    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    try {
+      DFSTestUtil.createFile(fs, new Path(zone, "fail"), 0, (short) 1, 0xFEED);
+      fail("Created a file without specifying a CipherSuite!");
+    } catch (UnknownCipherSuiteException e) {
+      assertExceptionContains("No cipher suites", e);
+    }
+    // Pass some unknown and a good cipherSuites, success
+    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
+    fs.getClient().cipherSuites.add(CipherSuite.AES_CTR_NOPADDING);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    DFSTestUtil
+        .createFile(fs, new Path(zone, "success2"), 0, (short) 1, 0xFEED);
+    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
+    fs.getClient().cipherSuites.add(CipherSuite.AES_CTR_NOPADDING);
+    DFSTestUtil
+        .createFile(fs, new Path(zone, "success3"), 4096, (short) 1, 0xFEED);
+    // Check KeyProvider state
+    // Flushing the KP on the NN, since it caches, and init a test one
+    cluster.getNamesystem().getProvider().flush();
+    KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
+    List<String> keys = provider.getKeys();
+    assertEquals("Expected NN to have created one key per zone", 1,
+        keys.size());
+    List<KeyProvider.KeyVersion> allVersions = Lists.newArrayList();
+    for (String key : keys) {
+      List<KeyProvider.KeyVersion> versions = provider.getKeyVersions(key);
+      assertEquals("Should only have one key version per key", 1,
+          versions.size());
+      allVersions.addAll(versions);
+    }
+    // Check that the specified CipherSuite was correctly saved on the NN
+    for (int i = 2; i <= 3; i++) {
+      FileEncryptionInfo feInfo =
+          getFileEncryptionInfo(new Path(zone.toString() +
+              "/success" + i));
+      assertEquals(feInfo.getCipherSuite(), CipherSuite.AES_CTR_NOPADDING);
+    }
+  }
+
+}

+ 0 - 469
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZonesAPI.java

@@ -1,469 +0,0 @@
-/**
- * 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;
-
-import java.io.File;
-import java.io.IOException;
-import java.security.NoSuchAlgorithmException;
-import java.security.PrivilegedExceptionAction;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.UUID;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.crypto.CipherSuite;
-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.FSDataInputStream;
-import org.apache.hadoop.fs.FileEncryptionInfo;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.protocol.EncryptionZone;
-import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager;
-import org.apache.hadoop.security.AccessControlException;
-import org.apache.hadoop.security.UserGroupInformation;
-import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.log4j.Level;
-import org.apache.log4j.Logger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotEquals;
-import static org.junit.Assert.fail;
-
-public class TestEncryptionZonesAPI {
-
-  private static final Path TEST_PATH = new Path("/test");
-  private static final Path TEST_PATH_WITH_CHILD = new Path(TEST_PATH, "foo");
-  private static final Path TEST_PATH_WITH_MULTIPLE_CHILDREN =
-    new Path(TEST_PATH_WITH_CHILD, "baz");
-  private static final String TEST_KEYID = "mykeyid";
-  private final Configuration conf = new Configuration();
-  private MiniDFSCluster cluster;
-  private static File tmpDir;
-  private DistributedFileSystem fs;
-
-  @Before
-  public void setUpCluster() throws IOException {
-    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");
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
-    fs = (DistributedFileSystem) createFileSystem(conf);
-    Logger.getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);
-  }
-
-  protected FileSystem createFileSystem(Configuration conf) throws IOException {
-    return cluster.getFileSystem();
-  }
-
-  @After
-  public void shutDownCluster() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  /** Test failure of Create EZ on a directory that doesn't exist. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneDirectoryDoesntExist() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    try {
-      dfsAdmin.createEncryptionZone(TEST_PATH, null);
-      fail("expected /test doesn't exist");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("cannot find", e);
-    }
-  }
-
-  /** Test failure of Create EZ on a directory which is already an EZ. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneWhichAlreadyExists()
-    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);
-    try {
-      dfsAdmin.createEncryptionZone(TEST_PATH, null);
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("already in an encryption zone",
-          e);
-    }
-  }
-
-  /** Test success of Create EZ in which a key is created. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneAndGenerateKeyDirectoryEmpty()
-    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);
-  }
-
-  /** Test failure of Create EZ operation in an existing EZ. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneInExistingEncryptionZone()
-    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.createEncryptionZone(TEST_PATH_WITH_CHILD, null);
-      fail("EZ in an EZ");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("already in an encryption zone", e);
-    }
-  }
-
-  /** Test failure of creating an EZ using a non-empty directory. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneAndGenerateKeyDirectoryNotEmpty()
-    throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    FileSystem.create(fs, new Path("/test/foo"),
-            new FsPermission((short) 0777));
-    try {
-      dfsAdmin.createEncryptionZone(TEST_PATH, null);
-      fail("expected key doesn't exist");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("create an encryption zone", e);
-    }
-  }
-
-  /** Test failure of creating an EZ passing a key that doesn't exist. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneKeyDoesntExist() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    try {
-      dfsAdmin.createEncryptionZone(TEST_PATH, TEST_KEYID);
-      fail("expected key doesn't exist");
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains("doesn't exist.", e);
-    }
-    final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
-    Preconditions.checkState(zones.size() == 0, "More than one zone found?");
-  }
-
-  /** Test success of creating an EZ when they key exists. */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneKeyExist() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0777));
-    createAKey(TEST_KEYID);
-    dfsAdmin.createEncryptionZone(TEST_PATH, TEST_KEYID);
-    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=");
-  }
-
-  /** Helper function to create a key in the Key Provider. */
-  private void createAKey(String keyId)
-    throws NoSuchAlgorithmException, IOException {
-    KeyProvider provider =
-        cluster.getNameNode().getNamesystem().getProvider();
-    final KeyProvider.Options options = KeyProvider.options(conf);
-    provider.createKey(keyId, options);
-    provider.flush();
-  }
-
-  /** 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);
-
-    final UserGroupInformation user = UserGroupInformation.
-      createUserForTesting("user", new String[] { "mygroup" });
-
-    FileSystem.mkdirs(fs, TEST_PATH, new FsPermission((short) 0700));
-
-    user.doAs(new PrivilegedExceptionAction<Object>() {
-        @Override
-        public Object run() throws Exception {
-          final HdfsAdmin userAdmin =
-            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-          try {
-            userAdmin.createEncryptionZone(TEST_PATH, null);
-            fail("createEncryptionZone is superuser-only operation");
-          } catch (AccessControlException e) {
-            GenericTestUtils.assertExceptionContains(
-                    "Superuser privilege is required", e);
-          }
-          return null;
-        }
-      });
-  }
-
-  /**
-   * Test success of creating an encryption zone a few levels down.
-   */
-  @Test(timeout = 60000)
-  public void testCreateEncryptionZoneDownAFewLevels()
-    throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH_WITH_MULTIPLE_CHILDREN,
-      new FsPermission((short) 0777));
-    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/foo/baz, keyId=");
-  }
-
-  /** Test failure of creating an EZ using a non-empty directory. */
-  @Test(timeout = 60000)
-  public void testCreateFileInEncryptionZone() 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, TEST_PATH_WITH_CHILD, new FsPermission((short) 0777));
-
-    final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
-    final EncryptionZone ez = zones.get(0);
-      GenericTestUtils.assertMatches(ez.toString(),
-         "EncryptionZone \\[path=/test, keyId=");
-  }
-
-  /** Test listing encryption zones. */
-  @Test(timeout = 60000)
-  public void testListEncryptionZones() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    final int N_EZs = 5;
-    final Set<String> ezPathNames = new HashSet<String>(N_EZs);
-    for (int i = 0; i < N_EZs; i++) {
-      final Path p = new Path(TEST_PATH, "" + i);
-      ezPathNames.add(p.toString());
-      FileSystem.mkdirs(fs, p, new FsPermission((short) 0777));
-      dfsAdmin.createEncryptionZone(p, null);
-    }
-
-    final List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
-    Preconditions.checkState(zones.size() == N_EZs, "wrong number of EZs returned");
-    for (EncryptionZone z : zones) {
-      final String ezPathName = z.getPath();
-      Preconditions.checkState(ezPathNames.remove(
-          ezPathName), "Path " + ezPathName + " not returned from listEZ");
-    }
-    Preconditions.checkState(ezPathNames.size() == 0);
-  }
-
-  /** Test listing encryption zones as a non super user. */
-  @Test(timeout = 60000)
-  public void testListEncryptionZonesAsNonSuperUser() throws Exception {
-    final HdfsAdmin dfsAdmin =
-      new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-
-    final UserGroupInformation user = UserGroupInformation.
-      createUserForTesting("user", new String[] {"mygroup"});
-
-    final Path TEST_PATH_SUPERUSER_ONLY = new Path(TEST_PATH, "superuseronly");
-    final Path TEST_PATH_ALL = new Path(TEST_PATH, "accessall");
-
-    FileSystem.mkdirs(fs, TEST_PATH_SUPERUSER_ONLY,
-      new FsPermission((short) 0700));
-    dfsAdmin.createEncryptionZone(TEST_PATH_SUPERUSER_ONLY, null);
-    FileSystem.mkdirs(fs, TEST_PATH_ALL,
-      new FsPermission((short) 0707));
-    dfsAdmin.createEncryptionZone(TEST_PATH_ALL, null);
-
-    user.doAs(new PrivilegedExceptionAction<Object>() {
-      @Override
-      public Object run() throws Exception {
-        final HdfsAdmin userAdmin =
-                new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-        try {
-          final List<EncryptionZone> zones = userAdmin.listEncryptionZones();
-        } catch (AccessControlException e) {
-          GenericTestUtils.assertExceptionContains(
-                  "Superuser privilege is required", e);
-        }
-        return null;
-      }
-    });
-  }
-
-  /** Test success of Rename EZ on a directory which is already an EZ. */
-  @Test(timeout = 60000)
-  public void testRenameEncryptionZone()
-          throws Exception {
-    final HdfsAdmin dfsAdmin =
-            new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    FileSystem.mkdirs(fs, TEST_PATH_WITH_CHILD,
-      new FsPermission((short) 0777));
-    dfsAdmin.createEncryptionZone(TEST_PATH_WITH_CHILD, null);
-    FileSystem.mkdirs(fs, TEST_PATH_WITH_MULTIPLE_CHILDREN,
-       new FsPermission((short) 0777));
-    try {
-      fs.rename(TEST_PATH_WITH_MULTIPLE_CHILDREN, TEST_PATH);
-    } catch (IOException e) {
-      GenericTestUtils.assertExceptionContains(
-              "/test/foo/baz can't be moved from an encryption zone.", e);
-    }
-  }
-
-  @Test(timeout = 60000)
-  public void testCipherSuiteNegotiation() throws Exception {
-    final HdfsAdmin dfsAdmin =
-        new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    final Path zone = new Path("/zone");
-    fs.mkdirs(zone);
-    dfsAdmin.createEncryptionZone(zone, null);
-    // Create a file in an EZ, which should succeed
-    DFSTestUtil.createFile(fs, new Path(zone, "success1"), 0, (short) 1,
-        0xFEED);
-    // Pass no cipherSuites, fail
-    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(0);
-    try {
-      DFSTestUtil.createFile(fs, new Path(zone, "fail"), 0, (short) 1,
-          0xFEED);
-      fail("Created a file without specifying a CipherSuite!");
-    } catch (UnknownCipherSuiteException e) {
-      GenericTestUtils.assertExceptionContains("No cipher suites", e);
-    }
-    // Pass some unknown cipherSuites, fail
-    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    try {
-      DFSTestUtil.createFile(fs, new Path(zone, "fail"), 0, (short) 1,
-          0xFEED);
-      fail("Created a file without specifying a CipherSuite!");
-    } catch (UnknownCipherSuiteException e) {
-      GenericTestUtils.assertExceptionContains("No cipher suites", e);
-    }
-    // Pass some unknown and a good cipherSuites, success
-    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
-    fs.getClient().cipherSuites.add(CipherSuite.AES_CTR_NOPADDING);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    DFSTestUtil.createFile(fs, new Path(zone, "success2"), 0, (short) 1,
-        0xFEED);
-    fs.getClient().cipherSuites = Lists.newArrayListWithCapacity(3);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    fs.getClient().cipherSuites.add(CipherSuite.UNKNOWN);
-    fs.getClient().cipherSuites.add(CipherSuite.AES_CTR_NOPADDING);
-    DFSTestUtil.createFile(fs, new Path(zone, "success3"), 4096, (short) 1,
-        0xFEED);
-    // Check KeyProvider state
-    // Flushing the KP on the NN, since it caches, and init a test one
-    cluster.getNamesystem().getProvider().flush();
-    KeyProvider provider = KeyProviderFactory.getProviders(conf).get(0);
-    List<String> keys = provider.getKeys();
-    assertEquals("Expected NN to have created one key per zone", 1,
-        keys.size());
-    List<KeyProvider.KeyVersion> allVersions = Lists.newArrayList();
-    for (String key : keys) {
-      List<KeyProvider.KeyVersion> versions = provider.getKeyVersions(key);
-      assertEquals("Should only have one key version per key", 1,
-          versions.size());
-      allVersions.addAll(versions);
-    }
-    // Check that the specified CipherSuite was correctly saved on the NN
-    for (int i=2; i<=3; i++) {
-      FileEncryptionInfo feInfo =
-          getFileEncryptionInfo(new Path(zone.toString() +
-              "/success" + i));
-      assertEquals(feInfo.getCipherSuite(), CipherSuite.AES_CTR_NOPADDING);
-    }
-  }
-
-  private void validateFiles(Path p1, Path p2, int len) throws Exception {
-    FSDataInputStream in1 = fs.open(p1);
-    FSDataInputStream in2 = fs.open(p2);
-    for (int i=0; i<len; i++) {
-      assertEquals("Mismatch at byte " + i, in1.read(), in2.read());
-    }
-    in1.close();
-    in2.close();
-  }
-
-  private FileEncryptionInfo getFileEncryptionInfo(Path path) throws Exception {
-    LocatedBlocks blocks = fs.getClient().getLocatedBlocks(path.toString(), 0);
-    return blocks.getFileEncryptionInfo();
-  }
-
-  @Test(timeout = 120000)
-  public void testReadWrite() throws Exception {
-    final HdfsAdmin dfsAdmin =
-        new HdfsAdmin(FileSystem.getDefaultUri(conf), conf);
-    // Create a base file for comparison
-    final Path baseFile = new Path("/base");
-    final int len = 8192;
-    DFSTestUtil.createFile(fs, baseFile, len, (short) 1, 0xFEED);
-    // Create the first enc file
-    final Path zone = new Path("/zone");
-    fs.mkdirs(zone);
-    dfsAdmin.createEncryptionZone(zone, null);
-    final Path encFile1 = new Path(zone, "myfile");
-    DFSTestUtil.createFile(fs, encFile1, len, (short) 1, 0xFEED);
-    // Read them back in and compare byte-by-byte
-    validateFiles(baseFile, encFile1, len);
-    // Roll the key of the encryption zone
-    List<EncryptionZone> zones = dfsAdmin.listEncryptionZones();
-    assertEquals("Expected 1 EZ", 1, zones.size());
-    String keyId = zones.get(0).getKeyId();
-    cluster.getNamesystem().getProvider().rollNewVersion(keyId);
-    cluster.getNamesystem().getFSDirectory().ezManager.kickMonitor();
-    // Read them back in and compare byte-by-byte
-    validateFiles(baseFile, encFile1, len);
-    // Write a new enc file and validate
-    final Path encFile2 = new Path(zone, "myfile2");
-    DFSTestUtil.createFile(fs, encFile2, len, (short) 1, 0xFEED);
-    // FEInfos should be different
-    FileEncryptionInfo feInfo1 = getFileEncryptionInfo(encFile1);
-    FileEncryptionInfo feInfo2 = getFileEncryptionInfo(encFile2);
-    assertFalse("EDEKs should be different", Arrays.equals(
-        feInfo1.getEncryptedDataEncryptionKey(),
-        feInfo2.getEncryptedDataEncryptionKey()));
-    assertNotEquals("Key was rolled, versions should be different",
-        feInfo1.getEzKeyVersionName(), feInfo2.getEzKeyVersionName());
-    // Contents still equal
-    validateFiles(encFile1, encFile2, len);
-  }
-}

+ 0 - 70
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileContextEncryptionZones.java

@@ -1,70 +0,0 @@
-/**
- * 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.server.namenode;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.EnumSet;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileContext;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.XAttrSetFlag;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.TestEncryptionZonesAPI;
-import org.junit.BeforeClass;
-
-/**
- * Tests of encryption zone operations using FileContext APIs.
- */
-public class TestFileContextEncryptionZones extends TestEncryptionZonesAPI  {
-
-  @Override
-  protected FileSystem createFileSystem(Configuration conf) throws IOException {
-    FileContextFS fcFs = new FileContextFS();
-    fcFs.initialize(FileSystem.getDefaultUri(conf), conf);
-    return fcFs;
-  }
-
-  /**
-   * This reuses FSXAttrBaseTest's testcases by creating a filesystem
-   * implementation which uses FileContext by only overriding the xattr related
-   * methods. Other operations will use the normal filesystem.
-   */
-  public static class FileContextFS extends DistributedFileSystem {
-
-    private FileContext fc;
-
-    @Override
-    public void initialize(URI uri, Configuration conf) throws IOException {
-      super.initialize(uri, conf);
-      fc = FileContext.getFileContext(conf);
-    }
-
-    @Override
-    public boolean rename(Path src, Path dst) throws IOException {
-      fc.rename(src, dst);
-      return true;
-    }
-  }
-}