You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by ar...@apache.org on 2019/04/13 14:27:17 UTC
[hadoop] branch trunk updated: HDDS-1198. Rename chill mode to safe
mode. Contributed by Siddharth Wagle. (#737)
This is an automated email from the ASF dual-hosted git repository.
arp pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git
The following commit(s) were added to refs/heads/trunk by this push:
new ebbda18 HDDS-1198. Rename chill mode to safe mode. Contributed by Siddharth Wagle. (#737)
ebbda18 is described below
commit ebbda181e4b79b5da5eaeb1e5b851435575de4b7
Author: Arpit Agarwal <ar...@users.noreply.github.com>
AuthorDate: Sat Apr 13 07:27:03 2019 -0700
HDDS-1198. Rename chill mode to safe mode. Contributed by Siddharth Wagle. (#737)
---
.../hdds/scm/client/ContainerOperationClient.java | 14 +-
.../org/apache/hadoop/hdds/HddsConfigKeys.java | 44 +++---
.../apache/hadoop/hdds/scm/client/ScmClient.java | 10 +-
.../protocol/StorageContainerLocationProtocol.java | 10 +-
...inerLocationProtocolClientSideTranslatorPB.java | 36 ++---
.../org/apache/hadoop/ozone/audit/SCMAction.java | 4 +-
...inerLocationProtocolServerSideTranslatorPB.java | 28 ++--
.../src/main/proto/ScmBlockLocationProtocol.proto | 2 +-
.../proto/StorageContainerLocationProtocol.proto | 22 +--
.../common/src/main/resources/ozone-default.xml | 24 +--
.../java/org/apache/hadoop/hdds/scm/ScmUtils.java | 2 +-
.../apache/hadoop/hdds/scm/block/BlockManager.java | 6 +-
.../hadoop/hdds/scm/block/BlockManagerImpl.java | 20 +--
.../replication/ReplicationActivityStatus.java | 6 +-
.../apache/hadoop/hdds/scm/events/SCMEvents.java | 8 +-
.../hadoop/hdds/scm/exceptions/SCMException.java | 2 +-
.../hadoop/hdds/scm/node/SCMNodeManager.java | 2 +-
.../hdds/scm/pipeline/PipelineReportHandler.java | 16 +-
.../ContainerSafeModeRule.java} | 32 ++--
.../DataNodeSafeModeRule.java} | 24 +--
.../HealthyPipelineSafeModeRule.java} | 30 ++--
.../OneReplicaPipelineSafeModeRule.java} | 28 ++--
.../hdds/scm/{chillmode => safemode}/Precheck.java | 2 +-
.../SCMSafeModeManager.java} | 146 +++++++++---------
.../SafeModeExitRule.java} | 32 ++--
.../SafeModeHandler.java} | 56 +++----
.../SafeModePrecheck.java} | 42 +++---
.../SafeModeRestrictedOps.java} | 10 +-
.../scm/{chillmode => safemode}/package-info.java | 2 +-
.../hdds/scm/server/SCMBlockProtocolServer.java | 4 +-
.../hdds/scm/server/SCMClientProtocolServer.java | 44 +++---
.../hadoop/hdds/scm/server/SCMConfigurator.java | 22 +--
.../apache/hadoop/hdds/scm/server/SCMMXBean.java | 8 +-
.../hdds/scm/server/StorageContainerManager.java | 48 +++---
.../main/resources/webapps/scm/scm-overview.html | 4 +-
.../hadoop/hdds/scm/block/TestBlockManager.java | 48 +++---
.../hadoop/hdds/scm/container/MockNodeManager.java | 12 +-
.../scm/container/TestSCMContainerManager.java | 6 +-
.../hadoop/hdds/scm/node/TestSCMNodeManager.java | 4 +-
.../TestHealthyPipelineSafeModeRule.java} | 56 +++----
.../TestOneReplicaPipelineSafeModeRule.java} | 20 +--
.../TestSCMSafeModeManager.java} | 164 ++++++++++-----------
.../TestSafeModeHandler.java} | 46 +++---
.../scm/{chillmode => safemode}/package-info.java | 4 +-
.../scm/server/TestSCMClientProtocolServer.java | 10 +-
.../org/apache/hadoop/hdds/scm/cli/SCMCLI.java | 2 +-
...ubcommand.java => SafeModeCheckSubcommand.java} | 16 +-
...hillModeCommands.java => SafeModeCommands.java} | 16 +-
...Subcommand.java => SafeModeExitSubcommand.java} | 14 +-
.../hadoop/ozone/om/exceptions/OMException.java | 2 +-
.../src/main/proto/OzoneManagerProtocol.proto | 2 +-
.../TestContainerStateManagerIntegration.java | 2 +-
.../hdds/scm/pipeline/TestSCMPipelineManager.java | 8 +-
.../TestSCMSafeModeWithPipelineRules.java} | 28 ++--
.../org/apache/hadoop/ozone/MiniOzoneCluster.java | 4 +-
.../apache/hadoop/ozone/MiniOzoneClusterImpl.java | 8 +-
...{TestScmChillMode.java => TestScmSafeMode.java} | 94 ++++++------
.../org/apache/hadoop/ozone/scm/TestSCMMXBean.java | 8 +-
.../org/apache/hadoop/ozone/om/KeyManagerImpl.java | 4 +-
.../ozone/om/ratis/OzoneManagerStateMachine.java | 2 +-
.../apache/hadoop/ozone/om/TestKeyManagerImpl.java | 14 +-
.../om/ratis/TestOzoneManagerStateMachine.java | 8 +-
.../ozone/genesis/BenchMarkOzoneManager.java | 6 +-
.../apache/hadoop/ozone/genesis/BenchMarkSCM.java | 6 +-
64 files changed, 702 insertions(+), 702 deletions(-)
diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
index 4697218..b0be34d 100644
--- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
+++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/client/ContainerOperationClient.java
@@ -447,22 +447,22 @@ public class ContainerOperationClient implements ScmClient {
}
/**
- * Check if SCM is in chill mode.
+ * Check if SCM is in safe mode.
*
- * @return Returns true if SCM is in chill mode else returns false.
+ * @return Returns true if SCM is in safe mode else returns false.
* @throws IOException
*/
- public boolean inChillMode() throws IOException {
- return storageContainerLocationClient.inChillMode();
+ public boolean inSafeMode() throws IOException {
+ return storageContainerLocationClient.inSafeMode();
}
/**
- * Force SCM out of chill mode.
+ * Force SCM out of safe mode.
*
* @return returns true if operation is successful.
* @throws IOException
*/
- public boolean forceExitChillMode() throws IOException {
- return storageContainerLocationClient.forceExitChillMode();
+ public boolean forceExitSafeMode() throws IOException {
+ return storageContainerLocationClient.forceExitSafeMode();
}
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
index 3bb3895..83e270b 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/HddsConfigKeys.java
@@ -63,47 +63,47 @@ public final class HddsConfigKeys {
public static final String HDDS_CONTAINER_CLOSE_THRESHOLD =
"hdds.container.close.threshold";
public static final float HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT = 0.9f;
- public static final String HDDS_SCM_CHILLMODE_ENABLED =
- "hdds.scm.chillmode.enabled";
+ public static final String HDDS_SCM_SAFEMODE_ENABLED =
+ "hdds.scm.safemode.enabled";
public static final String HDDS_CONTAINERSCRUB_ENABLED =
"hdds.containerscrub.enabled";
public static final boolean HDDS_CONTAINERSCRUB_ENABLED_DEFAULT = false;
- public static final boolean HDDS_SCM_CHILLMODE_ENABLED_DEFAULT = true;
- public static final String HDDS_SCM_CHILLMODE_MIN_DATANODE =
- "hdds.scm.chillmode.min.datanode";
- public static final int HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT = 1;
+ public static final boolean HDDS_SCM_SAFEMODE_ENABLED_DEFAULT = true;
+ public static final String HDDS_SCM_SAFEMODE_MIN_DATANODE =
+ "hdds.scm.safemode.min.datanode";
+ public static final int HDDS_SCM_SAFEMODE_MIN_DATANODE_DEFAULT = 1;
public static final String
- HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT =
- "hdds.scm.wait.time.after.chillmode.exit";
+ HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT =
+ "hdds.scm.wait.time.after.safemode.exit";
public static final String
- HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT_DEFAULT = "5m";
+ HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT_DEFAULT = "5m";
- public static final String HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK =
- "hdds.scm.chillmode.pipeline-availability.check";
+ public static final String HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK =
+ "hdds.scm.safemode.pipeline-availability.check";
public static final boolean
- HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT = false;
+ HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT = false;
// % of containers which should have at least one reported replica
- // before SCM comes out of chill mode.
- public static final String HDDS_SCM_CHILLMODE_THRESHOLD_PCT =
- "hdds.scm.chillmode.threshold.pct";
- public static final double HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT = 0.99;
+ // before SCM comes out of safe mode.
+ public static final String HDDS_SCM_SAFEMODE_THRESHOLD_PCT =
+ "hdds.scm.safemode.threshold.pct";
+ public static final double HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT = 0.99;
// percentage of healthy pipelines, where all 3 datanodes are reported in the
// pipeline.
- public static final String HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT =
- "hdds.scm.chillmode.healthy.pipelie.pct";
+ public static final String HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT =
+ "hdds.scm.safemode.healthy.pipelie.pct";
public static final double
- HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10;
+ HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT = 0.10;
- public static final String HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT =
- "hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct";
+ public static final String HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT =
+ "hdds.scm.safemode.atleast.one.node.reported.pipeline.pct";
public static final double
- HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT = 0.90;
+ HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT = 0.90;
public static final String HDDS_LOCK_MAX_CONCURRENCY =
"hdds.lock.max.concurrency";
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
index 4f4239f..85821ac 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java
@@ -189,18 +189,18 @@ public interface ScmClient extends Closeable {
void closePipeline(HddsProtos.PipelineID pipelineID) throws IOException;
/**
- * Check if SCM is in chill mode.
+ * Check if SCM is in safe mode.
*
- * @return Returns true if SCM is in chill mode else returns false.
+ * @return Returns true if SCM is in safe mode else returns false.
* @throws IOException
*/
- boolean inChillMode() throws IOException;
+ boolean inSafeMode() throws IOException;
/**
- * Force SCM out of chill mode.
+ * Force SCM out of safe mode.
*
* @return returns true if operation is successful.
* @throws IOException
*/
- boolean forceExitChillMode() throws IOException;
+ boolean forceExitSafeMode() throws IOException;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
index 8831d16..cc220a5 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java
@@ -163,18 +163,18 @@ public interface StorageContainerLocationProtocol extends Closeable {
ScmInfo getScmInfo() throws IOException;
/**
- * Check if SCM is in chill mode.
+ * Check if SCM is in safe mode.
*
- * @return Returns true if SCM is in chill mode else returns false.
+ * @return Returns true if SCM is in safe mode else returns false.
* @throws IOException
*/
- boolean inChillMode() throws IOException;
+ boolean inSafeMode() throws IOException;
/**
- * Force SCM out of Chill mode.
+ * Force SCM out of Safe mode.
*
* @return returns true if operation is successful.
* @throws IOException
*/
- boolean forceExitChillMode() throws IOException;
+ boolean forceExitSafeMode() throws IOException;
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
index 4a1442e..acd446e 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java
@@ -23,12 +23,12 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ListPipelineResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ClosePipelineRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitChillModeRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitChillModeResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitSafeModeRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ForceExitSafeModeResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InChillModeRequestProto;
-import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InChillModeResponseProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InSafeModeRequestProto;
+import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InSafeModeResponseProto;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
@@ -370,38 +370,38 @@ public final class StorageContainerLocationProtocolClientSideTranslatorPB
}
/**
- * Check if SCM is in chill mode.
+ * Check if SCM is in safe mode.
*
- * @return Returns true if SCM is in chill mode else returns false.
+ * @return Returns true if SCM is in safe mode else returns false.
* @throws IOException
*/
@Override
- public boolean inChillMode() throws IOException {
- InChillModeRequestProto request =
- InChillModeRequestProto.getDefaultInstance();
+ public boolean inSafeMode() throws IOException {
+ InSafeModeRequestProto request =
+ InSafeModeRequestProto.getDefaultInstance();
try {
- InChillModeResponseProto resp = rpcProxy.inChillMode(
+ InSafeModeResponseProto resp = rpcProxy.inSafeMode(
NULL_RPC_CONTROLLER, request);
- return resp.getInChillMode();
+ return resp.getInSafeMode();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
}
/**
- * Force SCM out of Chill mode.
+ * Force SCM out of Safe mode.
*
* @return returns true if operation is successful.
* @throws IOException
*/
@Override
- public boolean forceExitChillMode() throws IOException {
- ForceExitChillModeRequestProto request =
- ForceExitChillModeRequestProto.getDefaultInstance();
+ public boolean forceExitSafeMode() throws IOException {
+ ForceExitSafeModeRequestProto request =
+ ForceExitSafeModeRequestProto.getDefaultInstance();
try {
- ForceExitChillModeResponseProto resp = rpcProxy
- .forceExitChillMode(NULL_RPC_CONTROLLER, request);
- return resp.getExitedChillMode();
+ ForceExitSafeModeResponseProto resp = rpcProxy
+ .forceExitSafeMode(NULL_RPC_CONTROLLER, request);
+ return resp.getExitedSafeMode();
} catch (ServiceException e) {
throw ProtobufHelper.getRemoteException(e);
}
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
index 950a783..f662955 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java
@@ -34,8 +34,8 @@ public enum SCMAction implements AuditAction {
LIST_PIPELINE,
CLOSE_PIPELINE,
DELETE_CONTAINER,
- IN_CHILL_MODE,
- FORCE_EXIT_CHILL_MODE;
+ IN_SAFE_MODE,
+ FORCE_EXIT_SAFE_MODE;
@Override
public String getAction() {
diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java
index 0b681c5..a994ed7 100644
--- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/protocolPB/StorageContainerLocationProtocolServerSideTranslatorPB.java
@@ -24,13 +24,13 @@ import io.opentracing.Scope;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.hdds.protocol.proto
- .StorageContainerLocationProtocolProtos.InChillModeRequestProto;
+ .StorageContainerLocationProtocolProtos.InSafeModeRequestProto;
import org.apache.hadoop.hdds.protocol.proto
- .StorageContainerLocationProtocolProtos.InChillModeResponseProto;
+ .StorageContainerLocationProtocolProtos.InSafeModeResponseProto;
import org.apache.hadoop.hdds.protocol.proto
- .StorageContainerLocationProtocolProtos.ForceExitChillModeRequestProto;
+ .StorageContainerLocationProtocolProtos.ForceExitSafeModeRequestProto;
import org.apache.hadoop.hdds.protocol.proto
- .StorageContainerLocationProtocolProtos.ForceExitChillModeResponseProto;
+ .StorageContainerLocationProtocolProtos.ForceExitSafeModeResponseProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineRequestProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
import org.apache.hadoop.hdds.scm.ScmInfo;
@@ -282,26 +282,26 @@ public final class StorageContainerLocationProtocolServerSideTranslatorPB
}
@Override
- public InChillModeResponseProto inChillMode(
+ public InSafeModeResponseProto inSafeMode(
RpcController controller,
- InChillModeRequestProto request) throws ServiceException {
+ InSafeModeRequestProto request) throws ServiceException {
try (Scope scope = TracingUtil
- .importAndCreateScope("inChillMode", request.getTraceID())) {
- return InChillModeResponseProto.newBuilder()
- .setInChillMode(impl.inChillMode()).build();
+ .importAndCreateScope("inSafeMode", request.getTraceID())) {
+ return InSafeModeResponseProto.newBuilder()
+ .setInSafeMode(impl.inSafeMode()).build();
} catch (IOException ex) {
throw new ServiceException(ex);
}
}
@Override
- public ForceExitChillModeResponseProto forceExitChillMode(
- RpcController controller, ForceExitChillModeRequestProto request)
+ public ForceExitSafeModeResponseProto forceExitSafeMode(
+ RpcController controller, ForceExitSafeModeRequestProto request)
throws ServiceException {
try (Scope scope = TracingUtil
- .importAndCreateScope("forceExitChillMode", request.getTraceID())) {
- return ForceExitChillModeResponseProto.newBuilder()
- .setExitedChillMode(impl.forceExitChillMode()).build();
+ .importAndCreateScope("forceExitSafeMode", request.getTraceID())) {
+ return ForceExitSafeModeResponseProto.newBuilder()
+ .setExitedSafeMode(impl.forceExitSafeMode()).build();
} catch (IOException ex) {
throw new ServiceException(ex);
}
diff --git a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto
index c139089..eebda0a 100644
--- a/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/ScmBlockLocationProtocol.proto
@@ -88,7 +88,7 @@ message DeleteKeyBlocksResultProto {
message DeleteScmBlockResult {
enum Result {
success = 1;
- chillMode = 2;
+ safeMode = 2;
errorNotFound = 3;
unknownFailure = 4;
}
diff --git a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto
index ade54a4..2c397ef 100644
--- a/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto
+++ b/hadoop-hdds/common/src/main/proto/StorageContainerLocationProtocol.proto
@@ -176,20 +176,20 @@ message ClosePipelineRequestProto {
message ClosePipelineResponseProto {
}
-message InChillModeRequestProto {
+message InSafeModeRequestProto {
optional string traceID = 1;
}
-message InChillModeResponseProto {
- required bool inChillMode = 1;
+message InSafeModeResponseProto {
+ required bool inSafeMode = 1;
}
-message ForceExitChillModeRequestProto {
+message ForceExitSafeModeRequestProto {
optional string traceID = 1;
}
-message ForceExitChillModeResponseProto {
- required bool exitedChillMode = 1;
+message ForceExitSafeModeResponseProto {
+ required bool exitedSafeMode = 1;
}
/**
@@ -265,14 +265,14 @@ service StorageContainerLocationProtocolService {
returns (GetScmInfoRespsonseProto);
/**
- * Checks if SCM is in ChillMode.
+ * Checks if SCM is in SafeMode.
*/
- rpc inChillMode(InChillModeRequestProto)
- returns (InChillModeResponseProto);
+ rpc inSafeMode(InSafeModeRequestProto)
+ returns (InSafeModeResponseProto);
/**
* Returns information about SCM.
*/
- rpc forceExitChillMode(ForceExitChillModeRequestProto)
- returns (ForceExitChillModeResponseProto);
+ rpc forceExitSafeMode(ForceExitSafeModeRequestProto)
+ returns (ForceExitSafeModeResponseProto);
}
diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml
index ca61c1a..f40040c 100644
--- a/hadoop-hdds/common/src/main/resources/ozone-default.xml
+++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml
@@ -1300,52 +1300,52 @@
</property>
<property>
- <name>hdds.scm.chillmode.threshold.pct</name>
+ <name>hdds.scm.safemode.threshold.pct</name>
<value>0.99</value>
<tag>HDDS,SCM,OPERATION</tag>
<description> % of containers which should have at least one
- reported replica before SCM comes out of chill mode.
+ reported replica before SCM comes out of safe mode.
</description>
</property>
<property>
- <name>hdds.scm.wait.time.after.chillmode.exit</name>
+ <name>hdds.scm.wait.time.after.safemode.exit</name>
<value>5m</value>
<tag>HDDS,SCM,OPERATION</tag>
- <description> After exiting chillmode, wait for configured interval of
+ <description> After exiting safemode, wait for configured interval of
time to start replication monitor and cleanup activities of unhealthy
pipelines.
</description>
</property>
<property>
- <name>hdds.scm.chillmode.enabled</name>
+ <name>hdds.scm.safemode.enabled</name>
<value>true</value>
<tag>HDDS,SCM,OPERATION</tag>
- <description>Boolean value to enable or disable SCM chill mode.
+ <description>Boolean value to enable or disable SCM safe mode.
</description>
</property>
<property>
- <name>hdds.scm.chillmode.min.datanode</name>
+ <name>hdds.scm.safemode.min.datanode</name>
<value>1</value>
<tag>HDDS,SCM,OPERATION</tag>
<description>Minimum DataNodes which should be registered to get SCM out of
- chill mode.
+ safe mode.
</description>
</property>
<property>
- <name>hdds.scm.chillmode.pipeline-availability.check</name>
+ <name>hdds.scm.safemode.pipeline-availability.check</name>
<value>false</value>
<tag>HDDS,SCM,OPERATION</tag>
<description>
- Boolean value to enable pipeline availability check during SCM chill mode.
+ Boolean value to enable pipeline availability check during SCM safe mode.
</description>
</property>
<property>
- <name>hdds.scm.chillmode.healthy.pipelie.pct</name>
+ <name>hdds.scm.safemode.healthy.pipelie.pct</name>
<value>0.10</value>
<tag>HDDS,SCM,OPERATION</tag>
<description>
@@ -1355,7 +1355,7 @@
</property>
<property>
- <name>hdds.scm.chillmode.atleast.one.node.reported.pipeline.pct</name>
+ <name>hdds.scm.safemode.atleast.one.node.reported.pipeline.pct</name>
<value>0.90</value>
<tag>HDDS,SCM,OPERATION</tag>
<description>
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
index 3192a74..3770253 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java
@@ -22,7 +22,7 @@ package org.apache.hadoop.hdds.scm;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
-import org.apache.hadoop.hdds.scm.chillmode.Precheck;
+import org.apache.hadoop.hdds.scm.safemode.Precheck;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.server.ServerUtils;
import org.slf4j.Logger;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
index a12a49d..0bdbeb8 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManager.java
@@ -81,9 +81,9 @@ public interface BlockManager extends Closeable {
SCMBlockDeletingService getSCMBlockDeletingService();
/**
- * Set ChillMode status.
+ * Set SafeMode status.
*
- * @param chillModeStatus
+ * @param safeModeStatus
*/
- void setChillModeStatus(boolean chillModeStatus);
+ void setSafeModeStatus(boolean safeModeStatus);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
index 0ca6670..31d82f6 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
@@ -34,7 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.ScmUtils;
-import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
+import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
@@ -78,7 +78,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
private final SCMBlockDeletingService blockDeletingService;
private ObjectName mxBean;
- private ChillModePrecheck chillModePrecheck;
+ private SafeModePrecheck safeModePrecheck;
/**
* Constructor.
@@ -116,7 +116,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
new SCMBlockDeletingService(deletedBlockLog, containerManager,
scm.getScmNodeManager(), scm.getEventQueue(), svcInterval,
serviceTimeout, conf);
- chillModePrecheck = new ChillModePrecheck(conf);
+ safeModePrecheck = new SafeModePrecheck(conf);
}
/**
@@ -154,7 +154,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
ReplicationFactor factor, String owner, ExcludeList excludeList)
throws IOException {
LOG.trace("Size;{} , type : {}, factor : {} ", size, type, factor);
- ScmUtils.preCheck(ScmOps.allocateBlock, chillModePrecheck);
+ ScmUtils.preCheck(ScmOps.allocateBlock, safeModePrecheck);
if (size < 0 || size > containerSize) {
LOG.warn("Invalid block size requested : {}", size);
throw new SCMException("Unsupported block size: " + size,
@@ -254,7 +254,7 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
*/
@Override
public void deleteBlocks(List<BlockID> blockIDs) throws IOException {
- ScmUtils.preCheck(ScmOps.deleteBlock, chillModePrecheck);
+ ScmUtils.preCheck(ScmOps.deleteBlock, safeModePrecheck);
LOG.info("Deleting blocks {}", StringUtils.join(",", blockIDs));
Map<Long, List<Long>> containerBlocks = new HashMap<>();
@@ -324,15 +324,15 @@ public class BlockManagerImpl implements BlockManager, BlockmanagerMXBean {
}
@Override
- public void setChillModeStatus(boolean chillModeStatus) {
- this.chillModePrecheck.setInChillMode(chillModeStatus);
+ public void setSafeModeStatus(boolean safeModeStatus) {
+ this.safeModePrecheck.setInSafeMode(safeModeStatus);
}
/**
- * Returns status of scm chill mode determined by CHILL_MODE_STATUS event.
+ * Returns status of scm safe mode determined by SAFE_MODE_STATUS event.
* */
- public boolean isScmInChillMode() {
- return this.chillModePrecheck.isInChillMode();
+ public boolean isScmInSafeMode() {
+ return this.safeModePrecheck.isInSafeMode();
}
/**
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
index 2eca221..900e35c 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/replication/ReplicationActivityStatus.java
@@ -86,12 +86,12 @@ public class ReplicationActivityStatus implements
/**
* Waits for
- * {@link HddsConfigKeys#HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT} and set
+ * {@link HddsConfigKeys#HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT} and set
* replicationEnabled to start replication monitor thread.
*/
- public void fireReplicationStart(boolean chillModeStatus,
+ public void fireReplicationStart(boolean safeModeStatus,
long waitTime) {
- if (!chillModeStatus) {
+ if (!safeModeStatus) {
scheduler.schedule(() -> {
setReplicationEnabled(true);
LOG.info("Replication Timer sleep for {} ms completed. Enable "
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
index 5be512b..ed9727a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/events/SCMEvents.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdds.scm.events;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus;
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler
.ReplicationStatus;
@@ -106,7 +106,7 @@ public final class SCMEvents {
/**
* PipelineReport processed by pipeline report handler. This event is
- * received by HealthyPipelineChillModeRule.
+ * received by HealthyPipelineSafeModeRule.
*/
public static final TypedEvent<PipelineReportFromDatanode>
PROCESSED_PIPELINE_REPORT = new TypedEvent<>(
@@ -254,8 +254,8 @@ public final class SCMEvents {
*/
public static final TypedEvent<Boolean> START_REPLICATION =
new TypedEvent<>(Boolean.class);
- public static final TypedEvent<ChillModeStatus> CHILL_MODE_STATUS =
- new TypedEvent<>(ChillModeStatus.class);
+ public static final TypedEvent<SafeModeStatus> SAFE_MODE_STATUS =
+ new TypedEvent<>(SafeModeStatus.class);
/**
* Private Ctor. Never Constructed.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
index dae0b06..01166ad 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/exceptions/SCMException.java
@@ -103,7 +103,7 @@ public class SCMException extends IOException {
FAILED_TO_FIND_SUITABLE_NODE,
INVALID_CAPACITY,
INVALID_BLOCK_SIZE,
- CHILL_MODE_EXCEPTION,
+ SAFE_MODE_EXCEPTION,
FAILED_TO_LOAD_OPEN_CONTAINER,
FAILED_TO_ALLOCATE_CONTAINER,
FAILED_TO_CHANGE_CONTAINER_STATE,
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
index 4464ed1..17f72f6 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/node/SCMNodeManager.java
@@ -107,7 +107,7 @@ public class SCMNodeManager implements NodeManager {
this.version = VersionInfo.getLatestVersion();
this.commandQueue = new CommandQueue();
this.scmManager = scmManager;
- LOG.info("Entering startup chill mode.");
+ LOG.info("Entering startup safe mode.");
registerMXBean();
this.metrics = SCMNodeMetrics.create(this);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
index 330ad8b..1bba45d 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/PipelineReportHandler.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReport;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.server
.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
@@ -48,20 +48,20 @@ public class PipelineReportHandler implements
.getLogger(PipelineReportHandler.class);
private final PipelineManager pipelineManager;
private final Configuration conf;
- private final SCMChillModeManager scmChillModeManager;
+ private final SCMSafeModeManager scmSafeModeManager;
private final boolean pipelineAvailabilityCheck;
- public PipelineReportHandler(SCMChillModeManager scmChillModeManager,
+ public PipelineReportHandler(SCMSafeModeManager scmSafeModeManager,
PipelineManager pipelineManager,
Configuration conf) {
Preconditions.checkNotNull(pipelineManager);
- Objects.requireNonNull(scmChillModeManager);
- this.scmChillModeManager = scmChillModeManager;
+ Objects.requireNonNull(scmSafeModeManager);
+ this.scmSafeModeManager = scmSafeModeManager;
this.pipelineManager = pipelineManager;
this.conf = conf;
this.pipelineAvailabilityCheck = conf.getBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT);
}
@@ -83,7 +83,7 @@ public class PipelineReportHandler implements
report, dn, e);
}
}
- if (pipelineAvailabilityCheck && scmChillModeManager.getInChillMode()) {
+ if (pipelineAvailabilityCheck && scmSafeModeManager.getInSafeMode()) {
publisher.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
pipelineReportFromDatanode);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ContainerChillModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
similarity index 80%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ContainerChillModeRule.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
index cd08786..14091b2 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ContainerChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ContainerSafeModeRule.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import java.util.List;
import java.util.Map;
@@ -36,30 +36,30 @@ import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.hdds.server.events.TypedEvent;
/**
- * Class defining Chill mode exit criteria for Containers.
+ * Class defining Safe mode exit criteria for Containers.
*/
-public class ContainerChillModeRule extends
- ChillModeExitRule<NodeRegistrationContainerReport>{
+public class ContainerSafeModeRule extends
+ SafeModeExitRule<NodeRegistrationContainerReport>{
// Required cutoff % for containers with at least 1 reported replica.
- private double chillModeCutoff;
+ private double safeModeCutoff;
// Containers read from scm db (excluding containers in ALLOCATED state).
private Map<Long, ContainerInfo> containerMap;
private double maxContainer;
private AtomicLong containerWithMinReplicas = new AtomicLong(0);
- public ContainerChillModeRule(String ruleName, EventQueue eventQueue,
+ public ContainerSafeModeRule(String ruleName, EventQueue eventQueue,
Configuration conf,
- List<ContainerInfo> containers, SCMChillModeManager manager) {
+ List<ContainerInfo> containers, SCMSafeModeManager manager) {
super(manager, ruleName, eventQueue);
- chillModeCutoff = conf.getDouble(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT);
+ safeModeCutoff = conf.getDouble(
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT);
Preconditions.checkArgument(
- (chillModeCutoff >= 0.0 && chillModeCutoff <= 1.0),
- HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT +
+ (safeModeCutoff >= 0.0 && safeModeCutoff <= 1.0),
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT +
" value should be >= 0.0 and <= 1.0");
containerMap = new ConcurrentHashMap<>();
@@ -87,7 +87,7 @@ public class ContainerChillModeRule extends
@Override
protected boolean validate() {
- return getCurrentContainerThreshold() >= chillModeCutoff;
+ return getCurrentContainerThreshold() >= safeModeCutoff;
}
@VisibleForTesting
@@ -109,9 +109,9 @@ public class ContainerChillModeRule extends
}
});
- if (scmInChillMode()) {
- SCMChillModeManager.getLogger().info(
- "SCM in chill mode. {} % containers have at least one"
+ if (scmInSafeMode()) {
+ SCMSafeModeManager.getLogger().info(
+ "SCM in safe mode. {} % containers have at least one"
+ " reported replica.",
(containerWithMinReplicas.doubleValue() / maxContainer) * 100);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/DataNodeChillModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/DataNodeSafeModeRule.java
similarity index 76%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/DataNodeChillModeRule.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/DataNodeSafeModeRule.java
index aae2160..1029d71 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/DataNodeChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/DataNodeSafeModeRule.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import java.util.HashSet;
import java.util.UUID;
@@ -29,25 +29,25 @@ import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.hdds.server.events.TypedEvent;
/**
- * Class defining Chill mode exit criteria according to number of DataNodes
+ * Class defining Safe mode exit criteria according to number of DataNodes
* registered with SCM.
*/
-public class DataNodeChillModeRule extends
- ChillModeExitRule<NodeRegistrationContainerReport>{
+public class DataNodeSafeModeRule extends
+ SafeModeExitRule<NodeRegistrationContainerReport>{
- // Min DataNodes required to exit chill mode.
+ // Min DataNodes required to exit safe mode.
private int requiredDns;
private int registeredDns = 0;
// Set to track registered DataNodes.
private HashSet<UUID> registeredDnSet;
- public DataNodeChillModeRule(String ruleName, EventQueue eventQueue,
+ public DataNodeSafeModeRule(String ruleName, EventQueue eventQueue,
Configuration conf,
- SCMChillModeManager manager) {
+ SCMSafeModeManager manager) {
super(manager, ruleName, eventQueue);
requiredDns = conf.getInt(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE_DEFAULT);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE_DEFAULT);
registeredDnSet = new HashSet<>(requiredDns * 2);
}
@@ -67,9 +67,9 @@ public class DataNodeChillModeRule extends
registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
registeredDns = registeredDnSet.size();
- if (scmInChillMode()) {
- SCMChillModeManager.getLogger().info(
- "SCM in chill mode. {} DataNodes registered, {} required.",
+ if (scmInSafeMode()) {
+ SCMSafeModeManager.getLogger().info(
+ "SCM in safe mode. {} DataNodes registered, {} required.",
registeredDns, requiredDns);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/HealthyPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
similarity index 85%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/HealthyPipelineChillModeRule.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
index e0a9a40..aee17b1 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/HealthyPipelineChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.conf.Configuration;
@@ -42,38 +42,38 @@ import java.util.HashSet;
import java.util.Set;
/**
- * Class defining Chill mode exit criteria for Pipelines.
+ * Class defining Safe mode exit criteria for Pipelines.
*
* This rule defines percentage of healthy pipelines need to be reported.
- * Once chill mode exit happens, this rules take care of writes can go
+ * Once safe mode exit happens, this rules take care of writes can go
* through in a cluster.
*/
-public class HealthyPipelineChillModeRule
- extends ChillModeExitRule<PipelineReportFromDatanode>{
+public class HealthyPipelineSafeModeRule
+ extends SafeModeExitRule<PipelineReportFromDatanode>{
public static final Logger LOG =
- LoggerFactory.getLogger(HealthyPipelineChillModeRule.class);
+ LoggerFactory.getLogger(HealthyPipelineSafeModeRule.class);
private final PipelineManager pipelineManager;
private final int healthyPipelineThresholdCount;
private int currentHealthyPipelineCount = 0;
private final Set<DatanodeDetails> processedDatanodeDetails =
new HashSet<>();
- HealthyPipelineChillModeRule(String ruleName, EventQueue eventQueue,
+ HealthyPipelineSafeModeRule(String ruleName, EventQueue eventQueue,
PipelineManager pipelineManager,
- SCMChillModeManager manager, Configuration configuration) {
+ SCMSafeModeManager manager, Configuration configuration) {
super(manager, ruleName, eventQueue);
this.pipelineManager = pipelineManager;
double healthyPipelinesPercent =
configuration.getDouble(HddsConfigKeys.
- HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT,
+ HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT,
HddsConfigKeys.
- HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT);
+ HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT);
Preconditions.checkArgument(
(healthyPipelinesPercent >= 0.0 && healthyPipelinesPercent <= 1.0),
HddsConfigKeys.
- HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT
+ HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT
+ " value should be >= 0.0 and <= 1.0");
// As we want to wait for 3 node pipelines
@@ -108,7 +108,7 @@ public class HealthyPipelineChillModeRule
protected void process(PipelineReportFromDatanode
pipelineReportFromDatanode) {
- // When SCM is in chill mode for long time, already registered
+ // When SCM is in safe mode for long time, already registered
// datanode can send pipeline report again, then pipeline handler fires
// processed report event, we should not consider this pipeline report
// from datanode again during threshold calculation.
@@ -138,9 +138,9 @@ public class HealthyPipelineChillModeRule
}
}
- if (scmInChillMode()) {
- SCMChillModeManager.getLogger().info(
- "SCM in chill mode. Healthy pipelines reported count is {}, " +
+ if (scmInSafeMode()) {
+ SCMSafeModeManager.getLogger().info(
+ "SCM in safe mode. Healthy pipelines reported count is {}, " +
"required healthy pipeline reported count is {}",
currentHealthyPipelineCount, healthyPipelineThresholdCount);
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
similarity index 84%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
index 01db476..f8f41b7 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/OneReplicaPipelineChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
@@ -15,7 +15,7 @@
* the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
@@ -44,13 +44,13 @@ import java.util.Set;
/**
* This rule covers whether we have at least one datanode is reported for each
* pipeline. This rule is for all open containers, we have at least one
- * replica available for read when we exit chill mode.
+ * replica available for read when we exit safe mode.
*/
-public class OneReplicaPipelineChillModeRule extends
- ChillModeExitRule<PipelineReportFromDatanode> {
+public class OneReplicaPipelineSafeModeRule extends
+ SafeModeExitRule<PipelineReportFromDatanode> {
private static final Logger LOG =
- LoggerFactory.getLogger(OneReplicaPipelineChillModeRule.class);
+ LoggerFactory.getLogger(OneReplicaPipelineSafeModeRule.class);
private int thresholdCount;
private Set<PipelineID> reportedPipelineIDSet = new HashSet<>();
@@ -58,21 +58,21 @@ public class OneReplicaPipelineChillModeRule extends
private int currentReportedPipelineCount = 0;
- public OneReplicaPipelineChillModeRule(String ruleName, EventQueue eventQueue,
+ public OneReplicaPipelineSafeModeRule(String ruleName, EventQueue eventQueue,
PipelineManager pipelineManager,
- SCMChillModeManager chillModeManager, Configuration configuration) {
- super(chillModeManager, ruleName, eventQueue);
+ SCMSafeModeManager safeModeManager, Configuration configuration) {
+ super(safeModeManager, ruleName, eventQueue);
this.pipelineManager = pipelineManager;
double percent =
configuration.getDouble(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT,
HddsConfigKeys.
- HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT);
+ HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT_DEFAULT);
Preconditions.checkArgument((percent >= 0.0 && percent <= 1.0),
HddsConfigKeys.
- HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT +
+ HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT +
" value should be >= 0.0 and <= 1.0");
int totalPipelineCount =
@@ -125,9 +125,9 @@ public class OneReplicaPipelineChillModeRule extends
currentReportedPipelineCount = reportedPipelineIDSet.size();
- if (scmInChillMode()) {
- SCMChillModeManager.getLogger().info(
- "SCM in chill mode. Pipelines with atleast one datanode reported " +
+ if (scmInSafeMode()) {
+ SCMSafeModeManager.getLogger().info(
+ "SCM in safe mode. Pipelines with atleast one datanode reported " +
"count is {}, required atleast one datanode reported per " +
"pipeline count is {}",
currentReportedPipelineCount, thresholdCount);
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/Precheck.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java
similarity index 95%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/Precheck.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java
index 30daa85..12c6c31 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/Precheck.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
similarity index 58%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
index 194990d..365259a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/SCMChillModeManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import com.google.common.annotations.VisibleForTesting;
@@ -36,29 +36,29 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
- * StorageContainerManager enters chill mode on startup to allow system to
+ * StorageContainerManager enters safe mode on startup to allow system to
* reach a stable state before becoming fully functional. SCM will wait
- * for certain resources to be reported before coming out of chill mode.
+ * for certain resources to be reported before coming out of safe mode.
*
- * ChillModeExitRule defines format to define new rules which must be satisfied
- * to exit Chill mode.
+ * SafeModeExitRule defines format to define new rules which must be satisfied
+ * to exit Safe mode.
*
- * Current ChillMode rules:
- * 1. ContainerChillModeRule:
+ * Current SafeMode rules:
+ * 1. ContainerSafeModeRule:
* On every new datanode registration, it fires
* {@link SCMEvents#NODE_REGISTRATION_CONT_REPORT}. This rule handles this
* event. This rule process this report, increment the
* containerWithMinReplicas count when this reported replica is in the
* containerMap. Then validates if cutoff threshold for containers is meet.
*
- * 2. DatanodeChillModeRule:
+ * 2. DatanodeSafeModeRule:
* On every new datanode registration, it fires
* {@link SCMEvents#NODE_REGISTRATION_CONT_REPORT}. This rule handles this
* event. This rule process this report, and check if this is new node, add
* to its reported node list. Then validate it cutoff threshold for minimum
* number of datanode registered is met or not.
*
- * 3. HealthyPipelineChillModeRule:
+ * 3. HealthyPipelineSafeModeRule:
* Once the pipelineReportHandler processes the
* {@link SCMEvents#PIPELINE_REPORT}, it fires
* {@link SCMEvents#PROCESSED_PIPELINE_REPORT}. This rule handles this
@@ -66,7 +66,7 @@ import org.slf4j.LoggerFactory;
* and increments current healthy pipeline count. Then validate it cutoff
* threshold for healthy pipeline is met or not.
*
- * 4. OneReplicaPipelineChillModeRule:
+ * 4. OneReplicaPipelineSafeModeRule:
* Once the pipelineReportHandler processes the
* {@link SCMEvents#PIPELINE_REPORT}, it fires
* {@link SCMEvents#PROCESSED_PIPELINE_REPORT}. This rule handles this
@@ -75,19 +75,19 @@ import org.slf4j.LoggerFactory;
* per pipeline is met or not.
*
*/
-public class SCMChillModeManager {
+public class SCMSafeModeManager {
private static final Logger LOG =
- LoggerFactory.getLogger(SCMChillModeManager.class);
- private final boolean isChillModeEnabled;
- private AtomicBoolean inChillMode = new AtomicBoolean(true);
+ LoggerFactory.getLogger(SCMSafeModeManager.class);
+ private final boolean isSafeModeEnabled;
+ private AtomicBoolean inSafeMode = new AtomicBoolean(true);
- private Map<String, ChillModeExitRule> exitRules = new HashMap(1);
+ private Map<String, SafeModeExitRule> exitRules = new HashMap(1);
private Configuration config;
- private static final String CONT_EXIT_RULE = "ContainerChillModeRule";
- private static final String DN_EXIT_RULE = "DataNodeChillModeRule";
+ private static final String CONT_EXIT_RULE = "ContainerSafeModeRule";
+ private static final String DN_EXIT_RULE = "DataNodeSafeModeRule";
private static final String HEALTHY_PIPELINE_EXIT_RULE =
- "HealthyPipelineChillModeRule";
+ "HealthyPipelineSafeModeRule";
private static final String ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE =
"AtleastOneDatanodeReportedRule";
@@ -96,57 +96,57 @@ public class SCMChillModeManager {
private final EventQueue eventPublisher;
private final PipelineManager pipelineManager;
- public SCMChillModeManager(Configuration conf,
+ public SCMSafeModeManager(Configuration conf,
List<ContainerInfo> allContainers, PipelineManager pipelineManager,
EventQueue eventQueue) {
this.config = conf;
this.pipelineManager = pipelineManager;
this.eventPublisher = eventQueue;
- this.isChillModeEnabled = conf.getBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
+ this.isSafeModeEnabled = conf.getBoolean(
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT);
- if (isChillModeEnabled) {
- ContainerChillModeRule containerChillModeRule =
- new ContainerChillModeRule(CONT_EXIT_RULE, eventQueue, config,
+ if (isSafeModeEnabled) {
+ ContainerSafeModeRule containerSafeModeRule =
+ new ContainerSafeModeRule(CONT_EXIT_RULE, eventQueue, config,
allContainers, this);
- DataNodeChillModeRule dataNodeChillModeRule =
- new DataNodeChillModeRule(DN_EXIT_RULE, eventQueue, config, this);
- exitRules.put(CONT_EXIT_RULE, containerChillModeRule);
- exitRules.put(DN_EXIT_RULE, dataNodeChillModeRule);
+ DataNodeSafeModeRule dataNodeSafeModeRule =
+ new DataNodeSafeModeRule(DN_EXIT_RULE, eventQueue, config, this);
+ exitRules.put(CONT_EXIT_RULE, containerSafeModeRule);
+ exitRules.put(DN_EXIT_RULE, dataNodeSafeModeRule);
if (conf.getBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK_DEFAULT)
&& pipelineManager != null) {
- HealthyPipelineChillModeRule healthyPipelineChillModeRule =
- new HealthyPipelineChillModeRule(HEALTHY_PIPELINE_EXIT_RULE,
+ HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
+ new HealthyPipelineSafeModeRule(HEALTHY_PIPELINE_EXIT_RULE,
eventQueue, pipelineManager,
this, config);
- OneReplicaPipelineChillModeRule oneReplicaPipelineChillModeRule =
- new OneReplicaPipelineChillModeRule(
+ OneReplicaPipelineSafeModeRule oneReplicaPipelineSafeModeRule =
+ new OneReplicaPipelineSafeModeRule(
ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE, eventQueue,
pipelineManager, this, conf);
- exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, healthyPipelineChillModeRule);
+ exitRules.put(HEALTHY_PIPELINE_EXIT_RULE, healthyPipelineSafeModeRule);
exitRules.put(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE,
- oneReplicaPipelineChillModeRule);
+ oneReplicaPipelineSafeModeRule);
}
- emitChillModeStatus();
+ emitSafeModeStatus();
} else {
- exitChillMode(eventQueue);
+ exitSafeMode(eventQueue);
}
}
/**
- * Emit Chill mode status.
+ * Emit Safe mode status.
*/
@VisibleForTesting
- public void emitChillModeStatus() {
- eventPublisher.fireEvent(SCMEvents.CHILL_MODE_STATUS,
- new ChillModeStatus(getInChillMode()));
+ public void emitSafeModeStatus() {
+ eventPublisher.fireEvent(SCMEvents.SAFE_MODE_STATUS,
+ new SafeModeStatus(getInSafeMode()));
}
- public synchronized void validateChillModeExitRules(String ruleName,
+ public synchronized void validateSafeModeExitRules(String ruleName,
EventPublisher eventQueue) {
if (exitRules.get(ruleName) != null) {
@@ -158,47 +158,47 @@ public class SCMChillModeManager {
if (validatedRules.size() == exitRules.size()) {
- // All rules are satisfied, we can exit chill mode.
- LOG.info("ScmChillModeManager, all rules are successfully validated");
- exitChillMode(eventQueue);
+ // All rules are satisfied, we can exit safe mode.
+ LOG.info("ScmSafeModeManager, all rules are successfully validated");
+ exitSafeMode(eventQueue);
}
}
/**
- * Exit chill mode. It does following actions:
- * 1. Set chill mode status to false.
+ * Exit safe mode. It does following actions:
+ * 1. Set safe mode status to false.
* 2. Emits START_REPLICATION for ReplicationManager.
* 3. Cleanup resources.
- * 4. Emit chill mode status.
+ * 4. Emit safe mode status.
* @param eventQueue
*/
@VisibleForTesting
- public void exitChillMode(EventPublisher eventQueue) {
- LOG.info("SCM exiting chill mode.");
- setInChillMode(false);
+ public void exitSafeMode(EventPublisher eventQueue) {
+ LOG.info("SCM exiting safe mode.");
+ setInSafeMode(false);
// TODO: Remove handler registration as there is no need to listen to
// register events anymore.
- emitChillModeStatus();
- // TODO: #CLUTIL if we reenter chill mode the fixed interval pipeline
+ emitSafeModeStatus();
+ // TODO: #CLUTIL if we reenter safe mode the fixed interval pipeline
// creation job needs to stop
pipelineManager.startPipelineCreator();
}
- public boolean getInChillMode() {
- if (!isChillModeEnabled) {
+ public boolean getInSafeMode() {
+ if (!isSafeModeEnabled) {
return false;
}
- return inChillMode.get();
+ return inSafeMode.get();
}
/**
- * Set chill mode status.
+ * Set safe mode status.
*/
- public void setInChillMode(boolean inChillMode) {
- this.inChillMode.set(inChillMode);
+ public void setInSafeMode(boolean inSafeMode) {
+ this.inSafeMode.set(inSafeMode);
}
public static Logger getLogger() {
@@ -207,35 +207,35 @@ public class SCMChillModeManager {
@VisibleForTesting
public double getCurrentContainerThreshold() {
- return ((ContainerChillModeRule) exitRules.get(CONT_EXIT_RULE))
+ return ((ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE))
.getCurrentContainerThreshold();
}
@VisibleForTesting
- public HealthyPipelineChillModeRule getHealthyPipelineChillModeRule() {
- return (HealthyPipelineChillModeRule)
+ public HealthyPipelineSafeModeRule getHealthyPipelineSafeModeRule() {
+ return (HealthyPipelineSafeModeRule)
exitRules.get(HEALTHY_PIPELINE_EXIT_RULE);
}
@VisibleForTesting
- public OneReplicaPipelineChillModeRule getOneReplicaPipelineChillModeRule() {
- return (OneReplicaPipelineChillModeRule)
+ public OneReplicaPipelineSafeModeRule getOneReplicaPipelineSafeModeRule() {
+ return (OneReplicaPipelineSafeModeRule)
exitRules.get(ATLEAST_ONE_DATANODE_REPORTED_PIPELINE_EXIT_RULE);
}
/**
- * Class used during ChillMode status event.
+ * Class used during SafeMode status event.
*/
- public static class ChillModeStatus {
+ public static class SafeModeStatus {
- private boolean chillModeStatus;
- public ChillModeStatus(boolean chillModeState) {
- this.chillModeStatus = chillModeState;
+ private boolean safeModeStatus;
+ public SafeModeStatus(boolean safeModeState) {
+ this.safeModeStatus = safeModeState;
}
- public boolean getChillModeStatus() {
- return chillModeStatus;
+ public boolean getSafeModeStatus() {
+ return safeModeStatus;
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeExitRule.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
similarity index 70%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeExitRule.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
index 0c9b823..eceb8a3 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeExitRule.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeExitRule.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
@@ -23,31 +23,31 @@ import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.hdds.server.events.TypedEvent;
/**
- * Abstract class for ChillModeExitRules. When a new rule is added, the new
+ * Abstract class for SafeModeExitRules. When a new rule is added, the new
* rule should extend this abstract class.
*
* Each rule Should do:
* 1. Should add a handler for the event it is looking for during the
* initialization of the rule.
- * 2. Add the rule in ScmChillModeManager to list of the rules.
+ * 2. Add the rule in ScmSafeModeManager to list of the rules.
*
*
* @param <T>
*/
-public abstract class ChillModeExitRule<T> implements EventHandler<T> {
+public abstract class SafeModeExitRule<T> implements EventHandler<T> {
- private final SCMChillModeManager chillModeManager;
+ private final SCMSafeModeManager safeModeManager;
private final String ruleName;
- public ChillModeExitRule(SCMChillModeManager chillModeManager,
+ public SafeModeExitRule(SCMSafeModeManager safeModeManager,
String ruleName, EventQueue eventQueue) {
- this.chillModeManager = chillModeManager;
+ this.safeModeManager = safeModeManager;
this.ruleName = ruleName;
eventQueue.addHandler(getEventType(), this);
}
/**
- * Return's the name of this ChillModeExit Rule.
+ * Return's the name of this SafeModeExit Rule.
* @return ruleName
*/
public String getRuleName() {
@@ -55,7 +55,7 @@ public abstract class ChillModeExitRule<T> implements EventHandler<T> {
}
/**
- * Return's the event type this chillMode exit rule handles.
+ * Return's the event type this safeMode exit rule handles.
* @return TypedEvent
*/
protected abstract TypedEvent<T> getEventType();
@@ -81,11 +81,11 @@ public abstract class ChillModeExitRule<T> implements EventHandler<T> {
@Override
public final void onMessage(T report, EventPublisher publisher) {
- // TODO: when we have remove handlers, we can remove getInChillmode check
+ // TODO: when we have remove handlers, we can remove getInSafemode check
- if (scmInChillMode()) {
+ if (scmInSafeMode()) {
if (validate()) {
- chillModeManager.validateChillModeExitRules(ruleName, publisher);
+ safeModeManager.validateSafeModeExitRules(ruleName, publisher);
cleanup();
return;
}
@@ -93,18 +93,18 @@ public abstract class ChillModeExitRule<T> implements EventHandler<T> {
process(report);
if (validate()) {
- chillModeManager.validateChillModeExitRules(ruleName, publisher);
+ safeModeManager.validateSafeModeExitRules(ruleName, publisher);
cleanup();
}
}
}
/**
- * Return true if SCM is in chill mode, else false.
+ * Return true if SCM is in safe mode, else false.
* @return boolean
*/
- protected boolean scmInChillMode() {
- return chillModeManager.getInChillMode();
+ protected boolean scmInSafeMode() {
+ return safeModeManager.getInSafeMode();
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeHandler.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java
similarity index 72%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeHandler.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java
index fcad6e5..b9e5333 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeHandler.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java
@@ -15,7 +15,7 @@
* the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.scm.container.ReplicationManager;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.slf4j.Logger;
@@ -37,30 +37,30 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
/**
- * Class to handle the activities needed to be performed after exiting chill
+ * Class to handle the activities needed to be performed after exiting safe
* mode.
*/
-public class ChillModeHandler implements EventHandler<ChillModeStatus> {
+public class SafeModeHandler implements EventHandler<SafeModeStatus> {
private static final Logger LOG =
- LoggerFactory.getLogger(ChillModeHandler.class);
+ LoggerFactory.getLogger(SafeModeHandler.class);
private final SCMClientProtocolServer scmClientProtocolServer;
private final BlockManager scmBlockManager;
private final long waitTime;
- private final AtomicBoolean isInChillMode = new AtomicBoolean(true);
+ private final AtomicBoolean isInSafeMode = new AtomicBoolean(true);
private final ReplicationManager replicationManager;
private final PipelineManager scmPipelineManager;
/**
- * ChillModeHandler, to handle the logic once we exit chill mode.
+ * SafeModeHandler, to handle the logic once we exit safe mode.
* @param configuration
* @param clientProtocolServer
* @param blockManager
* @param replicationManager
*/
- public ChillModeHandler(Configuration configuration,
+ public SafeModeHandler(Configuration configuration,
SCMClientProtocolServer clientProtocolServer,
BlockManager blockManager,
ReplicationManager replicationManager, PipelineManager pipelineManager) {
@@ -73,43 +73,43 @@ public class ChillModeHandler implements EventHandler<ChillModeStatus> {
Objects.requireNonNull(pipelineManager, "PipelineManager object cannot " +
"be" + "null");
this.waitTime = configuration.getTimeDuration(
- HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT,
- HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT_DEFAULT,
+ HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT,
+ HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT_DEFAULT,
TimeUnit.MILLISECONDS);
this.scmClientProtocolServer = clientProtocolServer;
this.scmBlockManager = blockManager;
this.replicationManager = replicationManager;
this.scmPipelineManager = pipelineManager;
- final boolean chillModeEnabled = configuration.getBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
- isInChillMode.set(chillModeEnabled);
+ final boolean safeModeEnabled = configuration.getBoolean(
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT);
+ isInSafeMode.set(safeModeEnabled);
}
/**
- * Set ChillMode status based on
- * {@link org.apache.hadoop.hdds.scm.events.SCMEvents#CHILL_MODE_STATUS}.
+ * Set SafeMode status based on
+ * {@link org.apache.hadoop.hdds.scm.events.SCMEvents#SAFE_MODE_STATUS}.
*
* Inform BlockManager, ScmClientProtocolServer and replicationAcitivity
- * status about chillMode status.
+ * status about safeMode status.
*
- * @param chillModeStatus
+ * @param safeModeStatus
* @param publisher
*/
@Override
- public void onMessage(ChillModeStatus chillModeStatus,
+ public void onMessage(SafeModeStatus safeModeStatus,
EventPublisher publisher) {
- isInChillMode.set(chillModeStatus.getChillModeStatus());
- scmClientProtocolServer.setChillModeStatus(isInChillMode.get());
- scmBlockManager.setChillModeStatus(isInChillMode.get());
+ isInSafeMode.set(safeModeStatus.getSafeModeStatus());
+ scmClientProtocolServer.setSafeModeStatus(isInSafeMode.get());
+ scmBlockManager.setSafeModeStatus(isInSafeMode.get());
- if (!isInChillMode.get()) {
- final Thread chillModeExitThread = new Thread(() -> {
+ if (!isInSafeMode.get()) {
+ final Thread safeModeExitThread = new Thread(() -> {
try {
Thread.sleep(waitTime);
} catch (InterruptedException e) {
@@ -119,8 +119,8 @@ public class ChillModeHandler implements EventHandler<ChillModeStatus> {
cleanupPipelines();
});
- chillModeExitThread.setDaemon(true);
- chillModeExitThread.start();
+ safeModeExitThread.setDaemon(true);
+ safeModeExitThread.start();
}
}
@@ -139,8 +139,8 @@ public class ChillModeHandler implements EventHandler<ChillModeStatus> {
});
}
- public boolean getChillModeStatus() {
- return isInChillMode.get();
+ public boolean getSafeModeStatus() {
+ return isInSafeMode.get();
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModePrecheck.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java
similarity index 58%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModePrecheck.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java
index c6367c2..b63d04e 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModePrecheck.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.hadoop.conf.Configuration;
@@ -26,32 +26,32 @@ import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
/**
- * Chill mode pre-check for SCM operations.
+ * Safe mode pre-check for SCM operations.
* */
-public class ChillModePrecheck implements Precheck<ScmOps> {
+public class SafeModePrecheck implements Precheck<ScmOps> {
- private AtomicBoolean inChillMode;
- public static final String PRECHECK_TYPE = "ChillModePrecheck";
+ private AtomicBoolean inSafeMode;
+ public static final String PRECHECK_TYPE = "SafeModePrecheck";
- public ChillModePrecheck(Configuration conf) {
- boolean chillModeEnabled = conf.getBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
- if (chillModeEnabled) {
- inChillMode = new AtomicBoolean(true);
+ public SafeModePrecheck(Configuration conf) {
+ boolean safeModeEnabled = conf.getBoolean(
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT);
+ if (safeModeEnabled) {
+ inSafeMode = new AtomicBoolean(true);
} else {
- inChillMode = new AtomicBoolean(false);
+ inSafeMode = new AtomicBoolean(false);
}
}
@Override
public boolean check(ScmOps op) throws SCMException {
- if (inChillMode.get() && ChillModeRestrictedOps
- .isRestrictedInChillMode(op)) {
- throw new SCMException("ChillModePrecheck failed for " + op,
- ResultCodes.CHILL_MODE_EXCEPTION);
+ if (inSafeMode.get() && SafeModeRestrictedOps
+ .isRestrictedInSafeMode(op)) {
+ throw new SCMException("SafeModePrecheck failed for " + op,
+ ResultCodes.SAFE_MODE_EXCEPTION);
}
- return inChillMode.get();
+ return inSafeMode.get();
}
@Override
@@ -59,11 +59,11 @@ public class ChillModePrecheck implements Precheck<ScmOps> {
return PRECHECK_TYPE;
}
- public boolean isInChillMode() {
- return inChillMode.get();
+ public boolean isInSafeMode() {
+ return inSafeMode.get();
}
- public void setInChillMode(boolean inChillMode) {
- this.inChillMode.set(inChillMode);
+ public void setInSafeMode(boolean inSafeMode) {
+ this.inSafeMode.set(inSafeMode);
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeRestrictedOps.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java
similarity index 82%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeRestrictedOps.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java
index f3b003b..5f516e4 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/ChillModeRestrictedOps.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java
@@ -15,19 +15,19 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import java.util.EnumSet;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
/**
- * Operations restricted in SCM chill mode.
+ * Operations restricted in SCM safe mode.
*/
-public final class ChillModeRestrictedOps {
+public final class SafeModeRestrictedOps {
private static EnumSet restrictedOps = EnumSet.noneOf(ScmOps.class);
- private ChillModeRestrictedOps() {
+ private SafeModeRestrictedOps() {
}
static {
@@ -35,7 +35,7 @@ public final class ChillModeRestrictedOps {
restrictedOps.add(ScmOps.allocateContainer);
}
- public static boolean isRestrictedInChillMode(ScmOps opName) {
+ public static boolean isRestrictedInSafeMode(ScmOps opName) {
return restrictedOps.contains(opName);
}
}
\ No newline at end of file
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
similarity index 94%
rename from hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
rename to hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
index 2513ac7..b5fd826 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
@@ -15,4 +15,4 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
index b8f89a3..a14d003 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMBlockProtocolServer.java
@@ -223,9 +223,9 @@ public class SCMBlockProtocolServer implements
scmEx)
);
switch (scmEx.getResult()) {
- case CHILL_MODE_EXCEPTION:
+ case SAFE_MODE_EXCEPTION:
resultCode = ScmBlockLocationProtocolProtos.DeleteScmBlockResult
- .Result.chillMode;
+ .Result.safeMode;
break;
case FAILED_TO_FIND_BLOCK:
resultCode = ScmBlockLocationProtocolProtos.DeleteScmBlockResult
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
index 8330f70..6a875e6 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.ScmUtils;
-import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
+import org.apache.hadoop.hdds.scm.safemode.SafeModePrecheck;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
@@ -101,13 +101,13 @@ public class SCMClientProtocolServer implements
private final InetSocketAddress clientRpcAddress;
private final StorageContainerManager scm;
private final OzoneConfiguration conf;
- private ChillModePrecheck chillModePrecheck;
+ private SafeModePrecheck safeModePrecheck;
public SCMClientProtocolServer(OzoneConfiguration conf,
StorageContainerManager scm) throws IOException {
this.scm = scm;
this.conf = conf;
- chillModePrecheck = new ChillModePrecheck(conf);
+ safeModePrecheck = new SafeModePrecheck(conf);
final int handlerCount =
conf.getInt(OZONE_SCM_HANDLER_COUNT_KEY,
OZONE_SCM_HANDLER_COUNT_DEFAULT);
@@ -177,7 +177,7 @@ public class SCMClientProtocolServer implements
public ContainerWithPipeline allocateContainer(HddsProtos.ReplicationType
replicationType, HddsProtos.ReplicationFactor factor,
String owner) throws IOException {
- ScmUtils.preCheck(ScmOps.allocateContainer, chillModePrecheck);
+ ScmUtils.preCheck(ScmOps.allocateContainer, safeModePrecheck);
getScm().checkAdminAccess(getRpcRemoteUsername());
final ContainerInfo container = scm.getContainerManager()
@@ -220,14 +220,14 @@ public class SCMClientProtocolServer implements
auditMap.put("containerID", String.valueOf(containerID));
boolean auditSuccess = true;
try {
- if (chillModePrecheck.isInChillMode()) {
+ if (safeModePrecheck.isInSafeMode()) {
ContainerInfo contInfo = scm.getContainerManager()
.getContainer(ContainerID.valueof(containerID));
if (contInfo.isOpen()) {
if (!hasRequiredReplicas(contInfo)) {
throw new SCMException("Open container " + containerID + " doesn't"
+ " have enough replicas to service this operation in "
- + "Chill mode.", ResultCodes.CHILL_MODE_EXCEPTION);
+ + "Safe mode.", ResultCodes.SAFE_MODE_EXCEPTION);
}
}
}
@@ -446,31 +446,31 @@ public class SCMClientProtocolServer implements
}
/**
- * Check if SCM is in chill mode.
+ * Check if SCM is in safe mode.
*
- * @return Returns true if SCM is in chill mode else returns false.
+ * @return Returns true if SCM is in safe mode else returns false.
* @throws IOException
*/
@Override
- public boolean inChillMode() throws IOException {
+ public boolean inSafeMode() throws IOException {
AUDIT.logReadSuccess(
- buildAuditMessageForSuccess(SCMAction.IN_CHILL_MODE, null)
+ buildAuditMessageForSuccess(SCMAction.IN_SAFE_MODE, null)
);
- return scm.isInChillMode();
+ return scm.isInSafeMode();
}
/**
- * Force SCM out of Chill mode.
+ * Force SCM out of Safe mode.
*
* @return returns true if operation is successful.
* @throws IOException
*/
@Override
- public boolean forceExitChillMode() throws IOException {
+ public boolean forceExitSafeMode() throws IOException {
AUDIT.logWriteSuccess(
- buildAuditMessageForSuccess(SCMAction.FORCE_EXIT_CHILL_MODE, null)
+ buildAuditMessageForSuccess(SCMAction.FORCE_EXIT_SAFE_MODE, null)
);
- return scm.exitChillMode();
+ return scm.exitSafeMode();
}
/**
@@ -498,10 +498,10 @@ public class SCMClientProtocolServer implements
}
/**
- * Set chill mode status based on .
+ * Set safe mode status based on .
*/
- public boolean getChillModeStatus() {
- return chillModePrecheck.isInChillMode();
+ public boolean getSafeModeStatus() {
+ return safeModePrecheck.isInSafeMode();
}
@@ -556,11 +556,11 @@ public class SCMClientProtocolServer implements
}
/**
- * Set ChillMode status.
+ * Set SafeMode status.
*
- * @param chillModeStatus
+ * @param safeModeStatus
*/
- public void setChillModeStatus(boolean chillModeStatus) {
- chillModePrecheck.setInChillMode(chillModeStatus);
+ public void setSafeModeStatus(boolean safeModeStatus) {
+ safeModePrecheck.setInSafeMode(safeModeStatus);
}
}
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMConfigurator.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMConfigurator.java
index a6b0704..b45ce6f 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMConfigurator.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMConfigurator.java
@@ -21,7 +21,7 @@ package org.apache.hadoop.hdds.scm.server;
import org.apache.hadoop.hdds.scm.block.BlockManager;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ReplicationManager;
import org.apache.hadoop.hdds.scm.metadata.SCMMetadataStore;
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdds.security.x509.certificate.authority
* ContainerManager containerManager;
* BlockManager scmBlockManager;
* ReplicationManager replicationManager;
- * SCMChillModeManager scmChillModeManager;
+ * SCMSafeModeManager scmSafeModeManager;
* CertificateServer certificateServer;
* SCMMetadata scmMetadataStore.
*
@@ -62,7 +62,7 @@ public final class SCMConfigurator {
private ContainerManager containerManager;
private BlockManager scmBlockManager;
private ReplicationManager replicationManager;
- private SCMChillModeManager scmChillModeManager;
+ private SCMSafeModeManager scmSafeModeManager;
private CertificateServer certificateServer;
private SCMMetadataStore metadataStore;
@@ -111,12 +111,12 @@ public final class SCMConfigurator {
}
/**
- * Allows user to specify a custom version of Chill Mode Manager to use
+ * Allows user to specify a custom version of Safe Mode Manager to use
* with this SCM.
- * @param scmChillModeManager - ChillMode Manager.
+ * @param scmSafeModeManager - SafeMode Manager.
*/
- public void setScmChillModeManager(SCMChillModeManager scmChillModeManager) {
- this.scmChillModeManager = scmChillModeManager;
+ public void setScmSafeModeManager(SCMSafeModeManager scmSafeModeManager) {
+ this.scmSafeModeManager = scmSafeModeManager;
}
/**
@@ -178,11 +178,11 @@ public final class SCMConfigurator {
}
/**
- * Gets Chill Mode Manager.
- * @return Chill Mode manager.
+ * Gets Safe Mode Manager.
+ * @return Safe Mode manager.
*/
- public SCMChillModeManager getScmChillModeManager() {
- return scmChillModeManager;
+ public SCMSafeModeManager getScmSafeModeManager() {
+ return scmSafeModeManager;
}
/**
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMMXBean.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMMXBean.java
index dc09ceb..13b5551 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMMXBean.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMMXBean.java
@@ -49,16 +49,16 @@ public interface SCMMXBean extends ServiceRuntimeInfo {
Map<String, String> getContainerReport();
/**
- * Returns chill mode status.
+ * Returns safe mode status.
* @return boolean
*/
- boolean isInChillMode();
+ boolean isInSafeMode();
/**
- * Returns live chill mode container threshold.
+ * Returns live safe mode container threshold.
* @return String
*/
- double getChillModeCurrentContainerThreshold();
+ double getSafeModeCurrentContainerThreshold();
/**
* Returns the container count in all states.
diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
index 7ddc08f..100534a 100644
--- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
+++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java
@@ -40,8 +40,8 @@ import org.apache.hadoop.hdds.scm.block.BlockManager;
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
import org.apache.hadoop.hdds.scm.block.DeletedBlockLogImpl;
import org.apache.hadoop.hdds.scm.block.PendingDeleteHandler;
-import org.apache.hadoop.hdds.scm.chillmode.ChillModeHandler;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SafeModeHandler;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
@@ -196,12 +196,12 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
private final LeaseManager<Long> commandWatcherLeaseManager;
- private SCMChillModeManager scmChillModeManager;
+ private SCMSafeModeManager scmSafeModeManager;
private CertificateServer certificateServer;
private JvmPauseMonitor jvmPauseMonitor;
private final OzoneConfiguration configuration;
- private final ChillModeHandler chillModeHandler;
+ private final SafeModeHandler safeModeHandler;
private SCMContainerMetrics scmContainerMetrics;
/**
@@ -289,7 +289,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
NodeReportHandler nodeReportHandler =
new NodeReportHandler(scmNodeManager);
PipelineReportHandler pipelineReportHandler =
- new PipelineReportHandler(scmChillModeManager, pipelineManager, conf);
+ new PipelineReportHandler(scmSafeModeManager, pipelineManager, conf);
CommandStatusReportHandler cmdStatusReportHandler =
new CommandStatusReportHandler();
@@ -334,7 +334,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
clientProtocolServer = new SCMClientProtocolServer(conf, this);
httpServer = new StorageContainerManagerHttpServer(conf);
- chillModeHandler = new ChillModeHandler(configuration,
+ safeModeHandler = new SafeModeHandler(configuration,
clientProtocolServer, scmBlockManager, replicationManager,
pipelineManager);
@@ -358,7 +358,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
(DeletedBlockLogImpl) scmBlockManager.getDeletedBlockLog());
eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS, pipelineActionHandler);
eventQueue.addHandler(SCMEvents.PIPELINE_REPORT, pipelineReportHandler);
- eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, chillModeHandler);
+ eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, safeModeHandler);
registerMXBean();
registerMetricsSource(this);
}
@@ -372,7 +372,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
* Container Manager
* Block Manager
* Replication Manager
- * Chill Mode Manager
+ * Safe Mode Manager
*
* @param conf - Ozone Configuration.
* @param configurator - A customizer which allows different managers to be
@@ -418,10 +418,10 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
replicationManager = new ReplicationManager(conf,
containerManager, containerPlacementPolicy, eventQueue);
}
- if(configurator.getScmChillModeManager() != null) {
- scmChillModeManager = configurator.getScmChillModeManager();
+ if(configurator.getScmSafeModeManager() != null) {
+ scmSafeModeManager = configurator.getScmSafeModeManager();
} else {
- scmChillModeManager = new SCMChillModeManager(conf,
+ scmSafeModeManager = new SCMSafeModeManager(conf,
containerManager.getContainers(), pipelineManager, eventQueue);
}
}
@@ -1074,13 +1074,13 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
}
@VisibleForTesting
- public ChillModeHandler getChillModeHandler() {
- return chillModeHandler;
+ public SafeModeHandler getSafeModeHandler() {
+ return safeModeHandler;
}
@VisibleForTesting
- public SCMChillModeManager getScmChillModeManager() {
- return scmChillModeManager;
+ public SCMSafeModeManager getScmSafeModeManager() {
+ return scmSafeModeManager;
}
@VisibleForTesting
@@ -1149,22 +1149,22 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
}
/**
- * Returns live chill mode container threshold.
+ * Returns live safe mode container threshold.
*
* @return String
*/
@Override
- public double getChillModeCurrentContainerThreshold() {
+ public double getSafeModeCurrentContainerThreshold() {
return getCurrentContainerThreshold();
}
/**
- * Returns chill mode status.
+ * Returns safe mode status.
* @return boolean
*/
@Override
- public boolean isInChillMode() {
- return scmChillModeManager.getInChillMode();
+ public boolean isInSafeMode() {
+ return scmSafeModeManager.getInSafeMode();
}
/**
@@ -1175,16 +1175,16 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
}
/**
- * Force SCM out of chill mode.
+ * Force SCM out of safe mode.
*/
- public boolean exitChillMode() {
- scmChillModeManager.exitChillMode(eventQueue);
+ public boolean exitSafeMode() {
+ scmSafeModeManager.exitSafeMode(eventQueue);
return true;
}
@VisibleForTesting
public double getCurrentContainerThreshold() {
- return scmChillModeManager.getCurrentContainerThreshold();
+ return scmSafeModeManager.getCurrentContainerThreshold();
}
@Override
diff --git a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html
index fedfcdf..38ce638 100644
--- a/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html
+++ b/hadoop-hdds/server-scm/src/main/resources/webapps/scm/scm-overview.html
@@ -37,8 +37,8 @@
<td>{{$ctrl.overview.jmx.DatanodeRpcPort}}</td>
</tr>
<tr>
- <td>Node Manager: Chill mode status</td>
- <td>{{$ctrl.scmmetrics.InChillMode}}</td>
+ <td>Node Manager: Safe mode status</td>
+ <td>{{$ctrl.scmmetrics.InSafeMode}}</td>
</tr>
</tbody>
</table>
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
index 2e0af7b..afa7fcb 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/block/TestBlockManager.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager.ChillModeStatus;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager.SafeModeStatus;
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
@@ -71,7 +71,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
private static EventQueue eventQueue;
private int numContainerPerOwnerInPipeline;
private OzoneConfiguration conf;
- private ChillModeStatus chillModeStatus = new ChillModeStatus(false);
+ private SafeModeStatus safeModeStatus = new SafeModeStatus(false);
@Rule
public ExpectedException thrown = ExpectedException.none();
@@ -101,8 +101,8 @@ public class TestBlockManager implements EventHandler<Boolean> {
blockManager = (BlockManagerImpl) scm.getScmBlockManager();
eventQueue = new EventQueue();
- eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS,
- scm.getChillModeHandler());
+ eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS,
+ scm.getSafeModeHandler());
eventQueue.addHandler(SCMEvents.START_REPLICATION, this);
CloseContainerEventHandler closeContainerHandler =
new CloseContainerEventHandler(pipelineManager, mapping);
@@ -124,9 +124,9 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test
public void testAllocateBlock() throws Exception {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils.waitFor(() -> {
- return !blockManager.isScmInChillMode();
+ return !blockManager.isScmInSafeMode();
}, 10, 1000 * 5);
AllocatedBlock block = blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, containerOwner, new ExcludeList());
@@ -135,9 +135,9 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test
public void testAllocateOversizedBlock() throws Exception {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils.waitFor(() -> {
- return !blockManager.isScmInChillMode();
+ return !blockManager.isScmInSafeMode();
}, 10, 1000 * 5);
long size = 6 * GB;
thrown.expectMessage("Unsupported block size");
@@ -147,25 +147,25 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test
- public void testAllocateBlockFailureInChillMode() throws Exception {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS,
- new ChillModeStatus(true));
+ public void testAllocateBlockFailureInSafeMode() throws Exception {
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS,
+ new SafeModeStatus(true));
GenericTestUtils.waitFor(() -> {
- return blockManager.isScmInChillMode();
+ return blockManager.isScmInSafeMode();
}, 10, 1000 * 5);
- // Test1: In chill mode expect an SCMException.
- thrown.expectMessage("ChillModePrecheck failed for "
+ // Test1: In safe mode expect an SCMException.
+ thrown.expectMessage("SafeModePrecheck failed for "
+ "allocateBlock");
blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, containerOwner, new ExcludeList());
}
@Test
- public void testAllocateBlockSucInChillMode() throws Exception {
- // Test2: Exit chill mode and then try allocateBock again.
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ public void testAllocateBlockSucInSafeMode() throws Exception {
+ // Test2: Exit safe mode and then try allocateBock again.
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils.waitFor(() -> {
- return !blockManager.isScmInChillMode();
+ return !blockManager.isScmInSafeMode();
}, 10, 1000 * 5);
Assert.assertNotNull(blockManager.allocateBlock(DEFAULT_BLOCK_SIZE,
type, factor, containerOwner, new ExcludeList()));
@@ -174,9 +174,9 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test(timeout = 10000)
public void testMultipleBlockAllocation()
throws IOException, TimeoutException, InterruptedException {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils
- .waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
+ .waitFor(() -> !blockManager.isScmInSafeMode(), 10, 1000 * 5);
pipelineManager.createPipeline(type, factor);
pipelineManager.createPipeline(type, factor);
@@ -216,9 +216,9 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test(timeout = 10000)
public void testMultipleBlockAllocationWithClosedContainer()
throws IOException, TimeoutException, InterruptedException {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils
- .waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
+ .waitFor(() -> !blockManager.isScmInSafeMode(), 10, 1000 * 5);
// create pipelines
for (int i = 0;
@@ -268,9 +268,9 @@ public class TestBlockManager implements EventHandler<Boolean> {
@Test(timeout = 10000)
public void testBlockAllocationWithNoAvailablePipelines()
throws IOException, TimeoutException, InterruptedException {
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
GenericTestUtils
- .waitFor(() -> !blockManager.isScmInChillMode(), 10, 1000 * 5);
+ .waitFor(() -> !blockManager.isScmInSafeMode(), 10, 1000 * 5);
for (Pipeline pipeline : pipelineManager.getPipelines()) {
pipelineManager.finalizeAndDestroyPipeline(pipeline, false);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
index ffdea0e..129644e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/MockNodeManager.java
@@ -78,7 +78,7 @@ public class MockNodeManager implements NodeManager {
private final List<DatanodeDetails> deadNodes;
private final Map<DatanodeDetails, SCMNodeStat> nodeMetricMap;
private final SCMNodeStat aggregateStat;
- private boolean chillmode;
+ private boolean safemode;
private final Map<UUID, List<SCMCommand>> commandMap;
private final Node2PipelineMap node2PipelineMap;
private final Node2ContainerMap node2ContainerMap;
@@ -98,7 +98,7 @@ public class MockNodeManager implements NodeManager {
populateNodeMetric(dd, x);
}
}
- chillmode = false;
+ safemode = false;
this.commandMap = new HashMap<>();
}
@@ -132,11 +132,11 @@ public class MockNodeManager implements NodeManager {
}
/**
- * Sets the chill mode value.
- * @param chillmode boolean
+ * Sets the safe mode value.
+ * @param safemode boolean
*/
- public void setChillmode(boolean chillmode) {
- this.chillmode = chillmode;
+ public void setSafemode(boolean safemode) {
+ this.safemode = safemode;
}
/**
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java
index 91a36b7..a562efe 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/TestSCMContainerManager.java
@@ -107,8 +107,8 @@ public class TestSCMContainerManager {
}
@Before
- public void clearChillMode() {
- nodeManager.setChillmode(false);
+ public void clearSafeMode() {
+ nodeManager.setSafemode(false);
}
@Test
@@ -266,7 +266,7 @@ public class TestSCMContainerManager {
*/
private ContainerInfo createContainer()
throws IOException {
- nodeManager.setChillmode(false);
+ nodeManager.setSafemode(false);
return containerManager
.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
index 1e9e174..204c860 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/node/TestSCMNodeManager.java
@@ -132,7 +132,7 @@ public class TestSCMNodeManager {
/**
* Tests that Node manager handles heartbeats correctly, and comes out of
- * chill Mode.
+ * safe Mode.
*
* @throws IOException
* @throws InterruptedException
@@ -160,7 +160,7 @@ public class TestSCMNodeManager {
}
/**
- * asserts that if we send no heartbeats node manager stays in chillmode.
+ * asserts that if we send no heartbeats node manager stays in safemode.
*
* @throws IOException
* @throws InterruptedException
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestHealthyPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java
similarity index 81%
rename from hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestHealthyPipelineChillModeRule.java
rename to hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java
index 3ee7a7c..eb1f88b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestHealthyPipelineChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestHealthyPipelineSafeModeRule.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.HddsConfigKeys;
@@ -47,16 +47,16 @@ import java.util.List;
import java.util.UUID;
/**
- * This class tests HealthyPipelineChillMode rule.
+ * This class tests HealthyPipelineSafeMode rule.
*/
-public class TestHealthyPipelineChillModeRule {
+public class TestHealthyPipelineSafeModeRule {
@Test
- public void testHealthyPipelineChillModeRuleWithNoPipelines()
+ public void testHealthyPipelineSafeModeRuleWithNoPipelines()
throws Exception {
String storageDir = GenericTestUtils.getTempPath(
- TestHealthyPipelineChillModeRule.class.getName() + UUID.randomUUID());
+ TestHealthyPipelineSafeModeRule.class.getName() + UUID.randomUUID());
try {
EventQueue eventQueue = new EventQueue();
List<ContainerInfo> containers = new ArrayList<>();
@@ -67,7 +67,7 @@ public class TestHealthyPipelineChillModeRule {
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
// enable pipeline check
config.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true);
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
@@ -77,14 +77,14 @@ public class TestHealthyPipelineChillModeRule {
pipelineManager.getStateManager(), config);
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
mockRatisProvider);
- SCMChillModeManager scmChillModeManager = new SCMChillModeManager(
+ SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, pipelineManager, eventQueue);
- HealthyPipelineChillModeRule healthyPipelineChillModeRule =
- scmChillModeManager.getHealthyPipelineChillModeRule();
+ HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
+ scmSafeModeManager.getHealthyPipelineSafeModeRule();
// This should be immediately satisfied, as no pipelines are there yet.
- Assert.assertTrue(healthyPipelineChillModeRule.validate());
+ Assert.assertTrue(healthyPipelineSafeModeRule.validate());
} finally {
FileUtil.fullyDelete(new File(storageDir));
}
@@ -93,10 +93,10 @@ public class TestHealthyPipelineChillModeRule {
@Test
- public void testHealthyPipelineChillModeRuleWithPipelines() throws Exception {
+ public void testHealthyPipelineSafeModeRuleWithPipelines() throws Exception {
String storageDir = GenericTestUtils.getTempPath(
- TestHealthyPipelineChillModeRule.class.getName() + UUID.randomUUID());
+ TestHealthyPipelineSafeModeRule.class.getName() + UUID.randomUUID());
try {
EventQueue eventQueue = new EventQueue();
@@ -112,7 +112,7 @@ public class TestHealthyPipelineChillModeRule {
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
// enable pipeline check
config.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true);
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
@@ -136,15 +136,15 @@ public class TestHealthyPipelineChillModeRule {
HddsProtos.ReplicationFactor.THREE);
- SCMChillModeManager scmChillModeManager = new SCMChillModeManager(
+ SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, pipelineManager, eventQueue);
- HealthyPipelineChillModeRule healthyPipelineChillModeRule =
- scmChillModeManager.getHealthyPipelineChillModeRule();
+ HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
+ scmSafeModeManager.getHealthyPipelineSafeModeRule();
// No datanodes have sent pipelinereport from datanode
- Assert.assertFalse(healthyPipelineChillModeRule.validate());
+ Assert.assertFalse(healthyPipelineSafeModeRule.validate());
// Fire pipeline report from all datanodes in first pipeline, as here we
// have 3 pipelines, 10% is 0.3, when doing ceil it is 1. So, we should
@@ -157,7 +157,7 @@ public class TestHealthyPipelineChillModeRule {
// manager in open state for test case simplicity.
firePipelineEvent(pipeline1, eventQueue);
- GenericTestUtils.waitFor(() -> healthyPipelineChillModeRule.validate(),
+ GenericTestUtils.waitFor(() -> healthyPipelineSafeModeRule.validate(),
1000, 5000);
} finally {
FileUtil.fullyDelete(new File(storageDir));
@@ -167,11 +167,11 @@ public class TestHealthyPipelineChillModeRule {
@Test
- public void testHealthyPipelineChillModeRuleWithMixedPipelines()
+ public void testHealthyPipelineSafeModeRuleWithMixedPipelines()
throws Exception {
String storageDir = GenericTestUtils.getTempPath(
- TestHealthyPipelineChillModeRule.class.getName() + UUID.randomUUID());
+ TestHealthyPipelineSafeModeRule.class.getName() + UUID.randomUUID());
try {
EventQueue eventQueue = new EventQueue();
@@ -187,7 +187,7 @@ public class TestHealthyPipelineChillModeRule {
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
// enable pipeline check
config.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true);
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
@@ -210,20 +210,20 @@ public class TestHealthyPipelineChillModeRule {
HddsProtos.ReplicationFactor.THREE);
- SCMChillModeManager scmChillModeManager = new SCMChillModeManager(
+ SCMSafeModeManager scmSafeModeManager = new SCMSafeModeManager(
config, containers, pipelineManager, eventQueue);
- HealthyPipelineChillModeRule healthyPipelineChillModeRule =
- scmChillModeManager.getHealthyPipelineChillModeRule();
+ HealthyPipelineSafeModeRule healthyPipelineSafeModeRule =
+ scmSafeModeManager.getHealthyPipelineSafeModeRule();
// No datanodes have sent pipelinereport from datanode
- Assert.assertFalse(healthyPipelineChillModeRule.validate());
+ Assert.assertFalse(healthyPipelineSafeModeRule.validate());
GenericTestUtils.LogCapturer logCapturer =
GenericTestUtils.LogCapturer.captureLogs(LoggerFactory.getLogger(
- SCMChillModeManager.class));
+ SCMSafeModeManager.class));
// fire event with pipeline report with ratis type and factor 1
// pipeline, validate() should return false
@@ -232,12 +232,12 @@ public class TestHealthyPipelineChillModeRule {
GenericTestUtils.waitFor(() -> logCapturer.getOutput().contains(
"reported count is 0"),
1000, 5000);
- Assert.assertFalse(healthyPipelineChillModeRule.validate());
+ Assert.assertFalse(healthyPipelineSafeModeRule.validate());
firePipelineEvent(pipeline2, eventQueue);
firePipelineEvent(pipeline3, eventQueue);
- GenericTestUtils.waitFor(() -> healthyPipelineChillModeRule.validate(),
+ GenericTestUtils.waitFor(() -> healthyPipelineSafeModeRule.validate(),
1000, 5000);
} finally {
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java
similarity index 92%
rename from hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java
rename to hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java
index ac0d040..99677d6 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestOneReplicaPipelineChillModeRule.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestOneReplicaPipelineSafeModeRule.java
@@ -15,7 +15,7 @@
* the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -45,13 +45,13 @@ import java.util.ArrayList;
import java.util.List;
/**
- * This class tests OneReplicaPipelineChillModeRule.
+ * This class tests OneReplicaPipelineSafeModeRule.
*/
-public class TestOneReplicaPipelineChillModeRule {
+public class TestOneReplicaPipelineSafeModeRule {
@Rule
public TemporaryFolder folder = new TemporaryFolder();
- private OneReplicaPipelineChillModeRule rule;
+ private OneReplicaPipelineSafeModeRule rule;
private SCMPipelineManager pipelineManager;
private EventQueue eventQueue;
@@ -60,7 +60,7 @@ public class TestOneReplicaPipelineChillModeRule {
int pipelineFactorOneCount) throws Exception {
OzoneConfiguration ozoneConfiguration = new OzoneConfiguration();
ozoneConfiguration.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true);
ozoneConfiguration.set(HddsConfigKeys.OZONE_METADATA_DIRS,
folder.newFolder().toString());
@@ -84,11 +84,11 @@ public class TestOneReplicaPipelineChillModeRule {
createPipelines(pipelineFactorOneCount,
HddsProtos.ReplicationFactor.ONE);
- SCMChillModeManager scmChillModeManager =
- new SCMChillModeManager(ozoneConfiguration, containers,
+ SCMSafeModeManager scmSafeModeManager =
+ new SCMSafeModeManager(ozoneConfiguration, containers,
pipelineManager, eventQueue);
- rule = scmChillModeManager.getOneReplicaPipelineChillModeRule();
+ rule = scmSafeModeManager.getOneReplicaPipelineSafeModeRule();
}
@Test
@@ -104,7 +104,7 @@ public class TestOneReplicaPipelineChillModeRule {
GenericTestUtils.LogCapturer logCapturer =
GenericTestUtils.LogCapturer.captureLogs(
- LoggerFactory.getLogger(SCMChillModeManager.class));
+ LoggerFactory.getLogger(SCMSafeModeManager.class));
List<Pipeline> pipelines = pipelineManager.getPipelines();
for (int i = 0; i < pipelineFactorThreeCount -1; i++) {
@@ -141,7 +141,7 @@ public class TestOneReplicaPipelineChillModeRule {
GenericTestUtils.LogCapturer logCapturer =
GenericTestUtils.LogCapturer.captureLogs(
- LoggerFactory.getLogger(SCMChillModeManager.class));
+ LoggerFactory.getLogger(SCMSafeModeManager.class));
List<Pipeline> pipelines =
pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
similarity index 73%
rename from hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
rename to hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
index fda3e8e..fa582e3 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeManager.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeManager.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
@@ -52,12 +52,12 @@ import org.junit.rules.TemporaryFolder;
import org.junit.rules.Timeout;
import org.mockito.Mockito;
-/** Test class for SCMChillModeManager.
+/** Test class for SCMSafeModeManager.
*/
-public class TestSCMChillModeManager {
+public class TestSCMSafeModeManager {
private static EventQueue queue;
- private SCMChillModeManager scmChillModeManager;
+ private SCMSafeModeManager scmSafeModeManager;
private static Configuration config;
private List<ContainerInfo> containers;
@@ -74,62 +74,62 @@ public class TestSCMChillModeManager {
}
@Test
- public void testChillModeState() throws Exception {
+ public void testSafeModeState() throws Exception {
// Test 1: test for 0 containers
- testChillMode(0);
+ testSafeMode(0);
// Test 2: test for 20 containers
- testChillMode(20);
+ testSafeMode(20);
}
@Test
- public void testChillModeStateWithNullContainers() {
- new SCMChillModeManager(config, null, null, queue);
+ public void testSafeModeStateWithNullContainers() {
+ new SCMSafeModeManager(config, null, null, queue);
}
- private void testChillMode(int numContainers) throws Exception {
+ private void testSafeMode(int numContainers) throws Exception {
containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(numContainers));
- // Assign open state to containers to be included in the chill mode
+ // Assign open state to containers to be included in the safe mode
// container list
for (ContainerInfo container : containers) {
container.setState(HddsProtos.LifeCycleState.OPEN);
}
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, queue);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(containers));
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 100, 1000 * 5);
}
@Test
- public void testChillModeExitRule() throws Exception {
+ public void testSafeModeExitRule() throws Exception {
containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
- // Assign open state to containers to be included in the chill mode
+ // Assign open state to containers to be included in the safe mode
// container list
for (ContainerInfo container : containers) {
container.setState(HddsProtos.LifeCycleState.CLOSED);
}
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, queue);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers.subList(0, 25), 0.25);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers.subList(25, 50), 0.50);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers.subList(50, 75), 0.75);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers.subList(75, 100), 1.0);
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 100, 1000 * 5);
}
@@ -140,24 +140,24 @@ public class TestSCMChillModeManager {
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
tempDir.newFolder().toString());
conf.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
true);
conf.setDouble(HddsConfigKeys.
- HDDS_SCM_CHILLMODE_HEALTHY_PIPELINE_THRESHOLD_PCT, healthyPercent);
+ HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT, healthyPercent);
conf.setDouble(HddsConfigKeys.
- HDDS_SCM_CHILLMODE_ONE_NODE_REPORTED_PIPELINE_PCT, oneReplicaPercent);
+ HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT, oneReplicaPercent);
return conf;
}
@Test
- public void testChillModeExitRuleWithPipelineAvailabilityCheck()
+ public void testSafeModeExitRuleWithPipelineAvailabilityCheck()
throws Exception{
- testChillModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0.90, 1);
- testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0.10, 0.9);
- testChillModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0, 0.9);
- testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0);
- testChillModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0.5);
+ testSafeModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0.90, 1);
+ testSafeModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0.10, 0.9);
+ testSafeModeExitRuleWithPipelineAvailabilityCheck(100, 30, 8, 0, 0.9);
+ testSafeModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0);
+ testSafeModeExitRuleWithPipelineAvailabilityCheck(100, 90, 22, 0, 0.5);
}
@Test
@@ -169,7 +169,7 @@ public class TestSCMChillModeManager {
MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
PipelineManager pipelineManager = new SCMPipelineManager(conf,
mockNodeManager, queue);
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
conf, containers, pipelineManager, queue);
fail("testFailWithIncorrectValueForHealthyPipelinePercent");
} catch (IllegalArgumentException ex) {
@@ -187,7 +187,7 @@ public class TestSCMChillModeManager {
MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
PipelineManager pipelineManager = new SCMPipelineManager(conf,
mockNodeManager, queue);
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
conf, containers, pipelineManager, queue);
fail("testFailWithIncorrectValueForOneReplicaPipelinePercent");
} catch (IllegalArgumentException ex) {
@@ -197,16 +197,16 @@ public class TestSCMChillModeManager {
}
@Test
- public void testFailWithIncorrectValueForChillModePercent() throws Exception {
+ public void testFailWithIncorrectValueForSafeModePercent() throws Exception {
try {
OzoneConfiguration conf = createConf(0.9, 0.1);
- conf.setDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT, -1.0);
+ conf.setDouble(HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT, -1.0);
MockNodeManager mockNodeManager = new MockNodeManager(true, 10);
PipelineManager pipelineManager = new SCMPipelineManager(conf,
mockNodeManager, queue);
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
conf, containers, pipelineManager, queue);
- fail("testFailWithIncorrectValueForChillModePercent");
+ fail("testFailWithIncorrectValueForSafeModePercent");
} catch (IllegalArgumentException ex) {
GenericTestUtils.assertExceptionContains("value should be >= 0.0 and <=" +
" 1.0", ex);
@@ -214,7 +214,7 @@ public class TestSCMChillModeManager {
}
- public void testChillModeExitRuleWithPipelineAvailabilityCheck(
+ public void testSafeModeExitRuleWithPipelineAvailabilityCheck(
int containerCount, int nodeCount, int pipelineCount,
double healthyPipelinePercent, double oneReplicaPercent)
throws Exception {
@@ -244,20 +244,20 @@ public class TestSCMChillModeManager {
container.setState(HddsProtos.LifeCycleState.CLOSED);
}
- scmChillModeManager = new SCMChillModeManager(conf, containers,
+ scmSafeModeManager = new SCMSafeModeManager(conf, containers,
pipelineManager, queue);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
testContainerThreshold(containers, 1.0);
List<Pipeline> pipelines = pipelineManager.getPipelines();
int healthyPipelineThresholdCount =
- scmChillModeManager.getHealthyPipelineChillModeRule()
+ scmSafeModeManager.getHealthyPipelineSafeModeRule()
.getHealthyPipelineThresholdCount();
int oneReplicaThresholdCount =
- scmChillModeManager.getOneReplicaPipelineChillModeRule()
+ scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
.getThresholdCount();
// Because even if no pipelines are there, and threshold we set to zero,
@@ -282,20 +282,20 @@ public class TestSCMChillModeManager {
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 100, 1000 * 5);
}
private void checkHealthy(int expectedCount) throws Exception{
- GenericTestUtils.waitFor(() -> scmChillModeManager
- .getHealthyPipelineChillModeRule()
+ GenericTestUtils.waitFor(() -> scmSafeModeManager
+ .getHealthyPipelineSafeModeRule()
.getCurrentHealthyPipelineCount() == expectedCount,
100, 5000);
}
private void checkOpen(int expectedCount) throws Exception {
- GenericTestUtils.waitFor(() -> scmChillModeManager
- .getOneReplicaPipelineChillModeRule()
+ GenericTestUtils.waitFor(() -> scmSafeModeManager
+ .getOneReplicaPipelineSafeModeRule()
.getCurrentReportedPipelineCount() == expectedCount,
1000, 5000);
}
@@ -313,31 +313,31 @@ public class TestSCMChillModeManager {
@Test
- public void testDisableChillMode() {
+ public void testDisableSafeMode() {
OzoneConfiguration conf = new OzoneConfiguration(config);
- conf.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED, false);
+ conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, false);
PipelineManager pipelineManager = Mockito.mock(PipelineManager.class);
Mockito.doNothing().when(pipelineManager).startPipelineCreator();
- scmChillModeManager =
- new SCMChillModeManager(conf, containers, pipelineManager, queue);
- assertFalse(scmChillModeManager.getInChillMode());
+ scmSafeModeManager =
+ new SCMSafeModeManager(conf, containers, pipelineManager, queue);
+ assertFalse(scmSafeModeManager.getInSafeMode());
}
@Test
- public void testChillModeDataNodeExitRule() throws Exception {
+ public void testSafeModeDataNodeExitRule() throws Exception {
containers = new ArrayList<>();
- testChillModeDataNodes(0);
- testChillModeDataNodes(3);
- testChillModeDataNodes(5);
+ testSafeModeDataNodes(0);
+ testSafeModeDataNodes(3);
+ testSafeModeDataNodes(5);
}
/**
* Check that containers in Allocated state are not considered while
* computing percentage of containers with at least 1 reported replica in
- * chill mode exit rule.
+ * safe mode exit rule.
*/
@Test
- public void testContainerChillModeRule() throws Exception {
+ public void testContainerSafeModeRule() throws Exception {
containers = new ArrayList<>();
// Add 100 containers to the list of containers in SCM
containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
@@ -350,55 +350,55 @@ public class TestSCMChillModeManager {
container.setState(HddsProtos.LifeCycleState.OPEN);
}
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
config, containers, null, queue);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
// When 10 CLOSED containers are reported by DNs, the computed container
// threshold should be 10/25 as there are only 25 CLOSED containers.
// Containers in OPEN state should not contribute towards list of
- // containers while calculating container threshold in SCMChillNodeManager
+ // containers while calculating container threshold in SCMSafeNodeManager
testContainerThreshold(containers.subList(0, 10), 0.4);
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
// When remaining 15 OPEN containers are reported by DNs, the container
// threshold should be (10+15)/25.
testContainerThreshold(containers.subList(10, 25), 1.0);
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 100, 1000 * 5);
}
- private void testChillModeDataNodes(int numOfDns) throws Exception {
+ private void testSafeModeDataNodes(int numOfDns) throws Exception {
OzoneConfiguration conf = new OzoneConfiguration(config);
- conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, numOfDns);
- scmChillModeManager = new SCMChillModeManager(
+ conf.setInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE, numOfDns);
+ scmSafeModeManager = new SCMSafeModeManager(
conf, containers, null, queue);
- // Assert SCM is in Chill mode.
- assertTrue(scmChillModeManager.getInChillMode());
+ // Assert SCM is in Safe mode.
+ assertTrue(scmSafeModeManager.getInSafeMode());
- // Register all DataNodes except last one and assert SCM is in chill mode.
+ // Register all DataNodes except last one and assert SCM is in safe mode.
for (int i = 0; i < numOfDns-1; i++) {
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(containers));
- assertTrue(scmChillModeManager.getInChillMode());
- assertTrue(scmChillModeManager.getCurrentContainerThreshold() == 1);
+ assertTrue(scmSafeModeManager.getInSafeMode());
+ assertTrue(scmSafeModeManager.getCurrentContainerThreshold() == 1);
}
if(numOfDns == 0){
GenericTestUtils.waitFor(() -> {
- return scmChillModeManager.getInChillMode();
+ return scmSafeModeManager.getInSafeMode();
}, 10, 1000 * 10);
return;
}
- // Register last DataNode and check that SCM is out of Chill mode.
+ // Register last DataNode and check that SCM is out of Safe mode.
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(containers));
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 10, 1000 * 10);
}
@@ -408,23 +408,23 @@ public class TestSCMChillModeManager {
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(dnContainers));
GenericTestUtils.waitFor(() -> {
- double threshold = scmChillModeManager.getCurrentContainerThreshold();
+ double threshold = scmSafeModeManager.getCurrentContainerThreshold();
return threshold == expectedThreshold;
}, 100, 2000 * 9);
}
@Test
- public void testChillModePipelineExitRule() throws Exception {
+ public void testSafeModePipelineExitRule() throws Exception {
containers = new ArrayList<>();
containers.addAll(HddsTestUtils.getContainerInfo(25 * 4));
String storageDir = GenericTestUtils.getTempPath(
- TestSCMChillModeManager.class.getName() + UUID.randomUUID());
+ TestSCMSafeModeManager.class.getName() + UUID.randomUUID());
try{
MockNodeManager nodeManager = new MockNodeManager(true, 3);
config.set(HddsConfigKeys.OZONE_METADATA_DIRS, storageDir);
// enable pipeline check
config.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK, true);
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK, true);
SCMPipelineManager pipelineManager = new SCMPipelineManager(config,
nodeManager, queue);
@@ -443,12 +443,12 @@ public class TestSCMChillModeManager {
reportBuilder.addPipelineReport(PipelineReport.newBuilder()
.setPipelineID(pipeline.getId().getProtobuf()));
- scmChillModeManager = new SCMChillModeManager(
+ scmSafeModeManager = new SCMSafeModeManager(
config, containers, pipelineManager, queue);
queue.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
HddsTestUtils.createNodeRegistrationContainerReport(containers));
- assertTrue(scmChillModeManager.getInChillMode());
+ assertTrue(scmSafeModeManager.getInSafeMode());
// Trigger the processed pipeline report event
queue.fireEvent(SCMEvents.PROCESSED_PIPELINE_REPORT,
@@ -456,12 +456,12 @@ public class TestSCMChillModeManager {
reportBuilder.build()));
GenericTestUtils.waitFor(() -> {
- return !scmChillModeManager.getInChillMode();
+ return !scmSafeModeManager.getInSafeMode();
}, 100, 1000 * 10);
pipelineManager.close();
} finally {
config.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
false);
FileUtil.fullyDelete(new File(storageDir));
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestChillModeHandler.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSafeModeHandler.java
similarity index 70%
rename from hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestChillModeHandler.java
rename to hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSafeModeHandler.java
index 8ef555c..6a25bbd 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestChillModeHandler.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSafeModeHandler.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -39,25 +39,25 @@ import org.mockito.Mockito;
import java.util.HashSet;
/**
- * Tests ChillModeHandler behavior.
+ * Tests SafeModeHandler behavior.
*/
-public class TestChillModeHandler {
+public class TestSafeModeHandler {
private OzoneConfiguration configuration;
private SCMClientProtocolServer scmClientProtocolServer;
private ReplicationManager replicationManager;
private BlockManager blockManager;
- private ChillModeHandler chillModeHandler;
+ private SafeModeHandler safeModeHandler;
private EventQueue eventQueue;
- private SCMChillModeManager.ChillModeStatus chillModeStatus;
+ private SCMSafeModeManager.SafeModeStatus safeModeStatus;
private PipelineManager scmPipelineManager;
public void setup(boolean enabled) {
configuration = new OzoneConfiguration();
- configuration.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
+ configuration.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED,
enabled);
- configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT,
+ configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT,
"3s");
scmClientProtocolServer =
Mockito.mock(SCMClientProtocolServer.class);
@@ -71,45 +71,45 @@ public class TestChillModeHandler {
eventQueue);
scmPipelineManager = Mockito.mock(SCMPipelineManager.class);
blockManager = Mockito.mock(BlockManagerImpl.class);
- chillModeHandler =
- new ChillModeHandler(configuration, scmClientProtocolServer,
+ safeModeHandler =
+ new SafeModeHandler(configuration, scmClientProtocolServer,
blockManager, replicationManager, scmPipelineManager);
- eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, chillModeHandler);
- chillModeStatus = new SCMChillModeManager.ChillModeStatus(false);
+ eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, safeModeHandler);
+ safeModeStatus = new SCMSafeModeManager.SafeModeStatus(false);
}
@Test
- public void testChillModeHandlerWithChillModeEnabled() throws Exception {
+ public void testSafeModeHandlerWithSafeModeEnabled() throws Exception {
setup(true);
- Assert.assertTrue(chillModeHandler.getChillModeStatus());
+ Assert.assertTrue(safeModeHandler.getSafeModeStatus());
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
- GenericTestUtils.waitFor(() -> !chillModeHandler.getChillModeStatus(),
+ GenericTestUtils.waitFor(() -> !safeModeHandler.getSafeModeStatus(),
1000, 5000);
- Assert.assertFalse(scmClientProtocolServer.getChillModeStatus());
- Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInChillMode());
+ Assert.assertFalse(scmClientProtocolServer.getSafeModeStatus());
+ Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInSafeMode());
GenericTestUtils.waitFor(() ->
replicationManager.isRunning(), 1000, 5000);
}
@Test
- public void testChillModeHandlerWithChillModeDisbaled() throws Exception{
+ public void testSafeModeHandlerWithSafeModeDisbaled() throws Exception{
setup(false);
- Assert.assertFalse(chillModeHandler.getChillModeStatus());
+ Assert.assertFalse(safeModeHandler.getSafeModeStatus());
- eventQueue.fireEvent(SCMEvents.CHILL_MODE_STATUS, chillModeStatus);
+ eventQueue.fireEvent(SCMEvents.SAFE_MODE_STATUS, safeModeStatus);
- Assert.assertFalse(chillModeHandler.getChillModeStatus());
- Assert.assertFalse(scmClientProtocolServer.getChillModeStatus());
- Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInChillMode());
+ Assert.assertFalse(safeModeHandler.getSafeModeStatus());
+ Assert.assertFalse(scmClientProtocolServer.getSafeModeStatus());
+ Assert.assertFalse(((BlockManagerImpl) blockManager).isScmInSafeMode());
GenericTestUtils.waitFor(() ->
replicationManager.isRunning(), 1000, 5000);
}
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
similarity index 92%
rename from hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
rename to hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
index 22a522a..098c68b 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/chillmode/package-info.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/safemode/package-info.java
@@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
/**
- * SCM Chill mode tests.
+ * SCM Safe mode tests.
*/
diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java
index 9181ed6..f7ac00e 100644
--- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java
+++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/server/TestSCMClientProtocolServer.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.block.BlockManager;
import org.apache.hadoop.hdds.scm.block.BlockManagerImpl;
-import org.apache.hadoop.hdds.scm.chillmode.ChillModeHandler;
+import org.apache.hadoop.hdds.scm.safemode.SafeModeHandler;
import org.apache.hadoop.hdds.scm.container.ReplicationManager;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -53,10 +53,10 @@ public class TestSCMClientProtocolServer {
ReplicationManager replicationManager =
Mockito.mock(ReplicationManager.class);
PipelineManager pipelineManager = Mockito.mock(SCMPipelineManager.class);
- ChillModeHandler chillModeHandler = new ChillModeHandler(config,
+ SafeModeHandler safeModeHandler = new SafeModeHandler(config,
scmClientProtocolServer, blockManager, replicationManager,
pipelineManager);
- eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, chillModeHandler);
+ eventQueue.addHandler(SCMEvents.SAFE_MODE_STATUS, safeModeHandler);
}
@After
@@ -64,9 +64,9 @@ public class TestSCMClientProtocolServer {
}
@Test
- public void testAllocateContainerFailureInChillMode() throws Exception {
+ public void testAllocateContainerFailureInSafeMode() throws Exception {
LambdaTestUtils.intercept(SCMException.class,
- "hillModePrecheck failed for allocateContainer", () -> {
+ "SafeModePrecheck failed for allocateContainer", () -> {
scmClientProtocolServer.allocateContainer(
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, "");
});
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
index c722f2b..5013a74 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SCMCLI.java
@@ -77,7 +77,7 @@ import picocli.CommandLine.Option;
+ "operations.",
versionProvider = HddsVersionProvider.class,
subcommands = {
- ChillModeCommands.class,
+ SafeModeCommands.class,
ListSubcommand.class,
InfoSubcommand.class,
DeleteSubcommand.class,
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCheckSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCheckSubcommand.java
similarity index 78%
rename from hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCheckSubcommand.java
rename to hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCheckSubcommand.java
index 41f413b..f969f4c 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCheckSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCheckSubcommand.java
@@ -28,32 +28,32 @@ import picocli.CommandLine.Command;
import picocli.CommandLine.ParentCommand;
/**
- * This is the handler that process chill mode check command.
+ * This is the handler that process safe mode check command.
*/
@Command(
name = "status",
- description = "Check if SCM is in chill mode",
+ description = "Check if SCM is in safe mode",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ChillModeCheckSubcommand implements Callable<Void> {
+public class SafeModeCheckSubcommand implements Callable<Void> {
private static final Logger LOG =
- LoggerFactory.getLogger(ChillModeCheckSubcommand.class);
+ LoggerFactory.getLogger(SafeModeCheckSubcommand.class);
@ParentCommand
- private ChillModeCommands parent;
+ private SafeModeCommands parent;
@Override
public Void call() throws Exception {
try (ScmClient scmClient = parent.getParent().createScmClient()) {
- boolean execReturn = scmClient.inChillMode();
+ boolean execReturn = scmClient.inSafeMode();
// Output data list
if(execReturn){
- LOG.info("SCM is in chill mode.");
+ LOG.info("SCM is in safe mode.");
} else {
- LOG.info("SCM is out of chill mode.");
+ LOG.info("SCM is out of safe mode.");
}
return null;
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCommands.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
similarity index 79%
rename from hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCommands.java
rename to hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
index b06bab9..c1e953d 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeCommands.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeCommands.java
@@ -28,21 +28,21 @@ import picocli.CommandLine.Command;
import picocli.CommandLine.ParentCommand;
/**
- * Subcommand to group chill mode related operations.
+ * Subcommand to group safe mode related operations.
*/
@Command(
- name = "chillmode",
- description = "Chill mode specific operations",
+ name = "safemode",
+ description = "Safe mode specific operations",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class,
subcommands = {
- ChillModeCheckSubcommand.class,
- ChillModeExitSubcommand.class,
+ SafeModeCheckSubcommand.class,
+ SafeModeExitSubcommand.class,
})
-public class ChillModeCommands implements Callable<Void> {
+public class SafeModeCommands implements Callable<Void> {
private static final Logger LOG =
- LoggerFactory.getLogger(ChillModeCommands.class);
+ LoggerFactory.getLogger(SafeModeCommands.class);
@ParentCommand
private SCMCLI parent;
@@ -54,7 +54,7 @@ public class ChillModeCommands implements Callable<Void> {
@Override
public Void call() throws Exception {
throw new MissingSubcommandException(
- this.parent.getCmd().getSubcommands().get("chillmode").
+ this.parent.getCmd().getSubcommands().get("safemode").
getUsageMessage());
}
}
diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeExitSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeExitSubcommand.java
similarity index 79%
rename from hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeExitSubcommand.java
rename to hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeExitSubcommand.java
index 62df125..9f1db45 100644
--- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ChillModeExitSubcommand.java
+++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/SafeModeExitSubcommand.java
@@ -28,28 +28,28 @@ import picocli.CommandLine.Command;
import picocli.CommandLine.ParentCommand;
/**
- * This is the handler that process chill mode exit command.
+ * This is the handler that process safe mode exit command.
*/
@Command(
name = "exit",
- description = "Force SCM out of chill mode",
+ description = "Force SCM out of safe mode",
mixinStandardHelpOptions = true,
versionProvider = HddsVersionProvider.class)
-public class ChillModeExitSubcommand implements Callable<Void> {
+public class SafeModeExitSubcommand implements Callable<Void> {
private static final Logger LOG =
- LoggerFactory.getLogger(ChillModeExitSubcommand.class);
+ LoggerFactory.getLogger(SafeModeExitSubcommand.class);
@ParentCommand
- private ChillModeCommands parent;
+ private SafeModeCommands parent;
@Override
public Void call() throws Exception {
try (ScmClient scmClient = parent.getParent().createScmClient()) {
- boolean execReturn = scmClient.forceExitChillMode();
+ boolean execReturn = scmClient.forceExitSafeMode();
if(execReturn){
- LOG.info("SCM exit chill mode successfully.");
+ LOG.info("SCM exit safe mode successfully.");
}
return null;
}
diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
index b56ab7f..2c4418c 100644
--- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
+++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/om/exceptions/OMException.java
@@ -177,7 +177,7 @@ public class OMException extends IOException {
LIST_MULTIPART_UPLOAD_PARTS_FAILED,
- SCM_IN_CHILL_MODE,
+ SCM_IN_SAFE_MODE,
INVALID_REQUEST,
diff --git a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
index ff4c584..0f50a78 100644
--- a/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
+++ b/hadoop-ozone/common/src/main/proto/OzoneManagerProtocol.proto
@@ -243,7 +243,7 @@ enum Status {
TOKEN_EXPIRED = 35;
TOKEN_ERROR_OTHER = 36;
LIST_MULTIPART_UPLOAD_PARTS_FAILED = 37;
- SCM_IN_CHILL_MODE = 38;
+ SCM_IN_SAFE_MODE = 38;
INVALID_REQUEST = 39;
BUCKET_ENCRYPTION_KEY_NOT_FOUND = 40;
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
index 4133c73..9f90a2d 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/container/TestContainerStateManagerIntegration.java
@@ -75,7 +75,7 @@ public class TestContainerStateManagerIntegration {
ScmConfigKeys.OZONE_SCM_PIPELINE_OWNER_CONTAINER_COUNT_DEFAULT);
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
cluster.waitForClusterToBeReady();
- cluster.waitTobeOutOfChillMode();
+ cluster.waitTobeOutOfSafeMode();
xceiverClientManager = new XceiverClientManager(conf);
scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
index 53e968b..e1b8653 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/pipeline/TestSCMPipelineManager.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.TestUtils;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
@@ -158,8 +158,8 @@ public class TestSCMPipelineManager {
pipelineManager.setPipelineProvider(HddsProtos.ReplicationType.RATIS,
mockRatisProvider);
- SCMChillModeManager scmChillModeManager =
- new SCMChillModeManager(new OzoneConfiguration(),
+ SCMSafeModeManager scmSafeModeManager =
+ new SCMSafeModeManager(new OzoneConfiguration(),
new ArrayList<>(), pipelineManager, eventQueue);
// create a pipeline in allocated state with no dns yet reported
@@ -173,7 +173,7 @@ public class TestSCMPipelineManager {
// get pipeline report from each dn in the pipeline
PipelineReportHandler pipelineReportHandler =
- new PipelineReportHandler(scmChillModeManager, pipelineManager, conf);
+ new PipelineReportHandler(scmSafeModeManager, pipelineManager, conf);
for (DatanodeDetails dn: pipeline.getNodes()) {
PipelineReportFromDatanode pipelineReportFromDatanode =
TestUtils.getPipelineReportFromDatanode(dn, pipeline.getId());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeWithPipelineRules.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeWithPipelineRules.java
similarity index 87%
rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeWithPipelineRules.java
rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeWithPipelineRules.java
index 7e9b54b..7cfd555 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/chillmode/TestSCMChillModeWithPipelineRules.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/safemode/TestSCMSafeModeWithPipelineRules.java
@@ -16,7 +16,7 @@
* limitations under the License.
*/
-package org.apache.hadoop.hdds.scm.chillmode;
+package org.apache.hadoop.hdds.scm.safemode;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@@ -41,10 +41,10 @@ import java.util.concurrent.TimeoutException;
import static org.junit.Assert.fail;
/**
- * This class tests SCM Chill mode with pipeline rules.
+ * This class tests SCM Safe mode with pipeline rules.
*/
-public class TestSCMChillModeWithPipelineRules {
+public class TestSCMSafeModeWithPipelineRules {
private static MiniOzoneCluster cluster;
private OzoneConfiguration conf = new OzoneConfiguration();
@@ -58,9 +58,9 @@ public class TestSCMChillModeWithPipelineRules {
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
temporaryFolder.newFolder().toString());
conf.setBoolean(
- HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_AVAILABILITY_CHECK,
true);
- conf.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_CHILL_MODE_EXIT, "10s");
+ conf.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, "10s");
conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_INTERVAL, "10s");
clusterBuilder = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(numDatanodes)
@@ -75,7 +75,7 @@ public class TestSCMChillModeWithPipelineRules {
@Test
- public void testScmChillMode() throws Exception {
+ public void testScmSafeMode() throws Exception {
int datanodeCount = 6;
setup(datanodeCount);
@@ -106,24 +106,24 @@ public class TestSCMChillModeWithPipelineRules {
});
- SCMChillModeManager scmChillModeManager =
- cluster.getStorageContainerManager().getScmChillModeManager();
+ SCMSafeModeManager scmSafeModeManager =
+ cluster.getStorageContainerManager().getScmSafeModeManager();
// Ceil(0.1 * 2) is 1, as one pipeline is healthy healthy pipeline rule is
// satisfied
GenericTestUtils.waitFor(() ->
- scmChillModeManager.getHealthyPipelineChillModeRule()
+ scmSafeModeManager.getHealthyPipelineSafeModeRule()
.validate(), 1000, 60000);
// As Ceil(0.9 * 2) is 2, and from second pipeline no datanodes's are
// reported this rule is not met yet.
GenericTestUtils.waitFor(() ->
- !scmChillModeManager.getOneReplicaPipelineChillModeRule()
+ !scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
.validate(), 1000, 60000);
- Assert.assertTrue(cluster.getStorageContainerManager().isInChillMode());
+ Assert.assertTrue(cluster.getStorageContainerManager().isInSafeMode());
DatanodeDetails restartedDatanode = pipelineList.get(1).getFirstNode();
// Now restart one datanode from the 2nd pipeline
@@ -135,13 +135,13 @@ public class TestSCMChillModeWithPipelineRules {
GenericTestUtils.waitFor(() ->
- scmChillModeManager.getOneReplicaPipelineChillModeRule()
+ scmSafeModeManager.getOneReplicaPipelineSafeModeRule()
.validate(), 1000, 60000);
- GenericTestUtils.waitFor(() -> !scmChillModeManager.getInChillMode(), 1000,
+ GenericTestUtils.waitFor(() -> !scmSafeModeManager.getInSafeMode(), 1000,
60000);
- // As after chillmode wait time is not completed, we should have total
+ // As after safemode wait time is not completed, we should have total
// pipeline's as original count 6(1 node pipelines) + 2 (3 node pipeline)
Assert.assertEquals(totalPipelineCount,
pipelineManager.getPipelines().size());
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
index 12df745..4927da1 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java
@@ -89,12 +89,12 @@ public interface MiniOzoneCluster {
void setWaitForClusterToBeReadyTimeout(int timeoutInMs);
/**
- * Waits/blocks till the cluster is out of chill mode.
+ * Waits/blocks till the cluster is out of safe mode.
*
* @throws TimeoutException TimeoutException In case of timeout
* @throws InterruptedException In case of interrupt while waiting
*/
- void waitTobeOutOfChillMode() throws TimeoutException, InterruptedException;
+ void waitTobeOutOfSafeMode() throws TimeoutException, InterruptedException;
/**
* Returns {@link StorageContainerManager} associated with this
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
index 6a29562..8e28670 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneClusterImpl.java
@@ -159,17 +159,17 @@ public class MiniOzoneClusterImpl implements MiniOzoneCluster {
}
/**
- * Waits for SCM to be out of Chill Mode. Many tests can be run iff we are out
- * of Chill mode.
+ * Waits for SCM to be out of Safe Mode. Many tests can be run iff we are out
+ * of Safe mode.
*
* @throws TimeoutException
* @throws InterruptedException
*/
@Override
- public void waitTobeOutOfChillMode()
+ public void waitTobeOutOfSafeMode()
throws TimeoutException, InterruptedException {
GenericTestUtils.waitFor(() -> {
- if (!scm.isInChillMode()) {
+ if (!scm.isInSafeMode()) {
return true;
}
LOG.info("Waiting for cluster to be ready. No datanodes found");
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmChillMode.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
similarity index 79%
rename from hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmChillMode.java
rename to hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
index 5d52701..b34ea72 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmChillMode.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/om/TestScmSafeMode.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@@ -62,10 +62,10 @@ import static org.junit.Assert.fail;
/**
* Test Ozone Manager operation in distributed handler scenario.
*/
-public class TestScmChillMode {
+public class TestScmSafeMode {
private final static Logger LOG = LoggerFactory
- .getLogger(TestScmChillMode.class);
+ .getLogger(TestScmSafeMode.class);
private static MiniOzoneCluster cluster = null;
private static MiniOzoneCluster.Builder builder = null;
private static OzoneConfiguration conf;
@@ -115,7 +115,7 @@ public class TestScmChillMode {
}
@Test(timeout = 300_000)
- public void testChillModeOperations() throws Exception {
+ public void testSafeModeOperations() throws Exception {
// Create {numKeys} random names keys.
TestStorageContainerManagerHelper helper =
new TestStorageContainerManagerHelper(cluster, conf);
@@ -164,23 +164,23 @@ public class TestScmChillMode {
StorageContainerManager scm;
scm = cluster.getStorageContainerManager();
- Assert.assertTrue(scm.isInChillMode());
+ Assert.assertTrue(scm.isInSafeMode());
om = cluster.getOzoneManager();
// As cluster is restarted with out datanodes restart
LambdaTestUtils.intercept(IOException.class,
- "ChillModePrecheck failed for allocateBlock",
+ "SafeModePrecheck failed for allocateBlock",
() -> om.openKey(keyArgs));
}
/**
- * Tests inChillMode & forceExitChillMode api calls.
+ * Tests inSafeMode & forceExitSafeMode api calls.
*/
@Test(timeout = 300_000)
- public void testIsScmInChillModeAndForceExit() throws Exception {
- // Test 1: SCM should be out of chill mode.
- Assert.assertFalse(storageContainerLocationClient.inChillMode());
+ public void testIsScmInSafeModeAndForceExit() throws Exception {
+ // Test 1: SCM should be out of safe mode.
+ Assert.assertFalse(storageContainerLocationClient.inSafeMode());
cluster.stop();
// Restart the cluster with same metadata dir.
@@ -190,18 +190,18 @@ public class TestScmChillMode {
Assert.fail("Cluster startup failed.");
}
- // Test 2: Scm should be in chill mode as datanodes are not started yet.
+ // Test 2: Scm should be in safe mode as datanodes are not started yet.
storageContainerLocationClient = cluster
.getStorageContainerLocationClient();
- Assert.assertTrue(storageContainerLocationClient.inChillMode());
- // Force scm out of chill mode.
+ Assert.assertTrue(storageContainerLocationClient.inSafeMode());
+ // Force scm out of safe mode.
cluster.getStorageContainerManager().getClientProtocolServer()
- .forceExitChillMode();
- // Test 3: SCM should be out of chill mode.
+ .forceExitSafeMode();
+ // Test 3: SCM should be out of safe mode.
GenericTestUtils.waitFor(() -> {
try {
return !cluster.getStorageContainerManager().getClientProtocolServer()
- .inChillMode();
+ .inSafeMode();
} catch (IOException e) {
Assert.fail("Cluster");
return false;
@@ -211,8 +211,8 @@ public class TestScmChillMode {
}
@Test(timeout = 300_000)
- public void testSCMChillMode() throws Exception {
- // Test1: Test chill mode when there are no containers in system.
+ public void testSCMSafeMode() throws Exception {
+ // Test1: Test safe mode when there are no containers in system.
cluster.stop();
try {
@@ -220,12 +220,12 @@ public class TestScmChillMode {
} catch (IOException e) {
Assert.fail("Cluster startup failed.");
}
- assertTrue(cluster.getStorageContainerManager().isInChillMode());
+ assertTrue(cluster.getStorageContainerManager().isInSafeMode());
cluster.startHddsDatanodes();
cluster.waitForClusterToBeReady();
- assertFalse(cluster.getStorageContainerManager().isInChillMode());
+ assertFalse(cluster.getStorageContainerManager().isInSafeMode());
- // Test2: Test chill mode when containers are there in system.
+ // Test2: Test safe mode when containers are there in system.
// Create {numKeys} random names keys.
TestStorageContainerManagerHelper helper =
new TestStorageContainerManagerHelper(cluster, conf);
@@ -254,7 +254,7 @@ public class TestScmChillMode {
cluster.stop();
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
- .captureLogs(SCMChillModeManager.getLogger());
+ .captureLogs(SCMSafeModeManager.getLogger());
logCapturer.clearOutput();
try {
@@ -266,8 +266,8 @@ public class TestScmChillMode {
StorageContainerManager scm;
scm = cluster.getStorageContainerManager();
- assertTrue(scm.isInChillMode());
- assertFalse(logCapturer.getOutput().contains("SCM exiting chill mode."));
+ assertTrue(scm.isInSafeMode());
+ assertFalse(logCapturer.getOutput().contains("SCM exiting safe mode."));
assertTrue(scm.getCurrentContainerThreshold() == 0);
for (HddsDatanodeService dn : cluster.getHddsDatanodes()) {
dn.start(null);
@@ -275,25 +275,25 @@ public class TestScmChillMode {
GenericTestUtils
.waitFor(() -> scm.getCurrentContainerThreshold() == 1.0, 100, 20000);
- double chillModeCutoff = conf
- .getDouble(HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT,
- HddsConfigKeys.HDDS_SCM_CHILLMODE_THRESHOLD_PCT_DEFAULT);
- assertTrue(scm.getCurrentContainerThreshold() >= chillModeCutoff);
- assertTrue(logCapturer.getOutput().contains("SCM exiting chill mode."));
- assertFalse(scm.isInChillMode());
+ double safeModeCutoff = conf
+ .getDouble(HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT,
+ HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT);
+ assertTrue(scm.getCurrentContainerThreshold() >= safeModeCutoff);
+ assertTrue(logCapturer.getOutput().contains("SCM exiting safe mode."));
+ assertFalse(scm.isInSafeMode());
}
@Test(timeout = 300_000)
- public void testSCMChillModeRestrictedOp() throws Exception {
+ public void testSCMSafeModeRestrictedOp() throws Exception {
conf.set(OzoneConfigKeys.OZONE_METADATA_STORE_IMPL,
OzoneConfigKeys.OZONE_METADATA_STORE_IMPL_LEVELDB);
cluster.stop();
cluster = builder.build();
StorageContainerManager scm = cluster.getStorageContainerManager();
- assertTrue(scm.isInChillMode());
+ assertTrue(scm.isInSafeMode());
LambdaTestUtils.intercept(SCMException.class,
- "ChillModePrecheck failed for allocateContainer", () -> {
+ "SafeModePrecheck failed for allocateContainer", () -> {
scm.getClientProtocolServer()
.allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.ONE, "");
@@ -301,48 +301,48 @@ public class TestScmChillMode {
cluster.startHddsDatanodes();
cluster.waitForClusterToBeReady();
- assertFalse(scm.isInChillMode());
+ assertFalse(scm.isInSafeMode());
TestStorageContainerManagerHelper helper =
new TestStorageContainerManagerHelper(cluster, conf);
helper.createKeys(10, 4096);
SCMClientProtocolServer clientProtocolServer = cluster
.getStorageContainerManager().getClientProtocolServer();
- assertFalse((scm.getClientProtocolServer()).getChillModeStatus());
+ assertFalse((scm.getClientProtocolServer()).getSafeModeStatus());
final List<ContainerInfo> containers = scm.getContainerManager()
.getContainers();
- scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS,
- new SCMChillModeManager.ChillModeStatus(true));
+ scm.getEventQueue().fireEvent(SCMEvents.SAFE_MODE_STATUS,
+ new SCMSafeModeManager.SafeModeStatus(true));
GenericTestUtils.waitFor(() -> {
- return clientProtocolServer.getChillModeStatus();
+ return clientProtocolServer.getSafeModeStatus();
}, 50, 1000 * 30);
- assertTrue(clientProtocolServer.getChillModeStatus());
+ assertTrue(clientProtocolServer.getSafeModeStatus());
LambdaTestUtils.intercept(SCMException.class,
"Open container " + containers.get(0).getContainerID() + " "
- + "doesn't have enough replicas to service this operation in Chill"
+ + "doesn't have enough replicas to service this operation in Safe"
+ " mode.", () -> clientProtocolServer
.getContainerWithPipeline(containers.get(0).getContainerID()));
}
@Test(timeout = 300_000)
- public void testSCMChillModeDisabled() throws Exception {
+ public void testSCMSafeModeDisabled() throws Exception {
cluster.stop();
- // If chill mode is disabled, cluster should not be in chill mode even if
+ // If safe mode is disabled, cluster should not be in safe mode even if
// min number of datanodes are not started.
- conf.setBoolean(HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED, false);
- conf.setInt(HddsConfigKeys.HDDS_SCM_CHILLMODE_MIN_DATANODE, 3);
+ conf.setBoolean(HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, false);
+ conf.setInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_DATANODE, 3);
builder = MiniOzoneCluster.newBuilder(conf)
.setHbInterval(1000)
.setHbProcessorInterval(500)
.setNumDatanodes(1);
cluster = builder.build();
StorageContainerManager scm = cluster.getStorageContainerManager();
- assertFalse(scm.isInChillMode());
+ assertFalse(scm.isInSafeMode());
- // Even on SCM restart, cluster should be out of chill mode immediately.
+ // Even on SCM restart, cluster should be out of safe mode immediately.
cluster.restartStorageContainerManager(true);
- assertFalse(scm.isInChillMode());
+ assertFalse(scm.isInSafeMode());
}
}
diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java
index eabf5e0..e700a0e 100644
--- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java
+++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/scm/TestSCMMXBean.java
@@ -116,12 +116,12 @@ public class TestSCMMXBean {
assertEquals(stat.toJsonString(), value);
}
- boolean inChillMode = (boolean) mbs.getAttribute(bean,
- "InChillMode");
- assertEquals(scm.isInChillMode(), inChillMode);
+ boolean inSafeMode = (boolean) mbs.getAttribute(bean,
+ "InSafeMode");
+ assertEquals(scm.isInSafeMode(), inSafeMode);
double containerThreshold = (double) mbs.getAttribute(bean,
- "ChillModeCurrentContainerThreshold");
+ "SafeModeCurrentContainerThreshold");
assertEquals(scm.getCurrentContainerThreshold(), containerThreshold, 0);
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
index ff360e5..c8df904 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/KeyManagerImpl.java
@@ -331,8 +331,8 @@ public class KeyManagerImpl implements KeyManager {
keyInfo.getFactor(), omId, excludeList);
} catch (SCMException ex) {
if (ex.getResult()
- .equals(SCMException.ResultCodes.CHILL_MODE_EXCEPTION)) {
- throw new OMException(ex.getMessage(), ResultCodes.SCM_IN_CHILL_MODE);
+ .equals(SCMException.ResultCodes.SAFE_MODE_EXCEPTION)) {
+ throw new OMException(ex.getMessage(), ResultCodes.SCM_IN_SAFE_MODE);
}
throw ex;
}
diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
index 1c0b9a8..02b8b88 100644
--- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
+++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/ratis/OzoneManagerStateMachine.java
@@ -344,7 +344,7 @@ public class OzoneManagerStateMachine extends BaseStateMachine {
// If request is failed, no need to proceed further.
// Setting the exception with omResponse message and code.
- // TODO: the allocate block fails when scm is in chill mode or when scm is
+ // TODO: the allocate block fails when scm is in safe mode or when scm is
// down, but that error is not correctly received in OM end, once that
// is fixed, we need to see how to handle this failure case or how we
// need to retry or how to handle this scenario. For other errors like
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
index d422227..2d76c3f 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/TestKeyManagerImpl.java
@@ -85,7 +85,7 @@ public class TestKeyManagerImpl {
configurator.setScmNodeManager(nodeManager);
scm = TestUtils.getScm(conf, configurator);
scm.start();
- scm.exitChillMode();
+ scm.exitSafeMode();
scmBlockSize = (long) conf
.getStorageSize(OZONE_SCM_BLOCK_SIZE, OZONE_SCM_BLOCK_SIZE_DEFAULT,
StorageUnit.BYTES);
@@ -99,8 +99,8 @@ public class TestKeyManagerImpl {
Mockito.any(ReplicationType.class),
Mockito.any(ReplicationFactor.class), Mockito.anyString(),
Mockito.any(ExcludeList.class))).thenThrow(
- new SCMException("ChillModePrecheck failed for allocateBlock",
- ResultCodes.CHILL_MODE_EXCEPTION));
+ new SCMException("SafeModePrecheck failed for allocateBlock",
+ ResultCodes.SAFE_MODE_EXCEPTION));
createVolume(VOLUME_NAME);
createBucket(VOLUME_NAME, BUCKET_NAME);
}
@@ -133,7 +133,7 @@ public class TestKeyManagerImpl {
}
@Test
- public void allocateBlockFailureInChillMode() throws Exception {
+ public void allocateBlockFailureInSafeMode() throws Exception {
KeyManager keyManager1 = new KeyManagerImpl(mockScmBlockLocationProtocol,
metadataManager, conf, "om1", null);
OmKeyArgs keyArgs = createBuilder()
@@ -141,14 +141,14 @@ public class TestKeyManagerImpl {
.build();
OpenKeySession keySession = keyManager1.openKey(keyArgs);
LambdaTestUtils.intercept(OMException.class,
- "ChillModePrecheck failed for allocateBlock", () -> {
+ "SafeModePrecheck failed for allocateBlock", () -> {
keyManager1
.allocateBlock(keyArgs, keySession.getId(), new ExcludeList());
});
}
@Test
- public void openKeyFailureInChillMode() throws Exception {
+ public void openKeyFailureInSafeMode() throws Exception {
KeyManager keyManager1 = new KeyManagerImpl(mockScmBlockLocationProtocol,
metadataManager, conf, "om1", null);
OmKeyArgs keyArgs = createBuilder()
@@ -156,7 +156,7 @@ public class TestKeyManagerImpl {
.setDataSize(1000)
.build();
LambdaTestUtils.intercept(OMException.class,
- "ChillModePrecheck failed for allocateBlock", () -> {
+ "SafeModePrecheck failed for allocateBlock", () -> {
keyManager1.openKey(keyArgs);
});
}
diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
index b14eac7..a40404b 100644
--- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
+++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/ratis/TestOzoneManagerStateMachine.java
@@ -206,8 +206,8 @@ public class TestOzoneManagerStateMachine {
return OzoneManagerProtocolProtos.OMResponse.newBuilder().setSuccess(true)
.setAllocateBlockResponse(resp)
.setCmdType(OzoneManagerProtocolProtos.Type.AllocateBlock)
- .setStatus(OzoneManagerProtocolProtos.Status.SCM_IN_CHILL_MODE)
- .setMessage("Scm in Chill mode")
+ .setStatus(OzoneManagerProtocolProtos.Status.SCM_IN_SAFE_MODE)
+ .setMessage("Scm in Safe mode")
.setSuccess(status).build();
}
@@ -251,8 +251,8 @@ public class TestOzoneManagerStateMachine {
// As the request failed, check for keyLocation and the transaction
// context error message
Assert.assertFalse(newOmRequest.getAllocateBlockRequest().hasKeyLocation());
- Assert.assertEquals("Scm in Chill mode " + OMException.STATUS_CODE
- + OMException.ResultCodes.SCM_IN_CHILL_MODE,
+ Assert.assertEquals("Scm in Safe mode " + OMException.STATUS_CODE
+ + OMException.ResultCodes.SCM_IN_SAFE_MODE,
transactionContext.getException().getMessage());
Assert.assertTrue(transactionContext.getException() instanceof IOException);
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkOzoneManager.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkOzoneManager.java
index 6188845..45c90d3 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkOzoneManager.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkOzoneManager.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -94,8 +94,8 @@ public class BenchMarkOzoneManager {
.getPipelines(ReplicationType.RATIS, ReplicationFactor.THREE)) {
pipelineManager.openPipeline(pipeline.getId());
}
- scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS,
- new SCMChillModeManager.ChillModeStatus(false));
+ scm.getEventQueue().fireEvent(SCMEvents.SAFE_MODE_STATUS,
+ new SCMSafeModeManager.SafeModeStatus(false));
Thread.sleep(1000);
// prepare OM
diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkSCM.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkSCM.java
index 1d17030..6ec0833 100644
--- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkSCM.java
+++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/genesis/BenchMarkSCM.java
@@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.block.BlockManager;
-import org.apache.hadoop.hdds.scm.chillmode.SCMChillModeManager;
+import org.apache.hadoop.hdds.scm.safemode.SCMSafeModeManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
@@ -85,8 +85,8 @@ public class BenchMarkSCM {
.getPipelines(ReplicationType.RATIS, ReplicationFactor.THREE)) {
pipelineManager.openPipeline(pipeline.getId());
}
- scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS,
- new SCMChillModeManager.ChillModeStatus(false));
+ scm.getEventQueue().fireEvent(SCMEvents.SAFE_MODE_STATUS,
+ new SCMSafeModeManager.SafeModeStatus(false));
Thread.sleep(1000);
}
} finally {
---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org