浏览代码

HDFS-5784. reserve space in the edit log header and fsImage header for feature flag section (cmccabe)

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1559295 13f79535-47bb-0310-9956-ffa450edef68
Colin McCabe 11 年之前
父节点
当前提交
c5083c041e

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -202,6 +202,9 @@ Release 2.4.0 - UNRELEASED
 
     HDFS-5704. Change OP_UPDATE_BLOCKS with a new OP_ADD_BLOCK. (jing9)
 
+    HDFS-5784. Reserve space in edit log header and fsimage header for feature
+    flag section. (Colin Patrick McCabe)
+
   OPTIMIZATIONS
 
     HDFS-5239.  Allow FSNamesystem lock fairness to be configurable (daryn)

+ 64 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutFlags.java

@@ -0,0 +1,64 @@
+/**
+ * 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.protocol;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
+/**
+ * LayoutFlags represent features which the FSImage and edit logs can either
+ * support or not, independently of layout version.
+ * 
+ * Note: all flags starting with 'test' are reserved for unit test purposes.
+ */
+@InterfaceAudience.Private
+public class LayoutFlags {
+  /**
+   * Load a LayoutFlags object from a stream.
+   *
+   * @param in            The stream to read from.
+   * @throws IOException
+   */
+  public static LayoutFlags read(DataInputStream in)
+      throws IOException {
+    int length = in.readInt();
+    if (length < 0) {
+      throw new IOException("The length of the feature flag section " +
+          "was negative at " + length + " bytes.");
+    } else if (length > 0) {
+      throw new IOException("Found feature flags which we can't handle. " +
+          "Please upgrade your software.");
+    }
+    return new LayoutFlags();
+  }
+
+  private LayoutFlags() {
+  }
+
+  public static void write(DataOutputStream out) throws IOException {
+    out.writeInt(0);
+  }
+}

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LayoutVersion.java

@@ -110,7 +110,8 @@ public class LayoutVersion {
     EDITLOG_ADD_BLOCK(-48, "Add new editlog that only records allocation of "
         + "the new block instead of the entire block list"),
     ADD_DATANODE_AND_STORAGE_UUIDS(-49, "Replace StorageID with DatanodeUuid."
-        + " Use distinct StorageUuid per storage directory.");
+        + " Use distinct StorageUuid per storage directory."),
+    ADD_LAYOUT_FLAGS(-50, "Add support for layout flags.");
 
     final int lv;
     final int ancestorLV;

+ 11 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileInputStream.java

@@ -34,6 +34,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.namenode.TransferFsImage.HttpGetFailedException;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
@@ -147,6 +150,14 @@ public class EditLogFileInputStream extends EditLogInputStream {
       } catch (EOFException eofe) {
         throw new LogHeaderCorruptException("No header found in log");
       }
+      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, logVersion)) {
+        try {
+          LayoutFlags.read(dataIn);
+        } catch (EOFException eofe) {
+          throw new LogHeaderCorruptException("EOF while reading layout " +
+              "flags from log");
+        }
+      }
       reader = new FSEditLogOp.Reader(dataIn, tracker, logVersion);
       reader.setMaxOpSize(maxOpSize);
       state = State.OPEN;

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EditLogFileOutputStream.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.io.IOUtils;
 
 import com.google.common.annotations.VisibleForTesting;
@@ -132,6 +133,7 @@ public class EditLogFileOutputStream extends EditLogOutputStream {
   @VisibleForTesting
   public static void writeHeader(DataOutputStream out) throws IOException {
     out.writeInt(HdfsConstants.LAYOUT_VERSION);
+    LayoutFlags.write(out);
   }
 
   @Override

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java

@@ -49,6 +49,7 @@ import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -264,6 +265,9 @@ public class FSImageFormat {
         }
         boolean supportSnapshot = LayoutVersion.supports(Feature.SNAPSHOT,
             imgVersion);
+        if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imgVersion)) {
+          LayoutFlags.read(in);
+        }
 
         // read namespaceID: first appeared in version -2
         in.readInt();
@@ -1001,6 +1005,7 @@ public class FSImageFormat {
       DataOutputStream out = new DataOutputStream(fos);
       try {
         out.writeInt(HdfsConstants.LAYOUT_VERSION);
+        LayoutFlags.write(out);
         // We use the non-locked version of getNamespaceInfo here since
         // the coordinating thread of saveNamespace already has read-locked
         // the namespace for us. If we attempt to take another readlock

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/ImageLoaderCurrent.java

@@ -28,6 +28,7 @@ import java.util.Map;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
+import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
@@ -126,7 +127,7 @@ class ImageLoaderCurrent implements ImageLoader {
                                       new SimpleDateFormat("yyyy-MM-dd HH:mm");
   private static int[] versions = { -16, -17, -18, -19, -20, -21, -22, -23,
       -24, -25, -26, -27, -28, -30, -31, -32, -33, -34, -35, -36, -37, -38, -39,
-      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49 };
+      -40, -41, -42, -43, -44, -45, -46, -47, -48, -49, -50 };
   private int imageVersion = 0;
   
   private final Map<Long, Boolean> subtreeMap = new HashMap<Long, Boolean>();
@@ -157,6 +158,9 @@ class ImageLoaderCurrent implements ImageLoader {
       imageVersion = in.readInt();
       if( !canLoadVersion(imageVersion))
         throw new IOException("Cannot process fslayout version " + imageVersion);
+      if (LayoutVersion.supports(Feature.ADD_LAYOUT_FLAGS, imageVersion)) {
+        LayoutFlags.read(in);
+      }
 
       v.visit(ImageElement.IMAGE_VERSION, imageVersion);
       v.visit(ImageElement.NAMESPACE_ID, in.readInt());

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/qjournal/server/TestJournalNode.java

@@ -191,6 +191,7 @@ public class TestJournalNode {
         "/getJournal?segmentTxId=1&jid=" + journalId));
     byte[] expected = Bytes.concat(
             Ints.toByteArray(HdfsConstants.LAYOUT_VERSION),
+            (new byte[] { 0, 0, 0, 0 }), // layout flags section
             EDITS_DATA);
 
     assertArrayEquals(expected, retrievedViaHttp);

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java

@@ -377,8 +377,9 @@ public class TestFSEditLogLoader {
     File testDir = new File(TEST_DIR, "testValidateEmptyEditLog");
     SortedMap<Long, Long> offsetToTxId = Maps.newTreeMap();
     File logFile = prepareUnfinalizedTestEditLog(testDir, 0, offsetToTxId);
-    // Truncate the file so that there is nothing except the header
-    truncateFile(logFile, 4);
+    // Truncate the file so that there is nothing except the header and
+    // layout flags section.
+    truncateFile(logFile, 8);
     EditLogValidation validation =
         EditLogFileInputStream.validateEditLog(logFile);
     assertTrue(!validation.hasCorruptHeader());

二进制
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored


+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/resources/editsStored.xml

@@ -1,6 +1,6 @@
 <?xml version="1.0" encoding="UTF-8"?>
 <EDITS>
-  <EDITS_VERSION>-49</EDITS_VERSION>
+  <EDITS_VERSION>-50</EDITS_VERSION>
   <RECORD>
     <OPCODE>OP_START_LOG_SEGMENT</OPCODE>
     <DATA>