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 we...@apache.org on 2021/04/26 03:43:29 UTC

[hadoop] branch trunk updated (88a550b -> 2621d3f)

This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a change to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git.


    from 88a550b  HADOOP-17112. S3A committers can't handle whitespace in paths. (#2953)
     new 90c6caf  Revert "HDFS-15624. fix the function of setting quota by storage type (#2377)"
     new 2621d3f  HDFS-15566. NN restart fails after RollingUpgrade from 3.1.3/3.2.1 to 3.3.0. Contributed by Brahma Reddy Battula.

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/hadoop/fs/StorageType.java     |  7 +--
 .../java/org/apache/hadoop/fs/shell/TestCount.java |  4 +-
 .../server/federation/router/TestRouterQuota.java  | 14 +++---
 .../hadoop/hdfs/server/namenode/FSEditLogOp.java   | 58 ++++++++++++++++++----
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  7 ---
 .../server/namenode/NameNodeLayoutVersion.java     |  2 +-
 .../apache/hadoop/hdfs/TestBlockStoragePolicy.java |  6 +--
 .../hadoop/hdfs/protocol/TestLayoutVersion.java    |  2 +-
 8 files changed, 66 insertions(+), 34 deletions(-)

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


[hadoop] 02/02: HDFS-15566. NN restart fails after RollingUpgrade from 3.1.3/3.2.1 to 3.3.0. Contributed by Brahma Reddy Battula.

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 2621d3f15bfae9b3820f70c0dfda015b1d474a97
Author: Brahma Reddy Battula <br...@apache.org>
AuthorDate: Mon Apr 26 11:29:41 2021 +0800

    HDFS-15566. NN restart fails after RollingUpgrade from 3.1.3/3.2.1 to 3.3.0. Contributed by Brahma Reddy Battula.
    
    Signed-off-by: Wei-Chiu Chuang <we...@apache.org>
---
 .../hadoop/hdfs/server/namenode/FSEditLogOp.java   | 58 ++++++++++++++++++----
 .../server/namenode/NameNodeLayoutVersion.java     |  3 +-
 .../hadoop/hdfs/protocol/TestLayoutVersion.java    |  3 +-
 3 files changed, 52 insertions(+), 12 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
