瀏覽代碼

HDFS-5848. Add rolling upgrade status to heartbeat response.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-5535@1563384 13f79535-47bb-0310-9956-ffa450edef68
Tsz-wo Sze 11 年之前
父節點
當前提交
4f92eb2f61
共有 15 個文件被更改,包括 145 次插入36 次删除
  1. 2 0
      hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-5535.txt
  2. 12 8
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java
  3. 59 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java
  4. 1 3
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
  5. 3 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
  6. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
  7. 20 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
  8. 21 14
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
  9. 9 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java
  10. 3 2
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
  11. 1 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto
  12. 6 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
  13. 1 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java
  14. 2 1
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java
  15. 3 4
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES_HDFS-5535.txt

@@ -18,3 +18,5 @@ HDFS-5535 subtasks:
 
 
     HDFS-5754. Split LayoutVerion into NameNodeLayoutVersion and
     HDFS-5754. Split LayoutVerion into NameNodeLayoutVersion and
     DataNodeLayoutVersion. (Brandon Li via szetszwo)
     DataNodeLayoutVersion. (Brandon Li via szetszwo)
+
+    HDFS-5848. Add rolling upgrade status to heartbeat response. (szetszwo)

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeInfo.java

@@ -27,17 +27,19 @@ import org.apache.hadoop.classification.InterfaceStability;
  */
  */
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
-public class RollingUpgradeInfo {
-  public static final RollingUpgradeInfo EMPTY_INFO = new RollingUpgradeInfo(0); 
+public class RollingUpgradeInfo extends RollingUpgradeStatus {
+  public static final RollingUpgradeInfo EMPTY_INFO = new RollingUpgradeInfo(
+      null, 0); 
 
 
   private long startTime;
   private long startTime;
   private long finalizeTime;
   private long finalizeTime;
   
   
-  public RollingUpgradeInfo(long startTime) {
-    this(startTime, 0L);
+  public RollingUpgradeInfo(String blockPoolId, long startTime) {
+    this(blockPoolId, startTime, 0L);
   }
   }
 
 
-  public RollingUpgradeInfo(long startTime, long finalizeTime) {
+  public RollingUpgradeInfo(String blockPoolId, long startTime, long finalizeTime) {
+    super(blockPoolId);
     this.startTime = startTime;
     this.startTime = startTime;
     this.finalizeTime = finalizeTime;
     this.finalizeTime = finalizeTime;
   }
   }
@@ -62,7 +64,7 @@ public class RollingUpgradeInfo {
   @Override
   @Override
   public int hashCode() {
   public int hashCode() {
     //only use lower 32 bits
     //only use lower 32 bits
-    return (int)startTime ^ (int)finalizeTime;
+    return super.hashCode() ^ (int)startTime ^ (int)finalizeTime;
   }
   }
 
 
   @Override
   @Override
@@ -73,13 +75,15 @@ public class RollingUpgradeInfo {
       return false;
       return false;
     }
     }
     final RollingUpgradeInfo that = (RollingUpgradeInfo)obj;
     final RollingUpgradeInfo that = (RollingUpgradeInfo)obj;
-    return this.startTime == that.startTime
+    return super.equals(that)
+        && this.startTime == that.startTime
         && this.finalizeTime == that.finalizeTime;
         && this.finalizeTime == that.finalizeTime;
   }
   }
 
 
   @Override
   @Override
   public String toString() {
   public String toString() {
-    return "     Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
+    return super.toString()
+      +  "\n     Start Time: " + (startTime == 0? "<NOT STARTED>": timestamp2String(startTime))
       +  "\n  Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
       +  "\n  Finalize Time: " + (finalizeTime == 0? "<NOT FINALIZED>": timestamp2String(finalizeTime));
   }
   }
   
   

+ 59 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/RollingUpgradeStatus.java

