Browse Source

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

(cherry picked from commit 9b68eda2746702246c7f4b4abe412c974ba56339)
(cherry picked from commit 6de9213df111a9a4ed875db995d67af72d08a798)
(cherry picked from commit 06e38c835d0ad9619d4bea662f2dd7d0f62007a9)
(cherry picked from commit 0274636529cfac4d64582a7c21631aebbf4deb1b)
Zhe Zhang 9 năm trước cách đây
mục cha
commit
1b00cebcdb

+ 24 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -97,7 +97,7 @@ public class EncryptionZoneManager {
     }
   }
 
-  private final TreeMap<Long, EncryptionZoneInt> encryptionZones;
+  private TreeMap<Long, EncryptionZoneInt> encryptionZones = null;
   private final FSDirectory dir;
   private final int maxListEncryptionZonesResponses;
 
@@ -108,7 +108,6 @@ 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
@@ -145,6 +144,9 @@ public class EncryptionZoneManager {
       CipherSuite suite, CryptoProtocolVersion version, String keyName) {
     final EncryptionZoneInt ez = new EncryptionZoneInt(
         inodeId, suite, version, keyName);
+    if (encryptionZones == null) {
+      encryptionZones = new TreeMap<Long, EncryptionZoneInt>();
+    }
     encryptionZones.put(inodeId, ez);
   }
 
@@ -155,7 +157,9 @@ public class EncryptionZoneManager {
    */
   void removeEncryptionZone(Long inodeId) {
     assert dir.hasWriteLock();
-    encryptionZones.remove(inodeId);
+    if (hasCreatedEncryptionZone()) {
+      encryptionZones.remove(inodeId);
+    }
   }
 
   /**
@@ -203,6 +207,9 @@ public class EncryptionZoneManager {
   private EncryptionZoneInt getEncryptionZoneForPath(INodesInPath iip) {
     assert dir.hasReadLock();
     Preconditions.checkNotNull(iip);
+    if (!hasCreatedEncryptionZone()) {
+      return null;
+    }
     final INode[] inodes = iip.getINodes();
     for (int i = inodes.length - 1; i >= 0; i--) {
       final INode inode = inodes[i];
@@ -337,6 +344,10 @@ public class EncryptionZoneManager {
   BatchedListEntries<EncryptionZone> listEncryptionZones(long prevId)
       throws IOException {
     assert dir.hasReadLock();
+    if (!hasCreatedEncryptionZone()) {
+      final List<EncryptionZone> emptyZones = Lists.newArrayList();
+      return new BatchedListEntries<EncryptionZone>(emptyZones, false);
+    }
     NavigableMap<Long, EncryptionZoneInt> tailMap = encryptionZones.tailMap
         (prevId, false);
     final int numResponses = Math.min(maxListEncryptionZonesResponses,
@@ -371,4 +382,14 @@ public class EncryptionZoneManager {
     final boolean hasMore = (numResponses < tailMap.size());
     return new BatchedListEntries<EncryptionZone>(zones, hasMore);
   }
+
+  /**
+   * @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;
+  }
 }

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

@@ -2888,7 +2888,7 @@ public class FSDirectory implements Closeable {
    */
   FileEncryptionInfo getFileEncryptionInfo(INode inode, int snapshotId,
       INodesInPath iip) throws IOException {
-    if (!inode.isFile()) {
+    if (!inode.isFile() || !ezManager.hasCreatedEncryptionZone()) {
       return null;
     }
     readLock();