index feff8b4..69c95b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogOp.java
@@ -3477,17 +3477,30 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion) throws IOException {
       snapshotRoot = FSImageSerialization.readString(in);
       snapshotName = FSImageSerialization.readString(in);
-      mtime = FSImageSerialization.readLong(in);
-      
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        mtime = FSImageSerialization.readLong(in);
+      }
       // read RPC ids if necessary
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
+      throw new IOException("Unsupported without logversion");
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out, int logVersion)
+        throws IOException {
       FSImageSerialization.writeString(snapshotRoot, out);
       FSImageSerialization.writeString(snapshotName, out);
-      FSImageSerialization.writeLong(mtime, out);
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        FSImageSerialization.writeLong(mtime, out);
+      }
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
@@ -3569,17 +3582,30 @@ public abstract class FSEditLogOp {
     void readFields(DataInputStream in, int logVersion) throws IOException {
       snapshotRoot = FSImageSerialization.readString(in);
       snapshotName = FSImageSerialization.readString(in);
-      mtime = FSImageSerialization.readLong(in);
-      
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        mtime = FSImageSerialization.readLong(in);
+      }
       // read RPC ids if necessary
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
+      throw new IOException("Unsupported without logversion");
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out, int logVersion)
+        throws IOException {
       FSImageSerialization.writeString(snapshotRoot, out);
       FSImageSerialization.writeString(snapshotName, out);
-      FSImageSerialization.writeLong(mtime, out);
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        FSImageSerialization.writeLong(mtime, out);
+      }
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
@@ -3670,19 +3696,31 @@ public abstract class FSEditLogOp {
       snapshotRoot = FSImageSerialization.readString(in);
       snapshotOldName = FSImageSerialization.readString(in);
       snapshotNewName = FSImageSerialization.readString(in);
-      mtime = FSImageSerialization.readLong(in);
-      
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        mtime = FSImageSerialization.readLong(in);
+      }
       // read RPC ids if necessary
       readRpcIds(in, logVersion);
     }
 
     @Override
     public void writeFields(DataOutputStream out) throws IOException {
+      throw new IOException("Unsupported without logversion");
+    }
+
+    @Override
+    public void writeFields(DataOutputStream out, int logVersion)
+        throws IOException {
       FSImageSerialization.writeString(snapshotRoot, out);
       FSImageSerialization.writeString(snapshotOldName, out);
       FSImageSerialization.writeString(snapshotNewName, out);
-      FSImageSerialization.writeLong(mtime, out);
-      
+      if (NameNodeLayoutVersion
+          .supports(NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME,
+              logVersion)) {
+        FSImageSerialization.writeLong(mtime, out);
+      }
       writeRpcIds(rpcClientId, rpcCallId, out);
     }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index 297ca74..bcb3714 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@ -89,7 +89,8 @@ public class NameNodeLayoutVersion {
     APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
     QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"),
     ERASURE_CODING(-64, -61, "Support erasure coding"),
-    EXPANDED_STRING_TABLE(-65, -61, "Support expanded string table in fsimage");
+    EXPANDED_STRING_TABLE(-65, -61, "Support expanded string table in fsimage"),
+    SNAPSHOT_MODIFICATION_TIME(-66, -61, "Support modification time for snapshot");
 
     private final FeatureInfo info;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
index 2c9905d..9662942 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
@@ -128,7 +128,8 @@ public class TestLayoutVersion {
         NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
         NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE,
         NameNodeLayoutVersion.Feature.ERASURE_CODING,
-        NameNodeLayoutVersion.Feature.EXPANDED_STRING_TABLE);
+        NameNodeLayoutVersion.Feature.EXPANDED_STRING_TABLE,
+        NameNodeLayoutVersion.Feature.SNAPSHOT_MODIFICATION_TIME);
     for (LayoutFeature f : compatibleFeatures) {
       assertEquals(String.format("Expected minimum compatible layout version " +
           "%d for feature %s.", baseLV, f), baseLV,

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


[hadoop] 01/02: Revert "HDFS-15624. fix the function of setting quota by storage type (#2377)"

Posted by we...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

weichiu pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 90c6caf6505ea89c90ad59a52dc471ba6a24d102
Author: Wei-Chiu Chuang <we...@apache.org>
AuthorDate: Mon Apr 26 11:27:15 2021 +0800

    Revert "HDFS-15624. fix the function of setting quota by storage type (#2377)"
    
    This reverts commit 394b9f7a5c42ffa38bf7c3a44e7d50a4d19414f9.
    
    Ref: HDFS-15995.
    Had to revert this commit, so we can commit HDFS-15566 (a critical bug preventing rolling upgrade to Hadoop 3.3)
    Will re-work this fix again later.
---
 .../src/main/java/org/apache/hadoop/fs/StorageType.java    |  7 ++++---
 .../test/java/org/apache/hadoop/fs/shell/TestCount.java    |  4 ++--
 .../hdfs/server/federation/router/TestRouterQuota.java     | 14 +++++++-------
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java   |  7 -------
 .../hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java |  3 +--
 .../org/apache/hadoop/hdfs/TestBlockStoragePolicy.java     |  6 +++---
 .../org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java |  3 +--
 7 files changed, 18 insertions(+), 26 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
index 2b5e4f1..b17864a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/StorageType.java
@@ -34,12 +34,13 @@ import org.apache.hadoop.util.StringUtils;
 @InterfaceAudience.Public
 @InterfaceStability.Unstable
 public enum StorageType {
+  // sorted by the speed of the storage types, from fast to slow
   RAM_DISK(true, true),
+  NVDIMM(false, true),
   SSD(false, false),
   DISK(false, false),
   ARCHIVE(false, false),
-  PROVIDED(false, false),
-  NVDIMM(false, true);
+  PROVIDED(false, false);
 
   private final boolean isTransient;
   private final boolean isRAM;
@@ -121,4 +122,4 @@ public enum StorageType {
                                StorageType t, String name) {
     return conf.get(CONF_KEY_HEADER + t.toString() + "." + name);
   }
-}
+}
\ No newline at end of file
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
index 19516b8..618cb01 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/shell/TestCount.java
@@ -283,10 +283,10 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------> <----13-----> <------17------->
+        " NVDIMM_QUOTA  REM_NVDIMM_QUOTA " +
         "    SSD_QUOTA     REM_SSD_QUOTA    DISK_QUOTA    REM_DISK_QUOTA " +
         // <----13---> <-------17------>
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
-        " NVDIMM_QUOTA  REM_NVDIMM_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);
@@ -338,11 +338,11 @@ public class TestCount {
     count.processOptions(options);
     String withStorageTypeHeader =
         // <----13---> <-------17------>
+        " NVDIMM_QUOTA  REM_NVDIMM_QUOTA " +
         "    SSD_QUOTA     REM_SSD_QUOTA " +
         "   DISK_QUOTA    REM_DISK_QUOTA " +
         "ARCHIVE_QUOTA REM_ARCHIVE_QUOTA " +
         "PROVIDED_QUOTA REM_PROVIDED_QUOTA " +
-        " NVDIMM_QUOTA  REM_NVDIMM_QUOTA " +
         "PATHNAME";
     verify(out).println(withStorageTypeHeader);
     verifyNoMoreInteractions(out);
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java
index b690041..551ae8a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java
@@ -414,13 +414,13 @@ public class TestRouterQuota {
     QuotaUsage usage = client.getQuotaUsage("/type0");
     assertEquals(HdfsConstants.QUOTA_RESET, usage.getQuota());
     assertEquals(HdfsConstants.QUOTA_RESET, usage.getSpaceQuota());
-    verifyTypeQuotaAndConsume(new long[] {-1, -1, ssQuota * 2, -1, -1, -1},
-        null, usage);
+    verifyTypeQuotaAndConsume(new long[] {-1, -1, -1, ssQuota * 2, -1, -1}, null,
+        usage);
     // Verify /type1 quota on NN1.
     usage = client.getQuotaUsage("/type1");
     assertEquals(HdfsConstants.QUOTA_RESET, usage.getQuota());
     assertEquals(HdfsConstants.QUOTA_RESET, usage.getSpaceQuota());
-    verifyTypeQuotaAndConsume(new long[] {-1, -1, ssQuota, -1, -1, -1}, null,
+    verifyTypeQuotaAndConsume(new long[] {-1, -1, -1, ssQuota, -1, -1}, null,
         usage);
 
     FileSystem routerFs = routerContext.getFileSystem();
@@ -431,15 +431,15 @@ public class TestRouterQuota {
     assertEquals(2, u1.getFileAndDirectoryCount());
     assertEquals(HdfsConstants.QUOTA_RESET, u1.getSpaceQuota());
     assertEquals(fileSize * 3, u1.getSpaceConsumed());
-    verifyTypeQuotaAndConsume(new long[] {-1, -1, ssQuota, -1, -1, -1},
-        new long[] {0, 0, fileSize * 3, 0, 0, 0}, u1);
+    verifyTypeQuotaAndConsume(new long[] {-1, -1, -1, ssQuota, -1, -1},
+        new long[] {0, 0, 0, fileSize * 3, 0, 0}, u1);
     // Verify /type0 storage type quota usage on Router.
     assertEquals(HdfsConstants.QUOTA_RESET, u0.getQuota());
     assertEquals(4, u0.getFileAndDirectoryCount());
     assertEquals(HdfsConstants.QUOTA_RESET, u0.getSpaceQuota());
     assertEquals(fileSize * 3 * 2, u0.getSpaceConsumed());
-    verifyTypeQuotaAndConsume(new long[] {-1, -1, ssQuota * 2, -1, -1, -1},
-        new long[] {0, 0, fileSize * 3 * 2, 0, 0, 0}, u0);
+    verifyTypeQuotaAndConsume(new long[] {-1, -1, -1, ssQuota * 2, -1, -1},
+        new long[] {0, 0, 0, fileSize * 3 * 2, 0, 0}, u0);
   }
 
   @Test
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e559515..926c4e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2429,10 +2429,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws  IOException
    */
   void setStoragePolicy(String src, String policyName) throws IOException {
-    if (policyName.equalsIgnoreCase(
-            HdfsConstants.ALLNVDIMM_STORAGE_POLICY_NAME)) {
-      requireEffectiveLayoutVersionForFeature(Feature.NVDIMM_SUPPORT);
-    }
     final String operationName = "setStoragePolicy";
     checkOperation(OperationCategory.WRITE);
     checkStoragePolicyEnabled("set storage policy", true);
@@ -3575,9 +3571,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (type != null) {
       requireEffectiveLayoutVersionForFeature(Feature.QUOTA_BY_STORAGE_TYPE);
     }
-    if (type == StorageType.NVDIMM) {
-      requireEffectiveLayoutVersionForFeature(Feature.NVDIMM_SUPPORT);
-    }
     checkOperation(OperationCategory.WRITE);
     final String operationName = getQuotaCommand(nsQuota, ssQuota);
     final FSPermissionChecker pc = getPermissionChecker();
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index b247746..297ca74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@ -89,8 +89,7 @@ public class NameNodeLayoutVersion {
     APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"),
     QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"),
     ERASURE_CODING(-64, -61, "Support erasure coding"),
-    EXPANDED_STRING_TABLE(-65, -61, "Support expanded string table in fsimage"),
-    NVDIMM_SUPPORT(-66, -61, "Support NVDIMM storage type");
+    EXPANDED_STRING_TABLE(-65, -61, "Support expanded string table in fsimage");
 
     private final FeatureInfo info;
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index ef11695..efb4b4d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -1421,29 +1421,29 @@ public class TestBlockStoragePolicy {
     final EnumMap<StorageType, Integer> map = new EnumMap<>(StorageType.class);
 
     //put storage type is reversed order
-    map.put(StorageType.NVDIMM, 1);
     map.put(StorageType.ARCHIVE, 1);
     map.put(StorageType.DISK, 1);
     map.put(StorageType.SSD, 1);
     map.put(StorageType.RAM_DISK, 1);
+    map.put(StorageType.NVDIMM, 1);
 
     {
       final Iterator<StorageType> i = map.keySet().iterator();
       Assert.assertEquals(StorageType.RAM_DISK, i.next());
+      Assert.assertEquals(StorageType.NVDIMM, i.next());
       Assert.assertEquals(StorageType.SSD, i.next());
       Assert.assertEquals(StorageType.DISK, i.next());
       Assert.assertEquals(StorageType.ARCHIVE, i.next());
-      Assert.assertEquals(StorageType.NVDIMM, i.next());
     }
 
     {
       final Iterator<Map.Entry<StorageType, Integer>> i
           = map.entrySet().iterator();
       Assert.assertEquals(StorageType.RAM_DISK, i.next().getKey());
+      Assert.assertEquals(StorageType.NVDIMM, i.next().getKey());
       Assert.assertEquals(StorageType.SSD, i.next().getKey());
       Assert.assertEquals(StorageType.DISK, i.next().getKey());
       Assert.assertEquals(StorageType.ARCHIVE, i.next().getKey());
-      Assert.assertEquals(StorageType.NVDIMM, i.next().getKey());
     }
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
index 3164809..2c9905d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java
@@ -128,8 +128,7 @@ public class TestLayoutVersion {
         NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK,
         NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE,
         NameNodeLayoutVersion.Feature.ERASURE_CODING,
-        NameNodeLayoutVersion.Feature.EXPANDED_STRING_TABLE,
-        NameNodeLayoutVersion.Feature.NVDIMM_SUPPORT);
+        NameNodeLayoutVersion.Feature.EXPANDED_STRING_TABLE);
     for (LayoutFeature f : compatibleFeatures) {
       assertEquals(String.format("Expected minimum compatible layout version " +
           "%d for feature %s.", baseLV, f), baseLV,

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