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 wa...@apache.org on 2016/06/07 05:15:06 UTC

hadoop git commit: Revert "HDFS-10458. getFileEncryptionInfo should return quickly for non-encrypted cluster."

Repository: hadoop
Updated Branches:
  refs/heads/branch-2.8 51a16b1ff -> e91eba2f8


Revert "HDFS-10458. getFileEncryptionInfo should return quickly for non-encrypted cluster."

This reverts commit 0274636529cfac4d64582a7c21631aebbf4deb1b.


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

Branch: refs/heads/branch-2.8
Commit: e91eba2f81951c34690b0ec27a0c72fba1dcc862
Parents: 51a16b1
Author: Wangda Tan <wa...@apache.org>
Authored: Mon Jun 6 22:13:02 2016 -0700
Committer: Wangda Tan <wa...@apache.org>
Committed: Mon Jun 6 22:13:02 2016 -0700

----------------------------------------------------------------------
 .../server/namenode/EncryptionZoneManager.java  | 35 +++-----------------
 .../server/namenode/FSDirEncryptionZoneOp.java  |  2 +-
 2 files changed, 6 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e91eba2f/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 41dbb59..8454c04 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
@@ -95,7 +95,7 @@ public class EncryptionZoneManager {
     }
   }
 
-  private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
+  private final TreeMap<Long, EncryptionZoneInt> encryptionZones;
   private final FSDirectory dir;
   private final int maxListEncryptionZonesResponses;
 
@@ -106,6 +106,7 @@ public class EncryptionZoneManager {
    */
   public EncryptionZoneManager(FSDirectory dir, Configuration conf) {
     this.dir = dir;
+    encryptionZones = new TreeMap<Long, EncryptionZoneInt>();
     maxListEncryptionZonesResponses = conf.getInt(
         DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES,
         DFSConfigKeys.DFS_NAMENODE_LIST_ENCRYPTION_ZONES_NUM_RESPONSES_DEFAULT
@@ -142,9 +143,6 @@ public class EncryptionZoneManager {
       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
     final EncryptionZoneInt ez = new EncryptionZoneInt(
         inodeId, suite, version, keyName);
-    if (encryptionZones == null) {
-      encryptionZones = new TreeMap<>();
-    }
     encryptionZones.put(inodeId, ez);
   }
 
@@ -155,9 +153,7 @@ public class EncryptionZoneManager {
    */
   void removeEncryptionZone(Long inodeId) {
     assert dir.hasWriteLock();
-    if (hasCreatedEncryptionZone()) {
-      encryptionZones.remove(inodeId);
-    }
+    encryptionZones.remove(inodeId);
   }
 
   /**
@@ -205,9 +201,6 @@ public class EncryptionZoneManager {
   private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
-    if (!hasCreatedEncryptionZone()) {
-      return null;
-    }
     List<INode> inodes = iip.getReadOnlyINodes();
     for (int i = inodes.size() - 1; i >= 0; i--) {
       final INode inode = inodes.get(i);
@@ -320,8 +313,7 @@ public class EncryptionZoneManager {
       throw new IOException("Attempt to create an encryption zone for a file.");
     }
 
-    if (hasCreatedEncryptionZone() && encryptionZones.
-        get(srcINode.getId()) != null) {
+    if (encryptionZones.get(srcINode.getId()) != null) {
       throw new IOException("Directory " + src + " is already an encryption " +
           "zone.");
     }
@@ -348,9 +340,6 @@ public class EncryptionZoneManager {
   BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
     assert dir.hasReadLock();
-    if (!hasCreatedEncryptionZone()) {
-      return new BatchedListEntries<EncryptionZone>(Lists.newArrayList(), false);
-    }
     NavigableMap<Long, EncryptionZoneInt> tailMap = encryptionZones.tailMap
         (prevId, false);
     final int numResponses = Math.min(maxListEncryptionZonesResponses,
@@ -390,18 +379,7 @@ public class EncryptionZoneManager {
    * @return number of encryption zones.
    */
   public int getNumEncryptionZones() {
-    return hasCreatedEncryptionZone() ?
-        encryptionZones.size() : 0;
-  }
-
-  /**
-   * @return Whether there has been any attempt to create an encryption zone in
-   * the cluster at all. If not, it is safe to quickly return null when
-   * checking the encryption information of any file or directory in the
-   * cluster.
-   */
-  public boolean hasCreatedEncryptionZone() {
-    return encryptionZones != null;
+    return encryptionZones.size();
   }
 
   /**
@@ -409,9 +387,6 @@ public class EncryptionZoneManager {
    */
   String[] getKeyNames() {
     assert dir.hasReadLock();
-    if (!hasCreatedEncryptionZone()) {
-      return new String[0];
-    }
     String[] ret = new String[encryptionZones.size()];
     int index = 0;
     for (Map.Entry<Long, EncryptionZoneInt> entry : encryptionZones

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e91eba2f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index 2997179..bd25419 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -254,7 +254,7 @@ final class FSDirEncryptionZoneOp {
   static FileEncryptionInfo getFileEncryptionInfo(final FSDirectory fsd,
       final INode inode, final int snapshotId, final INodesInPath iip)
       throws IOException {
-    if (!inode.isFile() || !fsd.ezManager.hasCreatedEncryptionZone()) {
+    if (!inode.isFile()) {
       return null;
     }
     fsd.readLock();


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org