You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2016/02/23 00:10:45 UTC

hadoop git commit: HDFS-9688. Test the effect of nested encryption zones in HDFS downgrade.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2.7 541dc02a6 -> 79027309c


HDFS-9688. Test the effect of nested encryption zones in HDFS downgrade.

Change-Id: I2a802325a2bc82c97012d2c9fb388b292fe3af8f


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/79027309
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/79027309
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/79027309

Branch: refs/heads/branch-2.7
Commit: 79027309c9abb7a0ad5f4234fb18643ae57d3f72
Parents: 541dc02
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Feb 22 15:10:31 2016 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Feb 22 15:10:31 2016 -0800

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt     |   3 +
 .../server/namenode/EncryptionZoneManager.java  |  16 +-
 .../namenode/TestNestedEncryptionZones.java     | 220 +++++++++++++++++++
 3 files changed, 238 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/79027309/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
index 2ff5bc8..6f1c09c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
@@ -1253,6 +1253,9 @@ Release 2.6.5 - UNRELEASED
      HDFS-9629. Update the footer of Web UI to show year 2016
      (Xiao Chen and Vinayakumar B via vinayakumarb)
 
+     HDFS-9688. Test the effect of nested encryption zones in HDFS downgrade.
+     (zhz)
+
   OPTIMIZATIONS
 
   BUG FIXES

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79027309/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
index 3fe748d..5aa9e1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.NavigableMap;
 import java.util.TreeMap;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import org.apache.hadoop.conf.Configuration;
