Browse Source

HDFS-13068. RBF: Add router admin option to manage safe mode. Contributed by Yiqun Lin.

Yiqun Lin 7 years ago
parent
commit
b0627c891b
23 changed files with 1036 additions and 4 deletions
  1. 60 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java
  2. 59 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java
  3. 45 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java
  4. 4 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java
  5. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateManager.java
  6. 32 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/EnterSafeModeRequest.java
  7. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/EnterSafeModeResponse.java
  8. 31 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetSafeModeRequest.java
  9. 49 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetSafeModeResponse.java
  10. 32 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/LeaveSafeModeRequest.java
  11. 50 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/LeaveSafeModeResponse.java
  12. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/EnterSafeModeRequestPBImpl.java
  13. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/EnterSafeModeResponsePBImpl.java
  14. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetSafeModeRequestPBImpl.java
  15. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetSafeModeResponsePBImpl.java
  16. 62 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/LeaveSafeModeRequestPBImpl.java
  17. 73 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/LeaveSafeModeResponsePBImpl.java
  18. 74 1
      hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java
  19. 25 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto
  20. 15 0
      hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto
  21. 2 0
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md
  22. 5 1
      hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md
  23. 48 0
      hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolServerSideTranslatorPB.java

@@ -23,8 +23,14 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
@@ -32,16 +38,28 @@ import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProt
 import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
 import org.apache.hadoop.hdfs.server.federation.router.RouterAdminServer;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.EnterSafeModeRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.EnterSafeModeResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetSafeModeRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetSafeModeResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.LeaveSafeModeRequestPBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.LeaveSafeModeResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
@@ -148,4 +166,46 @@ public class RouterAdminProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
       throw new ServiceException(e);
     }
     }
   }
   }