@@ -0,0 +1,59 @@
+/**
+ * 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 org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+
+/**
+ * Rolling upgrade status
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class RollingUpgradeStatus {
+  private String blockPoolId;
+
+  public RollingUpgradeStatus(String blockPoolId) {
+    this.blockPoolId = blockPoolId;
+  }
+
+  public String getBlockPoolId() {
+    return blockPoolId;
+  }
+
+  @Override
+  public int hashCode() {
+    return blockPoolId.hashCode();
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    } else if (obj == null || !(obj instanceof RollingUpgradeStatus)) {
+      return false;
+    }
+    final RollingUpgradeStatus that = (RollingUpgradeStatus)obj;
+    return this.blockPoolId.equals(that.blockPoolId);
+  }
+
+  @Override
+  public String toString() {
+    return "  Block Pool ID: " + blockPoolId;
+  }
+}

+ 1 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java

@@ -120,7 +120,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Report
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RestoreFailedStorageRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollEditsResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.RollingUpgradeResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SaveNamespaceRequestProto;
@@ -646,8 +645,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setAction(PBHelper.convert(action)).build();
         .setAction(PBHelper.convert(action)).build();
     try {
     try {
       final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
       final RollingUpgradeResponseProto proto = rpcProxy.rollingUpgrade(null, r);
-      final RollingUpgradeInfoProto info = proto.getRollingUpgradeInfo();
-      return new RollingUpgradeInfo(info.getStartTime(), info.getFinalizeTime());
+      return PBHelper.convert(proto.getRollingUpgradeInfo());
     } catch (ServiceException e) {
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
       throw ProtobufHelper.getRemoteException(e);
     }
     }

+ 3 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java

@@ -183,7 +183,9 @@ public class DatanodeProtocolClientSideTranslatorPB implements
       cmds[index] = PBHelper.convert(p);
       cmds[index] = PBHelper.convert(p);
       index++;
       index++;
     }
     }
-    return new HeartbeatResponse(cmds, PBHelper.convert(resp.getHaStatus()));
+    return new HeartbeatResponse(cmds,
+        PBHelper.convert(resp.getHaStatus()),
+        PBHelper.convert(resp.getRollingUpgradeStatus()));
   }
   }
 
 
   @Override
   @Override

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java

@@ -121,6 +121,8 @@ public class DatanodeProtocolServerSideTranslatorPB implements
       }
       }
     }
     }
     builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
     builder.setHaStatus(PBHelper.convert(response.getNameNodeHaState()));
+    builder.setRollingUpgradeStatus(PBHelper.convertRollingUpgradeStatus(
+        response.getRollingUpdateStatus()));
     return builder.build();
     return builder.build();
   }
   }
 
 

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffReportEntry;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport.DiffType;
@@ -124,6 +125,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RecoveringBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogManifestProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RemoteEditLogProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ReplicaStateProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.RollingUpgradeStatusProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportEntryProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshotDiffReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.SnapshottableDirectoryListingProto;
@@ -1453,13 +1455,31 @@ public class PBHelper {
     }
     }
   }
   }
 
 
+  public static RollingUpgradeStatusProto convertRollingUpgradeStatus(
+      RollingUpgradeStatus status) {
+    return RollingUpgradeStatusProto.newBuilder()
+        .setBlockPoolId(status.getBlockPoolId())
+        .build();
+  }
+
+  public static RollingUpgradeStatus convert(RollingUpgradeStatusProto proto) {
+    return new RollingUpgradeStatus(proto.getBlockPoolId());
+  }
+
   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
   public static RollingUpgradeInfoProto convert(RollingUpgradeInfo info) {
     return RollingUpgradeInfoProto.newBuilder()
     return RollingUpgradeInfoProto.newBuilder()
+        .setStatus(convertRollingUpgradeStatus(info))
         .setStartTime(info.getStartTime())
         .setStartTime(info.getStartTime())
         .setFinalizeTime(info.getFinalizeTime())
         .setFinalizeTime(info.getFinalizeTime())
         .build();
         .build();
   }
   }
 
 
+  public static RollingUpgradeInfo convert(RollingUpgradeInfoProto proto) {
+    RollingUpgradeStatusProto status = proto.getStatus();
+    return new RollingUpgradeInfo(status.getBlockPoolId(),
+        proto.getStartTime(), proto.getFinalizeTime());
+  }
+
   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
   public static CorruptFileBlocks convert(CorruptFileBlocksProto c) {
     if (c == null)
     if (c == null)
       return null;
       return null;

+ 21 - 14
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -36,10 +36,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_STANDBY_CHECKPOINTS_KE
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOGGERS_KEY;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT;
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_ASYNC_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_DEFAULT;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_TOKEN_TRACKING_ID_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_DEFAULT_AUDIT_LOGGER_NAME;
@@ -4193,23 +4193,24 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
         throws IOException {
         throws IOException {
     readLock();
     readLock();
     try {
     try {
+      //get datanode commands
       final int maxTransfer = blockManager.getMaxReplicationStreams()
       final int maxTransfer = blockManager.getMaxReplicationStreams()
           - xmitsInProgress;
           - xmitsInProgress;
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
           nodeReg, reports, blockPoolId, cacheCapacity, cacheUsed,
           xceiverCount, maxTransfer, failedVolumes);
           xceiverCount, maxTransfer, failedVolumes);
-      return new HeartbeatResponse(cmds, createHaStatusHeartbeat());
+      
+      //create ha status
+      final NNHAStatusHeartbeat haState = new NNHAStatusHeartbeat(
+          haContext.getState().getServiceState(),
+          getFSImage().getLastAppliedOrWrittenTxId());
+
+      return new HeartbeatResponse(cmds, haState, rollingUpgradeInfo);
     } finally {
     } finally {
       readUnlock();
       readUnlock();
     }
     }
   }
   }
 
 
-  private NNHAStatusHeartbeat createHaStatusHeartbeat() {
-    HAState state = haContext.getState();
-    return new NNHAStatusHeartbeat(state.getServiceState(),
-        getFSImage().getLastAppliedOrWrittenTxId());
-  }
-
   /**
   /**
    * Returns whether or not there were available resources at the last check of
    * Returns whether or not there were available resources at the last check of
    * resources.
    * resources.
@@ -7102,12 +7103,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       final String err = "Failed to start rolling upgrade";
       final String err = "Failed to start rolling upgrade";
       checkNameNodeSafeMode(err);
       checkNameNodeSafeMode(err);
 
 
-      if (rollingUpgradeInfo != null) {
+      if (isRollingUpgrade()) {
         throw new RollingUpgradeException(err
         throw new RollingUpgradeException(err
             + " since a rolling upgrade is already in progress."
             + " since a rolling upgrade is already in progress."
             + "\nExisting rolling upgrade info: " + rollingUpgradeInfo);
             + "\nExisting rolling upgrade info: " + rollingUpgradeInfo);
       }
       }
-
+      
       final CheckpointSignature cs = getFSImage().rollEditLog();
       final CheckpointSignature cs = getFSImage().rollEditLog();
       LOG.info("Successfully rolled edit log for preparing rolling upgrade."
       LOG.info("Successfully rolled edit log for preparing rolling upgrade."
           + " Checkpoint signature: " + cs);
           + " Checkpoint signature: " + cs);
@@ -7125,7 +7126,12 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
   }
   }
 
 
   void setRollingUpgradeInfo(long startTime) {
   void setRollingUpgradeInfo(long startTime) {
-    rollingUpgradeInfo = new RollingUpgradeInfo(startTime);;
+    rollingUpgradeInfo = new RollingUpgradeInfo(blockPoolId, startTime);
+  }
+
+  /** Is rolling upgrade in progress? */
+  boolean isRollingUpgrade() {
+    return rollingUpgradeInfo != null;
   }
   }
 
 
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
   RollingUpgradeInfo finalizeRollingUpgrade() throws IOException {
@@ -7138,12 +7144,13 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       final String err = "Failed to finalize rolling upgrade";
       final String err = "Failed to finalize rolling upgrade";
       checkNameNodeSafeMode(err);
       checkNameNodeSafeMode(err);
 
 
-      if (rollingUpgradeInfo == null) {
+      if (!isRollingUpgrade()) {
         throw new RollingUpgradeException(err
         throw new RollingUpgradeException(err
             + " since there is no rolling upgrade in progress.");
             + " since there is no rolling upgrade in progress.");
       }
       }
 
 
-      returnInfo = new RollingUpgradeInfo(rollingUpgradeInfo.getStartTime(), now());
+      returnInfo = new RollingUpgradeInfo(blockPoolId,
+          rollingUpgradeInfo.getStartTime(), now());
       getFSImage().saveNamespace(this);
       getFSImage().saveNamespace(this);
       rollingUpgradeInfo = null;
       rollingUpgradeInfo = null;
     } finally {
     } finally {

+ 9 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/HeartbeatResponse.java

@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.protocol;
 
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 
 
 @InterfaceAudience.Private
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
 @InterfaceStability.Evolving
@@ -31,11 +32,14 @@ public class HeartbeatResponse {
   
   
   /** Information about the current HA-related state of the NN */
   /** Information about the current HA-related state of the NN */
   private NNHAStatusHeartbeat haStatus;
   private NNHAStatusHeartbeat haStatus;
+
+  private RollingUpgradeStatus rollingUpdateStatus;
   
   
   public HeartbeatResponse(DatanodeCommand[] cmds,
   public HeartbeatResponse(DatanodeCommand[] cmds,
-      NNHAStatusHeartbeat haStatus) {
+      NNHAStatusHeartbeat haStatus, RollingUpgradeStatus rollingUpdateStatus) {
     commands = cmds;
     commands = cmds;
     this.haStatus = haStatus;
     this.haStatus = haStatus;
+    this.rollingUpdateStatus = rollingUpdateStatus;
   }
   }
   
   
   public DatanodeCommand[] getCommands() {
   public DatanodeCommand[] getCommands() {
@@ -45,4 +49,8 @@ public class HeartbeatResponse {
   public NNHAStatusHeartbeat getNameNodeHaState() {
   public NNHAStatusHeartbeat getNameNodeHaState() {
     return haStatus;
     return haStatus;
   }
   }
+
+  public RollingUpgradeStatus getRollingUpdateStatus() {
+    return rollingUpdateStatus;
+  }
 }
 }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto

@@ -343,8 +343,9 @@ message RollingUpgradeRequestProto {
 }
 }
 
 
 message RollingUpgradeInfoProto {
 message RollingUpgradeInfoProto {
-  required uint64 startTime = 1;
-  required uint64 finalizeTime = 2;
+  required RollingUpgradeStatusProto status = 1;
+  required uint64 startTime = 2;
+  required uint64 finalizeTime = 3;
 }
 }
 
 
 message RollingUpgradeResponseProto {
 message RollingUpgradeResponseProto {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto

@@ -221,6 +221,7 @@ message NNHAStatusHeartbeatProto {
 message HeartbeatResponseProto {
 message HeartbeatResponseProto {
   repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
   repeated DatanodeCommandProto cmds = 1; // Returned commands can be null
   required NNHAStatusHeartbeatProto haStatus = 2;
   required NNHAStatusHeartbeatProto haStatus = 2;
+  optional RollingUpgradeStatusProto rollingUpgradeStatus = 3;
 }
 }
 
 
 /**
 /**

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

@@ -459,4 +459,9 @@ message SnapshotInfoProto {
   // TODO: do we need access time?
   // TODO: do we need access time?
 }
 }
 
 
-
+/**
+ * Rolling upgrade status
+ */
+message RollingUpgradeStatusProto {
+  required string blockPoolId = 1;
+}

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBPOfferService.java

@@ -153,7 +153,7 @@ public class TestBPOfferService {
     public HeartbeatResponse answer(InvocationOnMock invocation) throws Throwable {
     public HeartbeatResponse answer(InvocationOnMock invocation) throws Throwable {
       heartbeatCounts[nnIdx]++;
       heartbeatCounts[nnIdx]++;
       return new HeartbeatResponse(new DatanodeCommand[0],
       return new HeartbeatResponse(new DatanodeCommand[0],
-          mockHaStatuses[nnIdx]);
+          mockHaStatuses[nnIdx], null);
     }
     }
   }
   }
 
 

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestBlockRecovery.java

@@ -163,7 +163,8 @@ public class TestBlockRecovery {
             Mockito.anyInt()))
             Mockito.anyInt()))
         .thenReturn(new HeartbeatResponse(
         .thenReturn(new HeartbeatResponse(
             new DatanodeCommand[0],
             new DatanodeCommand[0],
-            new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1)));
+            new NNHAStatusHeartbeat(HAServiceState.ACTIVE, 1),
+            null));
 
 
     dn = new DataNode(conf, locations, null) {
     dn = new DataNode(conf, locations, null) {
       @Override
       @Override

+ 3 - 4
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestFsDatasetCache.java

@@ -151,10 +151,9 @@ public class TestFsDatasetCache {
 
 
   private static void setHeartbeatResponse(DatanodeCommand[] cmds)
   private static void setHeartbeatResponse(DatanodeCommand[] cmds)
       throws IOException {
       throws IOException {
-    HeartbeatResponse response = new HeartbeatResponse(
-        cmds,
-        new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
-        fsImage.getLastAppliedOrWrittenTxId()));
+    NNHAStatusHeartbeat ha = new NNHAStatusHeartbeat(HAServiceState.ACTIVE,
+        fsImage.getLastAppliedOrWrittenTxId());
+    HeartbeatResponse response = new HeartbeatResponse(cmds, ha, null);
     doReturn(response).when(spyNN).sendHeartbeat(
     doReturn(response).when(spyNN).sendHeartbeat(
         (DatanodeRegistration) any(),
         (DatanodeRegistration) any(),
         (StorageReport[]) any(), anyLong(), anyLong(),
         (StorageReport[]) any(), anyLong(), anyLong(),