@@ -57,6 +58,9 @@ public class EncryptionZoneManager {
   public static Logger LOG = LoggerFactory.getLogger(EncryptionZoneManager
       .class);
 
+  @VisibleForTesting
+  private boolean allowNestedEZ = false;
+
   /**
    * EncryptionZoneInt is the internal representation of an encryption zone. The
    * external representation of an EZ is embodied in an EncryptionZone and
@@ -278,6 +282,16 @@ public class EncryptionZoneManager {
     }
   }
 
+  @VisibleForTesting
+  void setAllowNestedEZ() {
+    allowNestedEZ = true;
+  }
+
+  @VisibleForTesting
+  void setDisallowNestedEZ() {
+    allowNestedEZ = false;
+  }
+
   /**
    * Create a new encryption zone.
    * <p/>
@@ -299,7 +313,7 @@ public class EncryptionZoneManager {
       throw new IOException("Attempt to create an encryption zone for a file.");
     }
     EncryptionZoneInt ezi = getEncryptionZoneForPath(srcIIP);
-    if (ezi != null) {
+    if (!allowNestedEZ && ezi != null) {
       throw new IOException("Directory " + src + " is already in an " +
           "encryption zone. (" + getFullPathName(ezi) + ")");
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/79027309/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
new file mode 100644
index 0000000..ce03325
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestNestedEncryptionZones.java
@@ -0,0 +1,220 @@
+/**
+ * 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 org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.crypto.key.JavaKeyStoreProvider;
+import org.apache.hadoop.fs.FileSystemTestHelper;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.log4j.Level;
+import org.apache.log4j.Logger;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test the behavior of nested encryption zones.
+ */
+public class TestNestedEncryptionZones {
+  private File testRootDir;
+  private final String TOP_EZ_KEY = "topezkey";
+  private final String NESTED_EZ_KEY = "nestedezkey";
+
+  private MiniDFSCluster cluster;
+  protected DistributedFileSystem fs;
+
+  private final Path rootDir = new Path("/");
+  private final Path rawDir = new Path("/.reserved/raw/");
+  private final Path topEZDir = new Path(rootDir, "topEZ");
+  private final Path nestedEZDir = new Path(topEZDir, "nestedEZ");
+
+  private final Path topEZBaseFile = new Path(rootDir, "topEZBaseFile");
+  private Path topEZFile = new Path(topEZDir, "file");
+  private Path topEZRawFile = new Path(rawDir, "topEZ/file");
+
+  private final Path nestedEZBaseFile = new Path(rootDir, "nestedEZBaseFile");
+  private Path nestedEZFile = new Path(nestedEZDir, "file");
+  private Path nestedEZRawFile = new Path(rawDir, "topEZ/nestedEZ/file");
+
+  // File length
+  private final int len = 8196;
+
+  private String getKeyProviderURI() {
+    return JavaKeyStoreProvider.SCHEME_NAME + "://file" +
+        new Path(testRootDir.toString(), "test.jks").toUri();
+  }
+
+  private void setProvider() {
+    // Need to set the client's KeyProvider to the NN's for JKS,
+    // else the updates do not get flushed properly
+    fs.getClient().setKeyProvider(cluster.getNameNode().getNamesystem()
+        .getProvider());
+  }
+
+  @Before
+  public void setup() throws Exception {
+    Configuration conf = new HdfsConfiguration();
+    FileSystemTestHelper fsHelper = new FileSystemTestHelper();
+    // Set up java key store
+    String testRoot = fsHelper.getTestRootDir();
+    testRootDir = new File(testRoot).getAbsoluteFile();
+    conf.set(DFSConfigKeys.DFS_ENCRYPTION_KEY_PROVIDER_URI, getKeyProviderURI());
+    conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
+    // Lower the batch size for testing
+    conf.setInt(DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
+        2);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
+    Logger.getLogger(EncryptionZoneManager.class).setLevel(Level.TRACE);
+    fs = cluster.getFileSystem();
+    setProvider();
+
+    // Create test keys and EZs
+    DFSTestUtil.createKey(TOP_EZ_KEY, cluster, conf);
+    DFSTestUtil.createKey(NESTED_EZ_KEY, cluster, conf);
+    fs.mkdir(topEZDir, FsPermission.getDirDefault());
+    fs.createEncryptionZone(topEZDir, TOP_EZ_KEY);
+    fs.mkdir(nestedEZDir, FsPermission.getDirDefault());
+
+    // Allow setting nested EZ temporarily
+    cluster.getNamesystem().getFSDirectory().ezManager.setAllowNestedEZ();
+    fs.createEncryptionZone(nestedEZDir, NESTED_EZ_KEY);
+    cluster.getNamesystem().getFSDirectory().ezManager.setDisallowNestedEZ();
+
+    DFSTestUtil.createFile(fs, topEZBaseFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, topEZFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, nestedEZBaseFile, len, (short) 1, 0xFEED);
+    DFSTestUtil.createFile(fs, nestedEZFile, len, (short) 1, 0xFEED);
+  }
+
+  @Test(timeout = 60000)
+  public void testNestedEncryptionZones() throws Exception {
+    verifyEncryption();
+
+    // Restart NameNode to test if nested EZs can be loaded from edit logs
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    verifyEncryption();
+
+    // Checkpoint and restart NameNode, to test if nested EZs can be loaded
+    // from fsimage
+    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    fs.saveNamespace();
+    fs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNodes();
+    cluster.waitActive();
+    verifyEncryption();
+
+    Path renamedTopEZFile = new Path(topEZDir, "renamedFile");
+    Path renamedNestedEZFile = new Path(nestedEZDir, "renamedFile");
+    try {
+      fs.rename(topEZFile, renamedTopEZFile);
+      fs.rename(nestedEZFile, renamedNestedEZFile);
+    } catch (Exception e) {
+      fail("Should be able to rename files within the same EZ.");
+    }
+
+    topEZFile = renamedTopEZFile;
+    nestedEZFile = renamedNestedEZFile;
+    topEZRawFile = new Path(rawDir, "topEZ/renamedFile");
+    nestedEZRawFile = new Path(rawDir, "topEZ/nestedEZ/renamedFile");
+    verifyEncryption();
+
+    // Verify that files in top EZ cannot be moved into the nested EZ, and
+    // vice versa.
+    try {
+      fs.rename(topEZFile, new Path(nestedEZDir, "movedTopEZFile"));
+      fail("Shouldn't be able to rename between top EZ and nested EZ.");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains(
+          "can't be moved from encryption zone " + topEZDir.toString() +
+              " to encryption zone " + nestedEZDir.toString()));
+    }
+    try {
+      fs.rename(nestedEZFile, new Path(topEZDir, "movedNestedEZFile"));
+      fail("Shouldn't be able to rename between top EZ and nested EZ.");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains(
+          "can't be moved from encryption zone " + nestedEZDir.toString() +
+              " to encryption zone " + topEZDir.toString()));
+    }
+
+    // Verify that the nested EZ cannot be moved out of the top EZ.
+    try {
+      fs.rename(nestedEZFile, new Path(rootDir, "movedNestedEZFile"));
+      fail("Shouldn't be able to move the nested EZ out of the top EZ.");
+    } catch (Exception e) {
+      assertTrue(e.getMessage().contains(
+          "can't be moved from an encryption zone"));
+    }
+
+    // Verify that a non-nested EZ cannot be moved into another EZ
+    Path topEZ2Dir = new Path(rootDir, "topEZ2");
+    fs.mkdir(topEZ2Dir, FsPermission.getDirDefault());
+    fs.createEncryptionZone(topEZ2Dir, TOP_EZ_KEY);
+    try {
+      fs.rename(topEZ2Dir, new Path(topEZDir, "topEZ2"));
+      fail("Shouldn't be able to move a non-nested EZ into another " +
+          "existing EZ.");
+    } catch (Exception e){
+      assertTrue(e.getMessage().contains(
+          "can't be moved from encryption zone " + topEZ2Dir.toString() +
+              " to encryption zone"));
+    }
+
+    try {
+      fs.rename(topEZDir, new Path(rootDir, "newTopEZDir"));
+    } catch (Exception e) {
+      fail("Should be able to rename the root dir of an EZ.");
+    }
+
+    try {
+      fs.rename(new Path(rootDir, "newTopEZDir/nestedEZDir"),
+          new Path(rootDir, "newTopEZDir/newNestedEZDir"));
+    } catch (Exception e) {
+      fail("Should be able to rename the nested EZ dir within " +
+          "the same top EZ.");
+    }
+  }
+
+  private void verifyEncryption() throws Exception {
+    assertEquals("Top EZ dir is encrypted",
+        true, fs.getFileStatus(topEZDir).isEncrypted());
+    assertEquals("Nested EZ dir is encrypted",
+        true, fs.getFileStatus(nestedEZDir).isEncrypted());
+    assertEquals("Top zone file is encrypted",
+        true, fs.getFileStatus(topEZFile).isEncrypted());
+    assertEquals("Nested zone file is encrypted",
+        true, fs.getFileStatus(nestedEZFile).isEncrypted());
+
+    DFSTestUtil.verifyFilesEqual(fs, topEZBaseFile, topEZFile, len);
+    DFSTestUtil.verifyFilesEqual(fs, nestedEZBaseFile, nestedEZFile, len);
+    DFSTestUtil.verifyFilesNotEqual(fs, topEZRawFile, nestedEZRawFile, len);
+  }
+}