+
+  @Override
+  public EnterSafeModeResponseProto enterSafeMode(RpcController controller,
+      EnterSafeModeRequestProto request) throws ServiceException {
+    try {
+      EnterSafeModeRequest req = new EnterSafeModeRequestPBImpl(request);
+      EnterSafeModeResponse response = server.enterSafeMode(req);
+      EnterSafeModeResponsePBImpl responsePB =
+          (EnterSafeModeResponsePBImpl) response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public LeaveSafeModeResponseProto leaveSafeMode(RpcController controller,
+      LeaveSafeModeRequestProto request) throws ServiceException {
+    try {
+      LeaveSafeModeRequest req = new LeaveSafeModeRequestPBImpl(request);
+      LeaveSafeModeResponse response = server.leaveSafeMode(req);
+      LeaveSafeModeResponsePBImpl responsePB =
+          (LeaveSafeModeResponsePBImpl) response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
+  public GetSafeModeResponseProto getSafeMode(RpcController controller,
+      GetSafeModeRequestProto request) throws ServiceException {
+    try {
+      GetSafeModeRequest req = new GetSafeModeRequestPBImpl(request);
+      GetSafeModeResponse response = server.getSafeMode(req);
+      GetSafeModeResponsePBImpl responsePB =
+          (GetSafeModeResponsePBImpl) response;
+      return responsePB.getProto();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }
 }

+ 59 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/RouterAdminProtocolTranslatorPB.java

@@ -24,25 +24,41 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.AddMountTableEntryResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetMountTableEntriesResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.RemoveMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryRequestProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.UpdateMountTableEntryResponseProto;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
+import org.apache.hadoop.hdfs.server.federation.router.RouterStateManager;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.AddMountTableEntryResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.EnterSafeModeResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetMountTableEntriesResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.GetSafeModeResponsePBImpl;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.LeaveSafeModeResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.RemoveMountTableEntryResponsePBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.impl.pb.UpdateMountTableEntryRequestPBImpl;
@@ -64,7 +80,7 @@ import com.google.protobuf.ServiceException;
 @InterfaceStability.Stable
 @InterfaceStability.Stable
 public class RouterAdminProtocolTranslatorPB
 public class RouterAdminProtocolTranslatorPB
     implements ProtocolMetaInterface, MountTableManager,
     implements ProtocolMetaInterface, MountTableManager,
-    Closeable, ProtocolTranslator {
+    Closeable, ProtocolTranslator, RouterStateManager {
   final private RouterAdminProtocolPB rpcProxy;
   final private RouterAdminProtocolPB rpcProxy;
 
 
   public RouterAdminProtocolTranslatorPB(RouterAdminProtocolPB proxy) {
   public RouterAdminProtocolTranslatorPB(RouterAdminProtocolPB proxy) {
@@ -147,4 +163,46 @@ public class RouterAdminProtocolTranslatorPB
       throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
       throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
     }
     }
   }
   }
+
+  @Override
+  public EnterSafeModeResponse enterSafeMode(EnterSafeModeRequest request)
+      throws IOException {
+    EnterSafeModeRequestProto proto =
+        EnterSafeModeRequestProto.newBuilder().build();
+    try {
+      EnterSafeModeResponseProto response =
+          rpcProxy.enterSafeMode(null, proto);
+      return new EnterSafeModeResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public LeaveSafeModeResponse leaveSafeMode(LeaveSafeModeRequest request)
+      throws IOException {
+    LeaveSafeModeRequestProto proto =
+        LeaveSafeModeRequestProto.newBuilder().build();
+    try {
+      LeaveSafeModeResponseProto response =
+          rpcProxy.leaveSafeMode(null, proto);
+      return new LeaveSafeModeResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
+
+  @Override
+  public GetSafeModeResponse getSafeMode(GetSafeModeRequest request)
+      throws IOException {
+    GetSafeModeRequestProto proto =
+        GetSafeModeRequestProto.newBuilder().build();
+    try {
+      GetSafeModeResponseProto response =
+          rpcProxy.getSafeMode(null, proto);
+      return new GetSafeModeResponsePBImpl(response);
+    } catch (ServiceException e) {
+      throw new IOException(ProtobufHelper.getRemoteException(e).getMessage());
+    }
+  }
 }
 }

+ 45 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterAdminServer.java

@@ -32,8 +32,14 @@ import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.MountTableStore;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
@@ -56,7 +62,7 @@ import com.google.protobuf.BlockingService;
  * router. It is created, started, and stopped by {@link Router}.
  * router. It is created, started, and stopped by {@link Router}.
  */
  */
 public class RouterAdminServer extends AbstractService
 public class RouterAdminServer extends AbstractService
-    implements MountTableManager {
+    implements MountTableManager, RouterStateManager {
 
 
   private static final Logger LOG =
   private static final Logger LOG =
       LoggerFactory.getLogger(RouterAdminServer.class);
       LoggerFactory.getLogger(RouterAdminServer.class);
@@ -212,6 +218,44 @@ public class RouterAdminServer extends AbstractService
     return getMountTableStore().getMountTableEntries(request);
     return getMountTableStore().getMountTableEntries(request);
   }
   }
 
 
+  @Override
+  public EnterSafeModeResponse enterSafeMode(EnterSafeModeRequest request)
+      throws IOException {
+    this.router.updateRouterState(RouterServiceState.SAFEMODE);
+    this.router.getRpcServer().setSafeMode(true);
+    return EnterSafeModeResponse.newInstance(verifySafeMode(true));
+  }
+
+  @Override
+  public LeaveSafeModeResponse leaveSafeMode(LeaveSafeModeRequest request)
+      throws IOException {
+    this.router.updateRouterState(RouterServiceState.RUNNING);
+    this.router.getRpcServer().setSafeMode(false);
+    return LeaveSafeModeResponse.newInstance(verifySafeMode(false));
+  }
+
+  @Override
+  public GetSafeModeResponse getSafeMode(GetSafeModeRequest request)
+      throws IOException {
+    boolean isInSafeMode = this.router.getRpcServer().isInSafeMode();
+    return GetSafeModeResponse.newInstance(isInSafeMode);
+  }
+
+  /**
+   * Verify if Router set safe mode state correctly.
+   * @param isInSafeMode Expected state to be set.
+   * @return
+   */
+  private boolean verifySafeMode(boolean isInSafeMode) {
+    boolean serverInSafeMode = this.router.getRpcServer().isInSafeMode();
+    RouterServiceState currentState = this.router.getRouterState();
+
+    return (isInSafeMode && currentState == RouterServiceState.SAFEMODE
+        && serverInSafeMode)
+        || (!isInSafeMode && currentState != RouterServiceState.SAFEMODE
+            && !serverInSafeMode);
+  }
+
   /**
   /**
    * Get a new permission checker used for making mount table access
    * Get a new permission checker used for making mount table access
    * control. This method will be invoked during each RPC call in router
    * control. This method will be invoked during each RPC call in router

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClient.java

@@ -69,6 +69,10 @@ public class RouterClient implements Closeable {
     return proxy;
     return proxy;
   }
   }
 
 
+  public RouterStateManager getRouterStateManager() {
+    return proxy;
+  }
+
   @Override
   @Override
   public synchronized void close() throws IOException {
   public synchronized void close() throws IOException {
     RPC.stopProxy(proxy);
     RPC.stopProxy(proxy);

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterStateManager.java

@@ -0,0 +1,50 @@
+/**
+ * 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.server.federation.router;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
+
+/**
+ * Interface of managing the Router state.
+ */
+public interface RouterStateManager {
+  /**
+   * Enter safe mode and change Router state to RouterServiceState#SAFEMODE.
+   */
+  EnterSafeModeResponse enterSafeMode(EnterSafeModeRequest request)
+      throws IOException;
+
+  /**
+   * Leave safe mode and change Router state to RouterServiceState#RUNNING.
+   */
+  LeaveSafeModeResponse leaveSafeMode(LeaveSafeModeRequest request)
+      throws IOException;
+
+  /**
+   * Verify if current Router state is safe mode.
+   */
+  GetSafeModeResponse getSafeMode(GetSafeModeRequest request)
+      throws IOException;
+}

+ 32 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/EnterSafeModeRequest.java

@@ -0,0 +1,32 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for the Router entering safe mode state and updating
+ * its state in State Store.
+ */
+public abstract class EnterSafeModeRequest {
+  public static EnterSafeModeRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(EnterSafeModeRequest.class);
+  }
+}

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/EnterSafeModeResponse.java

@@ -0,0 +1,50 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for the Router entering safe mode state and updating
+ * its state in State Store.
+ */
+public abstract class EnterSafeModeResponse {
+
+  public static EnterSafeModeResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(EnterSafeModeResponse.class);
+  }
+
+  public static EnterSafeModeResponse newInstance(boolean status)
+      throws IOException {
+    EnterSafeModeResponse response = newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}

+ 31 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetSafeModeRequest.java

@@ -0,0 +1,31 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for verifying if current Router state is safe mode.
+ */
+public abstract class GetSafeModeRequest {
+  public static GetSafeModeRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetSafeModeRequest.class);
+  }
+}

+ 49 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/GetSafeModeResponse.java

@@ -0,0 +1,49 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for verifying if current Router state is safe mode.
+ */
+public abstract class GetSafeModeResponse {
+
+  public static GetSafeModeResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(GetSafeModeResponse.class);
+  }
+
+  public static GetSafeModeResponse newInstance(boolean isInSafeMode)
+      throws IOException {
+    GetSafeModeResponse response = newInstance();
+    response.setSafeMode(isInSafeMode);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean isInSafeMode();
+
+  @Public
+  @Unstable
+  public abstract void setSafeMode(boolean isInSafeMode);
+}

+ 32 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/LeaveSafeModeRequest.java

@@ -0,0 +1,32 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API request for the Router leaving safe mode state and updating
+ * its state in State Store.
+ */
+public abstract class LeaveSafeModeRequest {
+  public static LeaveSafeModeRequest newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(LeaveSafeModeRequest.class);
+  }
+}

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/LeaveSafeModeResponse.java

@@ -0,0 +1,50 @@
+/**
+ * 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.server.federation.store.protocol;
+
+import java.io.IOException;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.hdfs.server.federation.store.driver.StateStoreSerializer;
+
+/**
+ * API response for the Router leaving safe mode state and updating
+ * its state in State Store.
+ */
+public abstract class LeaveSafeModeResponse {
+
+  public static LeaveSafeModeResponse newInstance() throws IOException {
+    return StateStoreSerializer.newRecord(LeaveSafeModeResponse.class);
+  }
+
+  public static LeaveSafeModeResponse newInstance(boolean status)
+      throws IOException {
+    LeaveSafeModeResponse response = newInstance();
+    response.setStatus(status);
+    return response;
+  }
+
+  @Public
+  @Unstable
+  public abstract boolean getStatus();
+
+  @Public
+  @Unstable
+  public abstract void setStatus(boolean result);
+}

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/EnterSafeModeRequestPBImpl.java

@@ -0,0 +1,62 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetRouterRegistrationResponseProto.Builder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * EnterSafeModeRequest.
+ */
+public class EnterSafeModeRequestPBImpl extends EnterSafeModeRequest
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<EnterSafeModeRequestProto,
+      Builder, EnterSafeModeRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(EnterSafeModeRequestProto.class);
+
+  public EnterSafeModeRequestPBImpl() {
+  }
+
+  public EnterSafeModeRequestPBImpl(EnterSafeModeRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public EnterSafeModeRequestProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/EnterSafeModeResponsePBImpl.java

@@ -0,0 +1,73 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.EnterSafeModeResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * EnterSafeModeResponse.
+ */
+public class EnterSafeModeResponsePBImpl extends EnterSafeModeResponse
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<EnterSafeModeResponseProto,
+      Builder, EnterSafeModeResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(
+              EnterSafeModeResponseProto.class);
+
+  public EnterSafeModeResponsePBImpl() {
+  }
+
+  public EnterSafeModeResponsePBImpl(EnterSafeModeResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public EnterSafeModeResponseProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetSafeModeRequestPBImpl.java

@@ -0,0 +1,62 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetSafeModeRequest.
+ */
+public class GetSafeModeRequestPBImpl extends GetSafeModeRequest
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetSafeModeRequestProto,
+      Builder, GetSafeModeRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(GetSafeModeRequestProto.class);
+
+  public GetSafeModeRequestPBImpl() {
+  }
+
+  public GetSafeModeRequestPBImpl(GetSafeModeRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetSafeModeRequestProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/GetSafeModeResponsePBImpl.java

@@ -0,0 +1,73 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.GetSafeModeResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * GetSafeModeResponse.
+ */
+public class GetSafeModeResponsePBImpl extends GetSafeModeResponse
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<GetSafeModeResponseProto,
+      Builder, GetSafeModeResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(
+              GetSafeModeResponseProto.class);
+
+  public GetSafeModeResponsePBImpl() {
+  }
+
+  public GetSafeModeResponsePBImpl(GetSafeModeResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public GetSafeModeResponseProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean isInSafeMode() {
+    return this.translator.getProtoOrBuilder().getIsInSafeMode();
+  }
+
+  @Override
+  public void setSafeMode(boolean isInSafeMode) {
+    this.translator.getBuilder().setIsInSafeMode(isInSafeMode);
+  }
+}

+ 62 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/LeaveSafeModeRequestPBImpl.java

@@ -0,0 +1,62 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeRequestProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeRequestProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeRequestProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * LeaveSafeModeRequest.
+ */
+public class LeaveSafeModeRequestPBImpl extends LeaveSafeModeRequest
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<LeaveSafeModeRequestProto,
+      Builder, LeaveSafeModeRequestProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(LeaveSafeModeRequestProto.class);
+
+  public LeaveSafeModeRequestPBImpl() {
+  }
+
+  public LeaveSafeModeRequestPBImpl(LeaveSafeModeRequestProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public LeaveSafeModeRequestProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+}

+ 73 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/federation/store/protocol/impl/pb/LeaveSafeModeResponsePBImpl.java

@@ -0,0 +1,73 @@
+/**
+ * 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.server.federation.store.protocol.impl.pb;
+
+import java.io.IOException;
+
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeResponseProto;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeResponseProto.Builder;
+import org.apache.hadoop.hdfs.federation.protocol.proto.HdfsServerFederationProtos.LeaveSafeModeResponseProtoOrBuilder;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.records.impl.pb.PBRecord;
+
+import com.google.protobuf.Message;
+
+/**
+ * Protobuf implementation of the state store API object
+ * LeaveSafeModeResponse.
+ */
+public class LeaveSafeModeResponsePBImpl extends LeaveSafeModeResponse
+    implements PBRecord {
+
+  private FederationProtocolPBTranslator<LeaveSafeModeResponseProto,
+      Builder, LeaveSafeModeResponseProtoOrBuilder> translator =
+          new FederationProtocolPBTranslator<>(
+              LeaveSafeModeResponseProto.class);
+
+  public LeaveSafeModeResponsePBImpl() {
+  }
+
+  public LeaveSafeModeResponsePBImpl(LeaveSafeModeResponseProto proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public LeaveSafeModeResponseProto getProto() {
+    return translator.build();
+  }
+
+  @Override
+  public void setProto(Message proto) {
+    this.translator.setProto(proto);
+  }
+
+  @Override
+  public void readInstance(String base64String) throws IOException {
+    this.translator.readInstance(base64String);
+  }
+
+  @Override
+  public boolean getStatus() {
+    return this.translator.getProtoOrBuilder().getStatus();
+  }
+
+  @Override
+  public void setStatus(boolean result) {
+    this.translator.getBuilder().setStatus(result);
+  }
+}

+ 74 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/federation/RouterAdmin.java

@@ -35,10 +35,17 @@ import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.resolver.order.DestinationOrder;
 import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
 import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
 import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
 import org.apache.hadoop.hdfs.server.federation.router.RouterQuotaUsage;
+import org.apache.hadoop.hdfs.server.federation.router.RouterStateManager;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.EnterSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.GetSafeModeResponse;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeRequest;
+import org.apache.hadoop.hdfs.server.federation.store.protocol.LeaveSafeModeResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
@@ -86,7 +93,9 @@ public class RouterAdmin extends Configured implements Tool {
         + "\t[-ls <path>]\n"
         + "\t[-ls <path>]\n"
         + "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
         + "\t[-setQuota <path> -nsQuota <nsQuota> -ssQuota "
         + "<quota in bytes or quota size string>]\n"
         + "<quota in bytes or quota size string>]\n"
-        + "\t[-clrQuota <path>\n";
+        + "\t[-clrQuota <path>]\n"
+        + "\t[-safemode enter | leave | get]\n";
+
     System.out.println(usage);
     System.out.println(usage);
   }
   }
 
 
@@ -127,6 +136,12 @@ public class RouterAdmin extends Configured implements Tool {
         printUsage();
         printUsage();
         return exitCode;
         return exitCode;
       }
       }
+    } else if ("-safemode".equalsIgnoreCase(cmd)) {
+      if (argv.length < 2) {
+        System.err.println("Not enough parameters specificed for cmd " + cmd);
+        printUsage();
+        return exitCode;
+      }
     }
     }
 
 
     // Initialize RouterClient
     // Initialize RouterClient
@@ -172,6 +187,8 @@ public class RouterAdmin extends Configured implements Tool {
           System.out.println(
           System.out.println(
               "Successfully clear quota for mount point " + argv[i]);
               "Successfully clear quota for mount point " + argv[i]);
         }
         }
+      } else if ("-safemode".equals(cmd)) {
+        manageSafeMode(argv[i]);
       } else {
       } else {
         printUsage();
         printUsage();
         return exitCode;
         return exitCode;
@@ -524,6 +541,62 @@ public class RouterAdmin extends Configured implements Tool {
     return updateResponse.getStatus();
     return updateResponse.getStatus();
   }
   }
 
 
