nandakumar131 commented on code in PR #7951: URL: https://github.com/apache/ozone/pull/7951#discussion_r1988587055
########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ECContainerSafeModeRule.java: ########## @@ -132,228 +119,142 @@ private boolean isMissing(ContainerID id) { * from the rule validation. */ return false; - } } @VisibleForTesting public double getCurrentContainerThreshold() { - return ratisMaxContainer == 0 ? 1 : - (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer); + return ecMaxContainer == 0 ? 1 : (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); } - @VisibleForTesting - public double getCurrentECContainerThreshold() { - return ecMaxContainer == 0 ? 1 : - (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); - } + /** + * Get the minimum replica. + * + * @param pContainerID containerID + * @return MinReplica. + */ + private int getMinReplica(long pContainerID) { + + try { + ContainerID containerID = ContainerID.valueOf(pContainerID); + ContainerInfo container = containerManager.getContainer(containerID); + ReplicationConfig replicationConfig = container.getReplicationConfig(); + return replicationConfig.getMinimumNodes(); + } catch (ContainerNotFoundException e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } catch (Exception e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } + return 1; + } - // TODO: Report processing logic will be removed in future. HDDS-11958. @Override - protected synchronized void process( - final NodeRegistrationContainerReport reportsProto) { - final DatanodeDetails datanodeDetails = reportsProto.getDatanodeDetails(); - final UUID datanodeUUID = datanodeDetails.getUuid(); - StorageContainerDatanodeProtocolProtos.ContainerReportsProto report = reportsProto.getReport(); + protected void process(NodeRegistrationContainerReport report) { + DatanodeDetails datanodeDetails = report.getDatanodeDetails(); + UUID datanodeUUID = datanodeDetails.getUuid(); - report.getReportsList().forEach(c -> { + report.getReport().getReportsList().forEach(c -> { long containerID = c.getContainerID(); - - - // If it is a Ratis container. - if (ratisContainers.contains(containerID)) { - recordReportedContainer(containerID, Boolean.FALSE); - ratisContainers.remove(containerID); - } - - // If it is an EC container. if (ecContainers.contains(containerID)) { putInContainerDNsMap(containerID, ecContainerDNsMap, datanodeUUID); - recordReportedContainer(containerID, Boolean.TRUE); + recordReportedContainer(containerID, true); } }); if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( - "SCM in safe mode. {} % containers [Ratis] have at least one" - + " reported replica, {} % containers [EC] have at N reported replica.", - getCurrentContainerThreshold() * 100, getCurrentECContainerThreshold() * 100); + "SCM in safe mode. {} % containers [EC] have at N reported replica", + getCurrentContainerThreshold() * 100); } } + private void putInContainerDNsMap(long containerID, + Map<Long, Set<UUID>> containerDNsMap, + UUID datanodeUUID) { + containerDNsMap.computeIfAbsent(containerID, key -> Sets.newHashSet()); + containerDNsMap.get(containerID).add(datanodeUUID); + } + /** * Record the reported Container. * - * We will differentiate and count according to the type of Container. - * * @param containerID containerID - * @param isEcContainer true, means ECContainer, false, means not ECContainer. */ private void recordReportedContainer(long containerID, boolean isEcContainer) { Review Comment: `isEcContainer` parameter can be removed. It's not used anywhere. ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/RatisContainerSafeModeRule.java: ########## @@ -0,0 +1,206 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hdds.scm.safemode; + +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT; +import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +import org.apache.hadoop.hdds.scm.container.ContainerID; +import org.apache.hadoop.hdds.scm.container.ContainerInfo; +import org.apache.hadoop.hdds.scm.container.ContainerManager; +import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; +import org.apache.hadoop.hdds.scm.events.SCMEvents; +import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport; +import org.apache.hadoop.hdds.server.events.EventQueue; +import org.apache.hadoop.hdds.server.events.TypedEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * Class defining Safe mode exit criteria for Ratis Containers. + */ +public class RatisContainerSafeModeRule extends SafeModeExitRule<NodeRegistrationContainerReport> { + + public static final Logger LOG = LoggerFactory.getLogger(RatisContainerSafeModeRule.class); + private static final String NAME = "RatisContainerSafeModeRule"; + + private final ContainerManager containerManager; + // Required cutoff % for containers with at least 1 reported replica. + private final double safeModeCutoff; + // Containers read from scm db (excluding containers in ALLOCATED state). + private final Set<Long> ratisContainers; + private final AtomicLong ratisContainerWithMinReplicas; + private double ratisMaxContainer; + + public RatisContainerSafeModeRule(EventQueue eventQueue, + ConfigurationSource conf, + ContainerManager containerManager, + SCMSafeModeManager manager) { + super(manager, NAME, eventQueue); + this.safeModeCutoff = getSafeModeCutoff(conf); + this.containerManager = containerManager; + this.ratisContainers = new HashSet<>(); + this.ratisContainerWithMinReplicas = new AtomicLong(0); + initializeRule(); + } + + private static double getSafeModeCutoff(ConfigurationSource conf) { + final double cutoff = conf.getDouble(HDDS_SCM_SAFEMODE_THRESHOLD_PCT, + HDDS_SCM_SAFEMODE_THRESHOLD_PCT_DEFAULT); + Preconditions.checkArgument((cutoff >= 0.0 && cutoff <= 1.0), + HDDS_SCM_SAFEMODE_THRESHOLD_PCT + " value should be >= 0.0 and <= 1.0"); + return cutoff; + } + + @Override + protected TypedEvent<NodeRegistrationContainerReport> getEventType() { + return SCMEvents.CONTAINER_REGISTRATION_REPORT; + } + + @Override + protected synchronized boolean validate() { + if (validateBasedOnReportProcessing()) { + return (getCurrentContainerThreshold() >= safeModeCutoff); + } + + final List<ContainerInfo> containers = containerManager.getContainers( + ReplicationType.RATIS); + + return containers.stream() + .filter(this::isClosed) + .map(ContainerInfo::containerID) + .noneMatch(this::isMissing); + } + + /** + * Checks if the container has any replica. + */ + private boolean isMissing(ContainerID id) { + try { + return containerManager.getContainerReplicas(id).isEmpty(); + } catch (ContainerNotFoundException ex) { + /* + * This should never happen, in case this happens the container + * somehow got removed from SCM. + * Safemode rule doesn't have to log/fix this. We will just exclude this + * from the rule validation. + */ + return false; + + } + } + + @VisibleForTesting + public double getCurrentContainerThreshold() { + return ratisMaxContainer == 0 ? 1 : (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer); + } + + @Override + protected void process(NodeRegistrationContainerReport report) { + report.getReport().getReportsList().forEach(c -> { + long containerID = c.getContainerID(); + if (ratisContainers.contains(containerID)) { + recordReportedContainer(containerID); + ratisContainers.remove(containerID); + } + }); + + if (scmInSafeMode()) { + SCMSafeModeManager.getLogger().info( + "SCM in safe mode. {} % containers [Ratis] have at least one reported replica", + getCurrentContainerThreshold() * 100); + } + } + + /** + * Record the reported Container. + * + * @param containerID containerID + */ + private void recordReportedContainer(long containerID) { + ratisContainerWithMinReplicas.getAndAdd(1); + getSafeModeMetrics() + .incCurrentContainersWithOneReplicaReportedCount(); + } + + private void initializeRule() { + ratisContainers.clear(); + containerManager.getContainers().forEach(container -> { + if (container.getReplicationType() == HddsProtos.ReplicationType.RATIS + && isClosed(container) + && container.getNumberOfKeys() > 0) { + ratisContainers.add(container.getContainerID()); + } + }); Review Comment: ```suggestion containerManager.getContainers(ReplicationType.RATIS).stream() .filter(this::isClosed).filter(c -> c.getNumberOfKeys() > 0) .map(ContainerInfo::getContainerID).forEach(ratisContainers::add); ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ECContainerSafeModeRule.java: ########## @@ -132,228 +119,142 @@ private boolean isMissing(ContainerID id) { * from the rule validation. */ return false; - } } @VisibleForTesting public double getCurrentContainerThreshold() { - return ratisMaxContainer == 0 ? 1 : - (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer); + return ecMaxContainer == 0 ? 1 : (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); } - @VisibleForTesting - public double getCurrentECContainerThreshold() { - return ecMaxContainer == 0 ? 1 : - (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); - } + /** + * Get the minimum replica. + * + * @param pContainerID containerID + * @return MinReplica. + */ + private int getMinReplica(long pContainerID) { + + try { + ContainerID containerID = ContainerID.valueOf(pContainerID); + ContainerInfo container = containerManager.getContainer(containerID); + ReplicationConfig replicationConfig = container.getReplicationConfig(); + return replicationConfig.getMinimumNodes(); + } catch (ContainerNotFoundException e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } catch (Exception e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } + return 1; + } - // TODO: Report processing logic will be removed in future. HDDS-11958. @Override - protected synchronized void process( - final NodeRegistrationContainerReport reportsProto) { - final DatanodeDetails datanodeDetails = reportsProto.getDatanodeDetails(); - final UUID datanodeUUID = datanodeDetails.getUuid(); - StorageContainerDatanodeProtocolProtos.ContainerReportsProto report = reportsProto.getReport(); + protected void process(NodeRegistrationContainerReport report) { + DatanodeDetails datanodeDetails = report.getDatanodeDetails(); + UUID datanodeUUID = datanodeDetails.getUuid(); - report.getReportsList().forEach(c -> { + report.getReport().getReportsList().forEach(c -> { long containerID = c.getContainerID(); - - - // If it is a Ratis container. - if (ratisContainers.contains(containerID)) { - recordReportedContainer(containerID, Boolean.FALSE); - ratisContainers.remove(containerID); - } - - // If it is an EC container. if (ecContainers.contains(containerID)) { putInContainerDNsMap(containerID, ecContainerDNsMap, datanodeUUID); - recordReportedContainer(containerID, Boolean.TRUE); + recordReportedContainer(containerID, true); } }); if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( - "SCM in safe mode. {} % containers [Ratis] have at least one" - + " reported replica, {} % containers [EC] have at N reported replica.", - getCurrentContainerThreshold() * 100, getCurrentECContainerThreshold() * 100); + "SCM in safe mode. {} % containers [EC] have at N reported replica", + getCurrentContainerThreshold() * 100); } } + private void putInContainerDNsMap(long containerID, + Map<Long, Set<UUID>> containerDNsMap, + UUID datanodeUUID) { + containerDNsMap.computeIfAbsent(containerID, key -> Sets.newHashSet()); + containerDNsMap.get(containerID).add(datanodeUUID); + } + /** * Record the reported Container. * - * We will differentiate and count according to the type of Container. - * * @param containerID containerID - * @param isEcContainer true, means ECContainer, false, means not ECContainer. */ private void recordReportedContainer(long containerID, boolean isEcContainer) { int uuids = 1; - if (isEcContainer && ecContainerDNsMap.containsKey(containerID)) { + if (ecContainerDNsMap.containsKey(containerID)) { uuids = ecContainerDNsMap.get(containerID).size(); } int minReplica = getMinReplica(containerID); if (uuids >= minReplica) { - if (isEcContainer) { - getSafeModeMetrics() - .incCurrentContainersWithECDataReplicaReportedCount(); - ecContainerWithMinReplicas.getAndAdd(1); - } else { - ratisContainerWithMinReplicas.getAndAdd(1); - getSafeModeMetrics() - .incCurrentContainersWithOneReplicaReportedCount(); - } + getSafeModeMetrics() + .incCurrentContainersWithECDataReplicaReportedCount(); + ecContainerWithMinReplicas.getAndAdd(1); } } - /** - * Get the minimum replica. - * - * If it is a Ratis Contianer, the minimum copy is 1. - * If it is an EC Container, the minimum copy will be the number of Data in replicationConfig. - * - * @param pContainerID containerID - * @return MinReplica. - */ - private int getMinReplica(long pContainerID) { - - try { - ContainerID containerID = ContainerID.valueOf(pContainerID); - ContainerInfo container = containerManager.getContainer(containerID); - ReplicationConfig replicationConfig = container.getReplicationConfig(); - return replicationConfig.getMinimumNodes(); - } catch (ContainerNotFoundException e) { - LOG.error("containerId = {} not found.", pContainerID, e); - } catch (Exception e) { - LOG.error("containerId = {} not found.", pContainerID, e); - } - - return 1; - } + private void initializeRule() { + ecContainers.clear(); + ecContainerDNsMap.clear(); + containerManager.getContainers().forEach(container -> { + if (container.getReplicationType() == HddsProtos.ReplicationType.EC + && isClosed(container) + && container.getNumberOfKeys() > 0) { + ecContainers.add(container.getContainerID()); + } + }); + ecMaxContainer = ecContainers.size(); + long ecCutOff = (long) Math.ceil(ecMaxContainer * safeModeCutoff); + getSafeModeMetrics().setNumContainerWithECDataReplicaReportedThreshold(ecCutOff); - private void putInContainerDNsMap(long containerID, Map<Long, Set<UUID>> containerDNsMap, - UUID datanodeUUID) { - containerDNsMap.computeIfAbsent(containerID, key -> Sets.newHashSet()); - containerDNsMap.get(containerID).add(datanodeUUID); + LOG.info("Refreshed Containers with ec n replica threshold count {}.", ecCutOff); } - @Override - protected synchronized void cleanup() { - ratisContainers.clear(); - ecContainers.clear(); - ecContainerDNsMap.clear(); + private boolean isClosed(ContainerInfo container) { + final LifeCycleState state = container.getState(); + return state == LifeCycleState.QUASI_CLOSED || state == LifeCycleState.CLOSED; } @Override public String getStatusText() { - - // ratis container String status = String.format( - "%1.2f%% of [Ratis] Containers(%s / %s) with at least one reported replica (=%1.2f) >= " + - "safeModeCutoff (=%1.2f);", + "%1.2f%% of [EC] Containers(%s / %s) with at least N reported replica (=%1.2f) >= " + + "safeModeCutoff (=%1.2f);", getCurrentContainerThreshold() * 100, - ratisContainerWithMinReplicas, (long) ratisMaxContainer, + ecContainerWithMinReplicas, (long) ecMaxContainer, getCurrentContainerThreshold(), this.safeModeCutoff); - Set<Long> sampleRatisContainers = ratisContainers.stream(). - limit(SAMPLE_CONTAINER_DISPLAY_LIMIT). - collect(Collectors.toSet()); - - if (!sampleRatisContainers.isEmpty()) { - String sampleContainerText = - "Sample Ratis Containers not satisfying the criteria : " + sampleRatisContainers + ";"; - status = status.concat("\n").concat(sampleContainerText); - } - - // ec container - String ecStatus = String.format( - "%1.2f%% of [EC] Containers(%s / %s) with at least N reported replica (=%1.2f) >= " + - "safeModeCutoff (=%1.2f);", - getCurrentECContainerThreshold() * 100, - ecContainerWithMinReplicas, (long) ecMaxContainer, - getCurrentECContainerThreshold(), this.safeModeCutoff); - status = status.concat("\n").concat(ecStatus); - - Set<Long> sampleEcContainers = ecContainerDNsMap.entrySet().stream(). - filter(entry -> { - Long containerId = entry.getKey(); - int minReplica = getMinReplica(containerId); - Set<UUID> allReplicas = entry.getValue(); - if (allReplicas.size() >= minReplica) { - return false; - } - return true; - }). - map(Map.Entry::getKey). - limit(SAMPLE_CONTAINER_DISPLAY_LIMIT). - collect(Collectors.toSet()); + Set<Long> sampleEcContainers = ecContainerDNsMap.entrySet().stream().filter(entry -> { + Long containerId = entry.getKey(); + int minReplica = getMinReplica(containerId); + Set<UUID> allReplicas = entry.getValue(); + if (allReplicas.size() >= minReplica) { + return false; + } + return true; Review Comment: ```suggestion return allReplicas.size() < minReplica; ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java: ########## @@ -317,30 +318,29 @@ public static Logger getLogger() { return LOG; } - @VisibleForTesting Review Comment: Any reason to remove `@visibleForTesting`? ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/ECContainerSafeModeRule.java: ########## @@ -132,228 +119,142 @@ private boolean isMissing(ContainerID id) { * from the rule validation. */ return false; - } } @VisibleForTesting public double getCurrentContainerThreshold() { - return ratisMaxContainer == 0 ? 1 : - (ratisContainerWithMinReplicas.doubleValue() / ratisMaxContainer); + return ecMaxContainer == 0 ? 1 : (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); } - @VisibleForTesting - public double getCurrentECContainerThreshold() { - return ecMaxContainer == 0 ? 1 : - (ecContainerWithMinReplicas.doubleValue() / ecMaxContainer); - } + /** + * Get the minimum replica. + * + * @param pContainerID containerID + * @return MinReplica. + */ + private int getMinReplica(long pContainerID) { + + try { + ContainerID containerID = ContainerID.valueOf(pContainerID); + ContainerInfo container = containerManager.getContainer(containerID); + ReplicationConfig replicationConfig = container.getReplicationConfig(); + return replicationConfig.getMinimumNodes(); + } catch (ContainerNotFoundException e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } catch (Exception e) { + LOG.error("containerId = {} not found.", pContainerID, e); + } + return 1; + } - // TODO: Report processing logic will be removed in future. HDDS-11958. @Override - protected synchronized void process( - final NodeRegistrationContainerReport reportsProto) { - final DatanodeDetails datanodeDetails = reportsProto.getDatanodeDetails(); - final UUID datanodeUUID = datanodeDetails.getUuid(); - StorageContainerDatanodeProtocolProtos.ContainerReportsProto report = reportsProto.getReport(); + protected void process(NodeRegistrationContainerReport report) { + DatanodeDetails datanodeDetails = report.getDatanodeDetails(); + UUID datanodeUUID = datanodeDetails.getUuid(); - report.getReportsList().forEach(c -> { + report.getReport().getReportsList().forEach(c -> { long containerID = c.getContainerID(); - - - // If it is a Ratis container. - if (ratisContainers.contains(containerID)) { - recordReportedContainer(containerID, Boolean.FALSE); - ratisContainers.remove(containerID); - } - - // If it is an EC container. if (ecContainers.contains(containerID)) { putInContainerDNsMap(containerID, ecContainerDNsMap, datanodeUUID); - recordReportedContainer(containerID, Boolean.TRUE); + recordReportedContainer(containerID, true); } }); if (scmInSafeMode()) { SCMSafeModeManager.getLogger().info( - "SCM in safe mode. {} % containers [Ratis] have at least one" - + " reported replica, {} % containers [EC] have at N reported replica.", - getCurrentContainerThreshold() * 100, getCurrentECContainerThreshold() * 100); + "SCM in safe mode. {} % containers [EC] have at N reported replica", + getCurrentContainerThreshold() * 100); } } + private void putInContainerDNsMap(long containerID, + Map<Long, Set<UUID>> containerDNsMap, + UUID datanodeUUID) { + containerDNsMap.computeIfAbsent(containerID, key -> Sets.newHashSet()); + containerDNsMap.get(containerID).add(datanodeUUID); + } + /** * Record the reported Container. * - * We will differentiate and count according to the type of Container. - * * @param containerID containerID - * @param isEcContainer true, means ECContainer, false, means not ECContainer. */ private void recordReportedContainer(long containerID, boolean isEcContainer) { int uuids = 1; - if (isEcContainer && ecContainerDNsMap.containsKey(containerID)) { + if (ecContainerDNsMap.containsKey(containerID)) { uuids = ecContainerDNsMap.get(containerID).size(); } int minReplica = getMinReplica(containerID); if (uuids >= minReplica) { - if (isEcContainer) { - getSafeModeMetrics() - .incCurrentContainersWithECDataReplicaReportedCount(); - ecContainerWithMinReplicas.getAndAdd(1); - } else { - ratisContainerWithMinReplicas.getAndAdd(1); - getSafeModeMetrics() - .incCurrentContainersWithOneReplicaReportedCount(); - } + getSafeModeMetrics() + .incCurrentContainersWithECDataReplicaReportedCount(); + ecContainerWithMinReplicas.getAndAdd(1); } } - /** - * Get the minimum replica. - * - * If it is a Ratis Contianer, the minimum copy is 1. - * If it is an EC Container, the minimum copy will be the number of Data in replicationConfig. - * - * @param pContainerID containerID - * @return MinReplica. - */ - private int getMinReplica(long pContainerID) { - - try { - ContainerID containerID = ContainerID.valueOf(pContainerID); - ContainerInfo container = containerManager.getContainer(containerID); - ReplicationConfig replicationConfig = container.getReplicationConfig(); - return replicationConfig.getMinimumNodes(); - } catch (ContainerNotFoundException e) { - LOG.error("containerId = {} not found.", pContainerID, e); - } catch (Exception e) { - LOG.error("containerId = {} not found.", pContainerID, e); - } - - return 1; - } + private void initializeRule() { + ecContainers.clear(); + ecContainerDNsMap.clear(); + containerManager.getContainers().forEach(container -> { + if (container.getReplicationType() == HddsProtos.ReplicationType.EC + && isClosed(container) + && container.getNumberOfKeys() > 0) { + ecContainers.add(container.getContainerID()); + } + }); Review Comment: ```suggestion containerManager.getContainers(ReplicationType.EC).stream() .filter(this::isClosed).filter(c -> c.getNumberOfKeys() > 0) .map(ContainerInfo::getContainerID).forEach(ecContainers::add); ``` ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java: ########## @@ -317,30 +318,29 @@ public static Logger getLogger() { return LOG; } - @VisibleForTesting public double getCurrentContainerThreshold() { - return ((ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE)) + return ((RatisContainerSafeModeRule) exitRules.get(RATIS_CONTAINER_EXIT_RULE)) .getCurrentContainerThreshold(); } - @VisibleForTesting public double getCurrentECContainerThreshold() { - return ((ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE)) - .getCurrentECContainerThreshold(); + return ((ECContainerSafeModeRule) exitRules.get(EC_CONTAINER_EXIT_RULE)) + .getCurrentContainerThreshold(); } - @VisibleForTesting - public ContainerSafeModeRule getContainerSafeModeRule() { - return (ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE); + public RatisContainerSafeModeRule getRatisContainerSafeModeRule() { + return (RatisContainerSafeModeRule) exitRules.get(RATIS_CONTAINER_EXIT_RULE); + } + + public ECContainerSafeModeRule getECContainerSafeModeRule() { + return (ECContainerSafeModeRule) exitRules.get(EC_CONTAINER_EXIT_RULE); } - @VisibleForTesting Review Comment: Why is this removed? ########## hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SCMSafeModeManager.java: ########## @@ -317,30 +318,29 @@ public static Logger getLogger() { return LOG; } - @VisibleForTesting public double getCurrentContainerThreshold() { - return ((ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE)) + return ((RatisContainerSafeModeRule) exitRules.get(RATIS_CONTAINER_EXIT_RULE)) .getCurrentContainerThreshold(); } - @VisibleForTesting public double getCurrentECContainerThreshold() { - return ((ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE)) - .getCurrentECContainerThreshold(); + return ((ECContainerSafeModeRule) exitRules.get(EC_CONTAINER_EXIT_RULE)) + .getCurrentContainerThreshold(); } - @VisibleForTesting - public ContainerSafeModeRule getContainerSafeModeRule() { - return (ContainerSafeModeRule) exitRules.get(CONT_EXIT_RULE); + public RatisContainerSafeModeRule getRatisContainerSafeModeRule() { + return (RatisContainerSafeModeRule) exitRules.get(RATIS_CONTAINER_EXIT_RULE); + } + + public ECContainerSafeModeRule getECContainerSafeModeRule() { + return (ECContainerSafeModeRule) exitRules.get(EC_CONTAINER_EXIT_RULE); } - @VisibleForTesting public HealthyPipelineSafeModeRule getHealthyPipelineSafeModeRule() { return (HealthyPipelineSafeModeRule) exitRules.get(HEALTHY_PIPELINE_EXIT_RULE); } - @VisibleForTesting Review Comment: Same as above, why is this removed? -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org