ソースを参照

HDFS-14726. Fix JN incompatibility issue in branch-2 due to backport of HDFS-10519. Contributed by Chen Liang.

Chen Liang 5 年 前
コミット
f05d87b32e

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -76,6 +76,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsServerProtos.StorageUuidsProto;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
@@ -292,8 +293,13 @@ public class PBHelper {
     for (RemoteEditLogProto l : manifest.getLogsList()) {
       logs.add(convert(l));
     }
-    return new RemoteEditLogManifest(logs,
-            manifest.getCommittedTxnId());
+    long committedId = HdfsServerConstants.INVALID_TXID;
+    if (manifest.hasCommittedTxnId()) {
+      // An older version JN may not have this field, in which case committedId
+      // is set to INVALID_TXID.
+      committedId = manifest.getCommittedTxnId();
+    }
+    return new RemoteEditLogManifest(logs, committedId);
   }
 
   public static CheckpointCommandProto convert(CheckpointCommand cmd) {

+ 15 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/qjournal/client/QuorumJournalManager.java

@@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.GetJourna
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.NewEpochResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.PrepareRecoveryResponseProto;
 import org.apache.hadoop.hdfs.qjournal.protocol.QJournalProtocolProtos.SegmentStateProto;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.Storage;
 import org.apache.hadoop.hdfs.server.common.StorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.EditLogFileInputStream;
@@ -632,13 +633,22 @@ public class QuorumJournalManager implements JournalManager {
 
         // If it's bounded by durable Txns, endTxId could not be larger
         // than committedTxnId. This ensures the consistency.
+        // An older version JN may not return the field committedTxnId,
+        // in which case it is set to INVALID_TXID.
         if (onlyDurableTxns && inProgressOk) {
-          endTxId = Math.min(endTxId, committedTxnId);
-          if (endTxId < remoteLog.getStartTxId()) {
-            LOG.warn("Found endTxId (" + endTxId + ") that is less than " +
-                "the startTxId (" + remoteLog.getStartTxId() +
-                ") - setting it to startTxId.");
+          if (committedTxnId == HdfsServerConstants.INVALID_TXID) {
+            LOG.warn("Received undefined committed txn id, "
+                + " NN and JN are on different version? "
+                + "- seting to startTxId");
             endTxId = remoteLog.getStartTxId();
+          } else {
+            endTxId = Math.min(endTxId, committedTxnId);
+            if (endTxId < remoteLog.getStartTxId()) {
+              LOG.warn("Found endTxId (" + endTxId + ") that is less than " +
+                  "the startTxId (" + remoteLog.getStartTxId() +
+                  ") - setting it to startTxId.");
+              endTxId = remoteLog.getStartTxId();
+            }
           }
         }
 

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto

@@ -92,7 +92,7 @@ message RemoteEditLogProto {
  */
 message RemoteEditLogManifestProto {
   repeated RemoteEditLogProto logs = 1;
-  required uint64 committedTxnId = 2;
+  optional uint64 committedTxnId = 2;
 }
 
 /**