+  /**
+   * Manager the safe mode state.
+   * @param cmd Input command, enter or leave safe mode.
+   * @throws IOException
+   */
+  private void manageSafeMode(String cmd) throws IOException {
+    if (cmd.equals("enter")) {
+      if (enterSafeMode()) {
+        System.out.println("Successfully enter safe mode.");
+      }
+    } else if (cmd.equals("leave")) {
+      if (leaveSafeMode()) {
+        System.out.println("Successfully leave safe mode.");
+      }
+    } else if (cmd.equals("get")) {
+      boolean result = getSafeMode();
+      System.out.println("Safe Mode: " + result);
+    }
+  }
+
+  /**
+   * Request the Router entering safemode state.
+   * @return Return true if entering safemode successfully.
+   * @throws IOException
+   */
+  private boolean enterSafeMode() throws IOException {
+    RouterStateManager stateManager = client.getRouterStateManager();
+    EnterSafeModeResponse response = stateManager.enterSafeMode(
+        EnterSafeModeRequest.newInstance());
+    return response.getStatus();
+  }
+
+  /**
+   * Request the Router leaving safemode state.
+   * @return Return true if leaving safemode successfully.
+   * @throws IOException
+   */
+  private boolean leaveSafeMode() throws IOException {
+    RouterStateManager stateManager = client.getRouterStateManager();
+    LeaveSafeModeResponse response = stateManager.leaveSafeMode(
+        LeaveSafeModeRequest.newInstance());
+    return response.getStatus();
+  }
+
+  /**
+   * Verify if current Router state is safe mode state.
+   * @return True if the Router is in safe mode.
+   * @throws IOException
+   */
+  private boolean getSafeMode() throws IOException {
+    RouterStateManager stateManager = client.getRouterStateManager();
+    GetSafeModeResponse response = stateManager.getSafeMode(
+        GetSafeModeRequest.newInstance());
+    return response.isInSafeMode();
+  }
+
   /**
   /**
    * Inner class that stores ACL info of mount table.
    * Inner class that stores ACL info of mount table.
    */
    */

+ 25 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/FederationProtocol.proto

@@ -215,4 +215,29 @@ message RouterHeartbeatRequestProto {
 
 
 message RouterHeartbeatResponseProto {
 message RouterHeartbeatResponseProto {
   optional bool status = 1;
   optional bool status = 1;
+}
+
+/////////////////////////////////////////////////
+// Route State
+/////////////////////////////////////////////////
+
+message EnterSafeModeRequestProto {
+}
+
+message EnterSafeModeResponseProto {
+  optional bool status = 1;
+}
+
+message LeaveSafeModeRequestProto {
+}
+
+message LeaveSafeModeResponseProto {
+  optional bool status = 1;
+}
+
+message GetSafeModeRequestProto {
+}
+
+message GetSafeModeResponseProto {
+  optional bool isInSafeMode = 1;
 }
 }

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/proto/RouterProtocol.proto

@@ -44,4 +44,19 @@ service RouterAdminProtocolService {
    * Get matching mount entries
    * Get matching mount entries
    */
    */
   rpc getMountTableEntries(GetMountTableEntriesRequestProto) returns(GetMountTableEntriesResponseProto);
   rpc getMountTableEntries(GetMountTableEntriesRequestProto) returns(GetMountTableEntriesResponseProto);
+
+  /**
+   * Transform Router state to safe mode state.
+   */
+  rpc enterSafeMode(EnterSafeModeRequestProto) returns (EnterSafeModeResponseProto);
+
+  /**
+   * Transform Router state from safe mode to running state.
+   */
+  rpc leaveSafeMode(LeaveSafeModeRequestProto) returns (LeaveSafeModeResponseProto);
+
+  /**
+   * Verify if current Router state is safe mode state.
+   */
+  rpc getSafeMode(GetSafeModeRequestProto) returns (GetSafeModeResponseProto);
 }
 }

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md

@@ -430,6 +430,7 @@ Usage:
           [-ls <path>]
           [-ls <path>]
           [-setQuota <path> -nsQuota <nsQuota> -ssQuota <quota in bytes or quota size string>]
           [-setQuota <path> -nsQuota <nsQuota> -ssQuota <quota in bytes or quota size string>]
           [-clrQuota <path>]
           [-clrQuota <path>]
+          [-safemode enter | leave | get]
 
 
 | COMMAND\_OPTION | Description |
 | COMMAND\_OPTION | Description |
 |:---- |:---- |
 |:---- |:---- |
@@ -438,6 +439,7 @@ Usage:
 | `-ls` *path* | List mount points under specified path. |
 | `-ls` *path* | List mount points under specified path. |
 | `-setQuota` *path* `-nsQuota` *nsQuota* `-ssQuota` *ssQuota* | Set quota for specified path. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
 | `-setQuota` *path* `-nsQuota` *nsQuota* `-ssQuota` *ssQuota* | Set quota for specified path. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
 | `-clrQuota` *path* | Clear quota of given mount point. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
 | `-clrQuota` *path* | Clear quota of given mount point. See [HDFS Quotas Guide](./HdfsQuotaAdminGuide.html) for the quota detail. |
+| `-safemode` `enter` `leave` `get` | Manually set the Router entering or leaving safe mode. The option *get* will be used for verifying if the Router is in safe mode state. |
 
 
 The commands for managing Router-based federation. See [Mount table management](./HDFSRouterFederation.html#Mount_table_management) for more info.
 The commands for managing Router-based federation. See [Mount table management](./HDFSRouterFederation.html#Mount_table_management) for more info.
 
 

+ 5 - 1
hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSRouterFederation.md

@@ -83,7 +83,11 @@ The clients configure their DFS HA client (e.g., ConfiguredFailoverProvider or R
 
 
 * **Unavailable State Store:**
 * **Unavailable State Store:**
 If a Router cannot contact the State Store, it will enter into a Safe Mode state which disallows it from serving requests.
 If a Router cannot contact the State Store, it will enter into a Safe Mode state which disallows it from serving requests.
-Clients will treat Routers in Safe Mode as it was an Standby NameNode and try another Router.
+Clients will treat Routers in Safe Mode as it was an Standby NameNode and try another Router. There is a manual way to manage the Safe Mode for the Router.
+
+The Safe Mode state can be managed by using the following command:
+
+    [hdfs]$ $HADOOP_HOME/bin/hdfs dfsrouteradmin -safemode enter | leave | get
 
 
 * **NameNode heartbeat HA:**
 * **NameNode heartbeat HA:**
 For high availability and flexibility, multiple Routers can monitor the same NameNode and heartbeat the information to the State Store.
 For high availability and flexibility, multiple Routers can monitor the same NameNode and heartbeat the information to the State Store.

+ 48 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

@@ -40,12 +40,15 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntr
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.tools.federation.RouterAdmin;
 import org.apache.hadoop.hdfs.tools.federation.RouterAdmin;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.ToolRunner;
 import org.apache.hadoop.util.ToolRunner;
 import org.junit.After;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.Test;
+
+import com.google.common.base.Supplier;
 /**
 /**
  * Tests Router admin commands.
  * Tests Router admin commands.
  */
  */
@@ -363,4 +366,49 @@ public class TestRouterAdminCLI {
     assertEquals(HdfsConstants.QUOTA_DONT_SET, quotaUsage.getQuota());
     assertEquals(HdfsConstants.QUOTA_DONT_SET, quotaUsage.getQuota());
     assertEquals(HdfsConstants.QUOTA_DONT_SET, quotaUsage.getSpaceQuota());
     assertEquals(HdfsConstants.QUOTA_DONT_SET, quotaUsage.getSpaceQuota());
   }
   }
+
+  @Test
+  public void testManageSafeMode() throws Exception {
+    // ensure the Router become RUNNING state
+    waitState(RouterServiceState.RUNNING);
+    assertFalse(routerContext.getRouter().getRpcServer().isInSafeMode());
+    assertEquals(0, ToolRunner.run(admin,
+        new String[] {"-safemode", "enter"}));
+    // verify state
+    assertEquals(RouterServiceState.SAFEMODE,
+        routerContext.getRouter().getRouterState());
+    assertTrue(routerContext.getRouter().getRpcServer().isInSafeMode());
+
+    System.setOut(new PrintStream(out));
+    assertEquals(0, ToolRunner.run(admin,
+        new String[] {"-safemode", "get"}));
+    assertTrue(out.toString().contains("true"));
+
+    assertEquals(0, ToolRunner.run(admin,
+        new String[] {"-safemode", "leave"}));
+    // verify state
+    assertEquals(RouterServiceState.RUNNING,
+        routerContext.getRouter().getRouterState());
+    assertFalse(routerContext.getRouter().getRpcServer().isInSafeMode());
+
+    out.reset();
+    assertEquals(0, ToolRunner.run(admin,
+        new String[] {"-safemode", "get"}));
+    assertTrue(out.toString().contains("false"));
+  }
+
+  /**
+   * Wait for the Router transforming to expected state.
+   * @param expectedState Expected Router state.
+   * @throws Exception
+   */
+  private void waitState(final RouterServiceState expectedState)
+      throws Exception {
+    GenericTestUtils.waitFor(new Supplier<Boolean>() {
+      @Override
+      public Boolean get() {
+        return expectedState == routerContext.getRouter().getRouterState();
+      }
+    }, 1000, 30000);
+  }
 }
 }