diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java index cba7311b3b4f..4476cbc3e38b 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/recon/TestReconTasks.java @@ -19,6 +19,7 @@ import java.time.Duration; import java.util.List; +import java.util.Map; import org.apache.hadoop.hdds.client.RatisReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -51,6 +52,7 @@ import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_PIPELINE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; import static org.apache.hadoop.ozone.container.ozoneimpl.TestOzoneContainer.runTestOzoneContainerViaDataNode; +import static org.apache.hadoop.ozone.recon.ReconConstants.CONTAINER_COUNT; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -236,6 +238,8 @@ public void testEmptyMissingContainerDownNode() throws Exception { // Bring down the Datanode that had the container replica. cluster.shutdownHddsDatanode(pipeline.getFirstNode()); + // Since we no longer add EMPTY_MISSING containers to the table, we should + // have zero EMPTY_MISSING containers in the DB but their information will be logged. LambdaTestUtils.await(25000, 1000, () -> { List allEmptyMissingContainers = reconContainerManager.getContainerSchemaManager() @@ -243,9 +247,18 @@ public void testEmptyMissingContainerDownNode() throws Exception { ContainerSchemaDefinition.UnHealthyContainerStates. EMPTY_MISSING, 0, 1000); - return (allEmptyMissingContainers.size() == 1); - }); + // Check if EMPTY_MISSING containers are not added to the DB and their count is logged + Map> + unhealthyContainerStateStatsMap = reconScm.getContainerHealthTask() + .getUnhealthyContainerStateStatsMap(); + + // Return true if the size of the fetched containers is 0 and the log shows 1 for EMPTY_MISSING state + return allEmptyMissingContainers.size() == 0 && + unhealthyContainerStateStatsMap.get( + ContainerSchemaDefinition.UnHealthyContainerStates.EMPTY_MISSING) + .getOrDefault(CONTAINER_COUNT, 0L) == 1; + }); // Now add a container to key mapping count as 3. This data is used to // identify if container is empty in terms of keys mapped to container. @@ -272,7 +285,17 @@ public void testEmptyMissingContainerDownNode() throws Exception { ContainerSchemaDefinition.UnHealthyContainerStates. EMPTY_MISSING, 0, 1000); - return (allEmptyMissingContainers.isEmpty()); + + + Map> + unhealthyContainerStateStatsMap = reconScm.getContainerHealthTask() + .getUnhealthyContainerStateStatsMap(); + + // Return true if the size of the fetched containers is 0 and the log shows 0 for EMPTY_MISSING state + return allEmptyMissingContainers.size() == 0 && + unhealthyContainerStateStatsMap.get( + ContainerSchemaDefinition.UnHealthyContainerStates.EMPTY_MISSING) + .getOrDefault(CONTAINER_COUNT, 0L) == 0; }); // Now remove keys from container. This data is used to @@ -283,8 +306,8 @@ public void testEmptyMissingContainerDownNode() throws Exception { reconContainerMetadataManager.commitBatchOperation(rdbBatchOperation); } - // Check existing container state in UNHEALTHY_CONTAINER table - // will be updated as EMPTY_MISSING + // Since we no longer add EMPTY_MISSING containers to the table, we should + // have zero EMPTY_MISSING containers in the DB but their information will be logged. LambdaTestUtils.await(25000, 1000, () -> { List allEmptyMissingContainers = reconContainerManager.getContainerSchemaManager() @@ -292,7 +315,16 @@ public void testEmptyMissingContainerDownNode() throws Exception { ContainerSchemaDefinition.UnHealthyContainerStates. EMPTY_MISSING, 0, 1000); - return (allEmptyMissingContainers.size() == 1); + + Map> + unhealthyContainerStateStatsMap = reconScm.getContainerHealthTask() + .getUnhealthyContainerStateStatsMap(); + + // Return true if the size of the fetched containers is 0 and the log shows 1 for EMPTY_MISSING state + return allEmptyMissingContainers.size() == 0 && + unhealthyContainerStateStatsMap.get( + ContainerSchemaDefinition.UnHealthyContainerStates.EMPTY_MISSING) + .getOrDefault(CONTAINER_COUNT, 0L) == 1; }); // Now restart the cluster and verify the container is no longer missing. diff --git a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java index 7c293ff1861e..0882de3bf4fa 100644 --- a/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java +++ b/hadoop-ozone/recon-codegen/src/main/java/org/hadoop/ozone/recon/schema/ContainerSchemaDefinition.java @@ -31,6 +31,7 @@ import javax.sql.DataSource; import java.sql.Connection; import java.sql.SQLException; +import java.util.Arrays; /** * Class used to create tables that are required for tracking containers. @@ -69,11 +70,39 @@ public enum UnHealthyContainerStates { public void initializeSchema() throws SQLException { Connection conn = dataSource.getConnection(); dslContext = DSL.using(conn); - if (!TABLE_EXISTS_CHECK.test(conn, UNHEALTHY_CONTAINERS_TABLE_NAME)) { + + if (TABLE_EXISTS_CHECK.test(conn, UNHEALTHY_CONTAINERS_TABLE_NAME)) { + // Drop the existing constraint if it exists + String constraintName = UNHEALTHY_CONTAINERS_TABLE_NAME + "ck1"; + dslContext.alterTable(UNHEALTHY_CONTAINERS_TABLE_NAME) + .dropConstraint(constraintName) + .execute(); + + // Add the updated constraint with all enum states + addUpdatedConstraint(); + } else { + // Create the table if it does not exist createUnhealthyContainersTable(); } } + /** + * Add the updated constraint to the table. + */ + private void addUpdatedConstraint() { + // Get all enum values as a list of strings + String[] enumStates = Arrays.stream(UnHealthyContainerStates.values()) + .map(Enum::name) + .toArray(String[]::new); + + // Alter the table to add the updated constraint + dslContext.alterTable(UNHEALTHY_CONTAINERS_TABLE_NAME) + .add(DSL.constraint(UNHEALTHY_CONTAINERS_TABLE_NAME + "ck1") + .check(field(name("container_state")) + .in(enumStates))) + .execute(); + } + /** * Create the Missing Containers table. */ diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java index 86ef6c022d57..cbdc198f8aaf 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/ContainerEndpoint.java @@ -408,13 +408,18 @@ public Response getUnhealthyContainers( summary = containerHealthSchemaManager.getUnhealthyContainersSummary(); List containers = containerHealthSchemaManager .getUnhealthyContainers(internalState, offset, limit); - List emptyMissingFiltered = containers.stream() - .filter( - container -> !container.getContainerState() - .equals(UnHealthyContainerStates.EMPTY_MISSING.toString())) - .collect( - Collectors.toList()); - for (UnhealthyContainers c : emptyMissingFiltered) { + + // Filtering out EMPTY_MISSING and NEGATIVE_SIZE containers from the response. + // These container states are not being inserted into the database as they represent + // edge cases that are not critical to track as unhealthy containers. + List filteredContainers = containers.stream() + .filter(container -> !container.getContainerState() + .equals(UnHealthyContainerStates.EMPTY_MISSING.toString()) + && !container.getContainerState() + .equals(UnHealthyContainerStates.NEGATIVE_SIZE.toString())) + .collect(Collectors.toList()); + + for (UnhealthyContainers c : filteredContainers) { long containerID = c.getContainerId(); ContainerInfo containerInfo = containerManager.getContainer(ContainerID.valueOf(containerID)); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java index ba03ec61f145..eaf08d9ca83e 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/api/types/UnhealthyContainersResponse.java @@ -50,12 +50,6 @@ public class UnhealthyContainersResponse { @JsonProperty("misReplicatedCount") private long misReplicatedCount = 0; - /** - * Total count of containers with negative size. - */ - @JsonProperty("negativeSizeCount") - private long negativeSizeCount = 0; - /** * A collection of unhealthy containers. */ @@ -83,9 +77,6 @@ public void setSummaryCount(String state, long count) { } else if (state.equals( UnHealthyContainerStates.MIS_REPLICATED.toString())) { this.misReplicatedCount = count; - } else if (state.equals( - UnHealthyContainerStates.NEGATIVE_SIZE.toString())) { - this.negativeSizeCount = count; } } @@ -105,10 +96,6 @@ public long getMisReplicatedCount() { return misReplicatedCount; } - public long getNegativeSizeCount() { - return negativeSizeCount; - } - public Collection getContainers() { return containers; } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java index 639047d37bd6..11af6eaff53d 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/fsck/ContainerHealthTask.java @@ -29,6 +29,7 @@ import java.util.concurrent.locks.ReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.PlacementPolicy; @@ -78,6 +79,8 @@ public class ContainerHealthTask extends ReconScmTask { private final ReconContainerMetadataManager reconContainerMetadataManager; private final PlacementPolicy placementPolicy; private final long interval; + private Map> + unhealthyContainerStateStatsMapForTesting; private final Set processedContainers = new HashSet<>(); @@ -185,10 +188,12 @@ private void checkAndProcessContainers( private void logUnhealthyContainerStats( Map> unhealthyContainerStateStatsMap) { + unhealthyContainerStateStatsMapForTesting = + new HashMap<>(unhealthyContainerStateStatsMap); // If any EMPTY_MISSING containers, then it is possible that such // containers got stuck in the closing state which never got // any replicas created on the datanodes. In this case, we log it as - // EMPTY, and insert as EMPTY_MISSING in UNHEALTHY_CONTAINERS table. + // EMPTY_MISSING in unhealthy container statistics but do not add it to the table. unhealthyContainerStateStatsMap.entrySet().forEach(stateEntry -> { UnHealthyContainerStates unhealthyContainerState = stateEntry.getKey(); Map containerStateStatsMap = stateEntry.getValue(); @@ -256,6 +261,11 @@ private void completeProcessingContainer( * completeProcessingContainer is called. This will check to see if any * additional records need to be added to the database. * + * If a container is identified as missing, empty-missing, under-replicated, + * over-replicated or mis-replicated, the method checks with SCM to determine + * if it has been deleted, using {@code containerDeletedInSCM}. If the container is + * deleted in SCM, the corresponding record is removed from Recon. + * * @param currentTime Timestamp to place on all records generated by this run * @param unhealthyContainerStateCountMap * @return Count of records processed @@ -273,9 +283,11 @@ private long processExistingDBRecords(long currentTime, recordCount++; UnhealthyContainersRecord rec = cursor.fetchNext(); try { + // Set the current container if it's not already set if (currentContainer == null) { currentContainer = setCurrentContainer(rec.getContainerId()); } + // If the container ID has changed, finish processing the previous one if (currentContainer.getContainerID() != rec.getContainerId()) { completeProcessingContainer( currentContainer, existingRecords, currentTime, @@ -283,24 +295,29 @@ private long processExistingDBRecords(long currentTime, existingRecords.clear(); currentContainer = setCurrentContainer(rec.getContainerId()); } - if (ContainerHealthRecords - .retainOrUpdateRecord(currentContainer, rec - )) { - // Check if the missing container is deleted in SCM - if (currentContainer.isMissing() && - containerDeletedInSCM(currentContainer.getContainer())) { - rec.delete(); - } - existingRecords.add(rec.getContainerState()); - if (rec.changed()) { - rec.update(); - } - } else { + + // Unhealthy Containers such as MISSING, UNDER_REPLICATED, + // OVER_REPLICATED, MIS_REPLICATED can have their unhealthy states changed or retained. + if (!ContainerHealthRecords.retainOrUpdateRecord(currentContainer, rec)) { + rec.delete(); LOG.info("DELETED existing unhealthy container record...for Container: {}", currentContainer.getContainerID()); + } + + // If the container is marked as MISSING and it's deleted in SCM, remove the record + if (currentContainer.isMissing() && containerDeletedInSCM(currentContainer.getContainer())) { rec.delete(); + LOG.info("DELETED existing unhealthy container record...for Container: {}", + currentContainer.getContainerID()); + } + + existingRecords.add(rec.getContainerState()); + // If the record was changed, update it + if (rec.changed()) { + rec.update(); } } catch (ContainerNotFoundException cnf) { + // If the container is not found, delete the record and reset currentContainer rec.delete(); currentContainer = null; } @@ -326,13 +343,6 @@ private void processContainer(ContainerInfo container, long currentTime, containerReplicas, placementPolicy, reconContainerMetadataManager, conf); - // Handle negative sized containers separately - if (h.getContainer().getUsedBytes() < 0) { - handleNegativeSizedContainers(h, currentTime, - unhealthyContainerStateStatsMap); - return; - } - if (h.isHealthilyReplicated() || h.isDeleted()) { return; } @@ -349,6 +359,18 @@ private void processContainer(ContainerInfo container, long currentTime, } } + /** + * Ensures the container's state in Recon is updated to match its state in SCM. + * + * If SCM reports the container as DELETED, this method attempts to transition + * the container's state in Recon from CLOSED to DELETING, or from DELETING to + * DELETED, based on the current state in Recon. It logs each transition attempt + * and handles any exceptions that may occur. + * + * @param containerInfo the container whose state is being checked and potentially updated. + * @return {@code true} if the container was found to be DELETED in SCM and the + * state transition was attempted in Recon; {@code false} otherwise. + */ private boolean containerDeletedInSCM(ContainerInfo containerInfo) { try { ContainerWithPipeline containerWithPipeline = @@ -358,6 +380,8 @@ private boolean containerDeletedInSCM(ContainerInfo containerInfo) { if (containerInfo.getState() == HddsProtos.LifeCycleState.CLOSED) { containerManager.updateContainerState(containerInfo.containerID(), HddsProtos.LifeCycleEvent.DELETE); + LOG.debug("Successfully changed container {} state from CLOSED to DELETING.", + containerInfo.containerID()); } if (containerInfo.getState() == HddsProtos.LifeCycleState.DELETING && containerManager.getContainerReplicas(containerInfo.containerID()) @@ -365,6 +389,7 @@ private boolean containerDeletedInSCM(ContainerInfo containerInfo) { ) { containerManager.updateContainerState(containerInfo.containerID(), HddsProtos.LifeCycleEvent.CLEANUP); + LOG.info("Successfully Deleted container {} from Recon.", containerInfo.containerID()); } return true; } @@ -380,28 +405,50 @@ private boolean containerDeletedInSCM(ContainerInfo containerInfo) { /** * This method is used to handle containers with negative sizes. It logs an - * error message and inserts a record into the UNHEALTHY_CONTAINERS table. + * error message. * @param containerHealthStatus * @param currentTime * @param unhealthyContainerStateStatsMap */ - private void handleNegativeSizedContainers( + private static void handleNegativeSizedContainers( ContainerHealthStatus containerHealthStatus, long currentTime, Map> unhealthyContainerStateStatsMap) { + // NEGATIVE_SIZE containers are also not inserted into the database. + // This condition usually arises due to corrupted or invalid metadata, where + // the container's size is inaccurately recorded as negative. Since this does not + // represent a typical unhealthy scenario and may not have any meaningful + // impact on system health, such containers are logged for investigation but + // excluded from the UNHEALTHY_CONTAINERS table to maintain data integrity. ContainerInfo container = containerHealthStatus.getContainer(); - LOG.error( - "Container {} has negative size. Please visit Recon's unhealthy " + - "container endpoint for more details.", - container.getContainerID()); - UnhealthyContainers record = - ContainerHealthRecords.recordForState(containerHealthStatus, - UnHealthyContainerStates.NEGATIVE_SIZE, currentTime); - List records = Collections.singletonList(record); - populateContainerStats(containerHealthStatus, - UnHealthyContainerStates.NEGATIVE_SIZE, + LOG.error("Container {} has negative size.", container.getContainerID()); + populateContainerStats(containerHealthStatus, UnHealthyContainerStates.NEGATIVE_SIZE, + unhealthyContainerStateStatsMap); + } + + /** + * This method is used to handle containers that are empty and missing. It logs + * a debug message. + * @param containerHealthStatus + * @param currentTime + * @param unhealthyContainerStateStatsMap + */ + private static void handleEmptyMissingContainers( + ContainerHealthStatus containerHealthStatus, long currentTime, + Map> + unhealthyContainerStateStatsMap) { + // EMPTY_MISSING containers are not inserted into the database. + // These containers typically represent those that were never written to + // or remain in an incomplete state. Tracking such containers as unhealthy + // would not provide valuable insights since they don't pose a risk or issue + // to the system. Instead, they are logged for awareness, but not stored in + // the UNHEALTHY_CONTAINERS table to avoid unnecessary entries. + ContainerInfo container = containerHealthStatus.getContainer(); + LOG.debug("Empty container {} is missing. It will be logged in the " + + "unhealthy container statistics, but no record will be created in the " + + "UNHEALTHY_CONTAINERS table.", container.getContainerID()); + populateContainerStats(containerHealthStatus, EMPTY_MISSING, unhealthyContainerStateStatsMap); - containerHealthSchemaManager.insertUnhealthyContainerRecords(records); } /** @@ -492,22 +539,21 @@ public static List generateUnhealthyRecords( populateContainerStats(container, UnHealthyContainerStates.MISSING, unhealthyContainerStateStatsMap); } else { - - LOG.debug("Empty container {} is missing. Kindly check the " + - "consolidated container stats per UNHEALTHY state logged as " + - "starting with **Container State Stats:**"); - - records.add( - recordForState(container, EMPTY_MISSING, - time)); - populateContainerStats(container, - EMPTY_MISSING, + handleEmptyMissingContainers(container, time, unhealthyContainerStateStatsMap); } // A container cannot have any other records if it is missing so return return records; } + // For Negative sized containers we only log but not insert into DB + if (container.getContainer().getUsedBytes() < 0 + && !recordForStateExists.contains( + UnHealthyContainerStates.NEGATIVE_SIZE.toString())) { + handleNegativeSizedContainers(container, time, + unhealthyContainerStateStatsMap); + } + if (container.isUnderReplicated() && !recordForStateExists.contains( UnHealthyContainerStates.UNDER_REPLICATED.toString())) { @@ -650,4 +696,23 @@ private static void populateContainerStats( (value + container.getContainer().getUsedBytes())); } } + + /** + * Expose the logger for testing purposes. + * + * @return the logger instance + */ + @VisibleForTesting + public Logger getLogger() { + return LOG; + } + + /** + * Expose the unhealthyContainerStateStatsMap for testing purposes. + */ + @VisibleForTesting + public Map> getUnhealthyContainerStateStatsMap() { + return unhealthyContainerStateStatsMapForTesting; + } + } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java index a7f486ea5acc..c773187c4b1d 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/scm/ReconStorageContainerManagerFacade.java @@ -155,6 +155,7 @@ public class ReconStorageContainerManagerFacade private final SCMNodeDetails reconNodeDetails; private final SCMHAManager scmhaManager; private final SequenceIdGenerator sequenceIdGen; + private final ContainerHealthTask containerHealthTask; private DBStore dbStore; private ReconNodeManager nodeManager; @@ -272,7 +273,7 @@ public ReconStorageContainerManagerFacade(OzoneConfiguration conf, scmServiceProvider, reconTaskStatusDao, reconTaskConfig); - ContainerHealthTask containerHealthTask = new ContainerHealthTask( + containerHealthTask = new ContainerHealthTask( containerManager, scmServiceProvider, reconTaskStatusDao, containerHealthSchemaManager, containerPlacementPolicy, reconTaskConfig, reconContainerMetadataManager, conf); @@ -741,6 +742,12 @@ public StorageContainerServiceProvider getScmServiceProvider() { public ContainerSizeCountTask getContainerSizeCountTask() { return containerSizeCountTask; } + + @VisibleForTesting + public ContainerHealthTask getContainerHealthTask() { + return containerHealthTask; + } + @VisibleForTesting public ContainerCountBySizeDao getContainerCountBySizeDao() { return containerCountBySizeDao; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java index 82c7c1b5bef0..3c39e4192d2c 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestContainerEndpoint.java @@ -886,7 +886,9 @@ public void testUnhealthyContainersFilteredResponse() throws IOException, TimeoutException { String missing = UnHealthyContainerStates.MISSING.toString(); String emptyMissing = UnHealthyContainerStates.EMPTY_MISSING.toString(); + String negativeSize = UnHealthyContainerStates.NEGATIVE_SIZE.toString(); // For NEGATIVE_SIZE state + // Initial empty response verification Response response = containerEndpoint .getUnhealthyContainers(missing, 1000, 1); @@ -899,44 +901,55 @@ public void testUnhealthyContainersFilteredResponse() assertEquals(0, responseObject.getMisReplicatedCount()); assertEquals(Collections.EMPTY_LIST, responseObject.getContainers()); + // Add unhealthy records putContainerInfos(5); uuid1 = newDatanode("host1", "127.0.0.1"); uuid2 = newDatanode("host2", "127.0.0.2"); uuid3 = newDatanode("host3", "127.0.0.3"); uuid4 = newDatanode("host4", "127.0.0.4"); createUnhealthyRecords(5, 4, 3, 2); - createEmptyMissingUnhealthyRecords(2); + createEmptyMissingUnhealthyRecords(2); // For EMPTY_MISSING state + createNegativeSizeUnhealthyRecords(2); // For NEGATIVE_SIZE state + // Check for unhealthy containers response = containerEndpoint.getUnhealthyContainers(missing, 1000, 1); responseObject = (UnhealthyContainersResponse) response.getEntity(); + // Summary should have the count for all unhealthy: assertEquals(5, responseObject.getMissingCount()); assertEquals(4, responseObject.getOverReplicatedCount()); assertEquals(3, responseObject.getUnderReplicatedCount()); assertEquals(2, responseObject.getMisReplicatedCount()); - Collection records - = responseObject.getContainers(); + Collection records = responseObject.getContainers(); assertTrue(records.stream() .flatMap(containerMetadata -> containerMetadata.getReplicas().stream() .map(ContainerHistory::getState)) .allMatch(s -> s.equals("UNHEALTHY"))); - // There should only be 5 missing containers and no others as we asked for - // only missing. + + // Verify only missing containers are returned assertEquals(5, records.size()); for (UnhealthyContainerMetadata r : records) { assertEquals(missing, r.getContainerState()); } + // Check for empty missing containers, should return zero Response filteredEmptyMissingResponse = containerEndpoint .getUnhealthyContainers(emptyMissing, 1000, 1); responseObject = (UnhealthyContainersResponse) filteredEmptyMissingResponse.getEntity(); records = responseObject.getContainers(); - // Assert for zero empty missing containers. + assertEquals(0, records.size()); + + // Check for negative size containers, should return zero + Response filteredNegativeSizeResponse = containerEndpoint + .getUnhealthyContainers(negativeSize, 1000, 1); + responseObject = (UnhealthyContainersResponse) filteredNegativeSizeResponse.getEntity(); + records = responseObject.getContainers(); assertEquals(0, records.size()); } + @Test public void testUnhealthyContainersInvalidState() { WebApplicationException e = assertThrows(WebApplicationException.class, @@ -1043,6 +1056,15 @@ private void createEmptyMissingUnhealthyRecords(int emptyMissing) { } } + private void createNegativeSizeUnhealthyRecords(int negativeSize) { + int cid = 0; + for (int i = 0; i < negativeSize; i++) { + createUnhealthyRecord(++cid, UnHealthyContainerStates.NEGATIVE_SIZE.toString(), + 3, 3, 0, null); // Added for NEGATIVE_SIZE state + } + } + + private void createUnhealthyRecords(int missing, int overRep, int underRep, int misRep) { int cid = 0; diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java index ae46bd8b5b5f..46e4506a5ef9 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTask.java @@ -22,15 +22,18 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_BAD; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.fail; -import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.any; import static org.mockito.Mockito.anyInt; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; + import java.io.IOException; import java.time.Duration; import java.util.ArrayList; @@ -105,7 +108,7 @@ public void testRun() throws Exception { // Create 7 containers. The first 5 will have various unhealthy states // defined below. The container with ID=6 will be healthy and - // container with ID=7 will be EMPTY_MISSING + // container with ID=7 will be EMPTY_MISSING (but not inserted into DB) List mockContainers = getMockContainers(7); when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock); when(scmMock.getContainerManager()).thenReturn(containerManagerMock); @@ -132,20 +135,20 @@ public void testRun() throws Exception { when(containerManagerMock.getContainerReplicas(containerInfo2.containerID())) .thenReturn(getMockReplicas(2L, State.UNHEALTHY)); - // return 0 replicas for container ID 3 -> Empty Missing + // return 0 replicas for container ID 3 -> EMPTY_MISSING (will not be inserted into DB) ContainerInfo containerInfo3 = TestContainerInfo.newBuilderForTest().setContainerID(3).setReplicationConfig(replicationConfig).build(); when(containerManagerMock.getContainer(ContainerID.valueOf(3L))).thenReturn(containerInfo3); when(containerManagerMock.getContainerReplicas(containerInfo3.containerID())) .thenReturn(Collections.emptySet()); - // Return 5 Healthy -> Over replicated + // Return 5 Healthy Replicas -> Over-replicated ContainerInfo containerInfo4 = TestContainerInfo.newBuilderForTest().setContainerID(4).setReplicationConfig(replicationConfig).build(); when(containerManagerMock.getContainer(ContainerID.valueOf(4L))).thenReturn(containerInfo4); when(containerManagerMock.getContainerReplicas(containerInfo4.containerID())) .thenReturn(getMockReplicas(4L, State.CLOSED, State.CLOSED, - State.CLOSED, State.CLOSED, State.CLOSED)); + State.CLOSED, State.CLOSED, State.CLOSED)); // Mis-replicated ContainerInfo containerInfo5 = @@ -158,7 +161,7 @@ public void testRun() throws Exception { when(containerManagerMock.getContainerReplicas(containerInfo5.containerID())) .thenReturn(misReplicas); - // Return 3 Healthy -> Healthy container + // Return 3 Healthy Replicas -> Healthy container ContainerInfo containerInfo6 = TestContainerInfo.newBuilderForTest().setContainerID(6).setReplicationConfig(replicationConfig).build(); when(containerManagerMock.getContainer(ContainerID.valueOf(6L))).thenReturn(containerInfo6); @@ -166,12 +169,14 @@ public void testRun() throws Exception { .thenReturn(getMockReplicas(6L, State.CLOSED, State.CLOSED, State.CLOSED)); - // return 0 replicas for container ID 7 -> MISSING + // return 0 replicas for container ID 7 -> MISSING (will later transition to EMPTY_MISSING but not inserted into DB) ContainerInfo containerInfo7 = TestContainerInfo.newBuilderForTest().setContainerID(7).setReplicationConfig(replicationConfig).build(); when(containerManagerMock.getContainer(ContainerID.valueOf(7L))).thenReturn(containerInfo7); when(containerManagerMock.getContainerReplicas(containerInfo7.containerID())) .thenReturn(Collections.emptySet()); + when(reconContainerMetadataManager.getKeyCountForContainer( + 7L)).thenReturn(5L); // Indicates non-empty container 7 for now List all = unHealthyContainersTableHandle.findAll(); assertThat(all).isEmpty(); @@ -180,8 +185,8 @@ public void testRun() throws Exception { ReconTaskStatusDao reconTaskStatusDao = getDao(ReconTaskStatusDao.class); ReconTaskConfig reconTaskConfig = new ReconTaskConfig(); reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(5)); - when(reconContainerMetadataManager.getKeyCountForContainer( - 7L)).thenReturn(5L); + + // Start container health task ContainerHealthTask containerHealthTask = new ContainerHealthTask(scmMock.getContainerManager(), scmMock.getScmServiceProvider(), @@ -189,8 +194,12 @@ public void testRun() throws Exception { placementMock, reconTaskConfig, reconContainerMetadataManager, new OzoneConfiguration()); containerHealthTask.start(); + + // Ensure unhealthy container count in DB matches expected LambdaTestUtils.await(60000, 1000, () -> - (unHealthyContainersTableHandle.count() == 6)); + (unHealthyContainersTableHandle.count() == 5)); + + // Check for UNDER_REPLICATED container states UnhealthyContainers rec = unHealthyContainersTableHandle.fetchByContainerId(1L).get(0); assertEquals("UNDER_REPLICATED", rec.getContainerState()); @@ -200,6 +209,10 @@ public void testRun() throws Exception { assertEquals("UNDER_REPLICATED", rec.getContainerState()); assertEquals(3, rec.getReplicaDelta().intValue()); + // Assert that EMPTY_MISSING state containers were never added to DB. + assertEquals(0, + unHealthyContainersTableHandle.fetchByContainerId(3L).size()); + List unhealthyContainers = containerHealthSchemaManager.getUnhealthyContainers( ALL_REPLICAS_BAD, 0, Integer.MAX_VALUE); @@ -209,10 +222,7 @@ public void testRun() throws Exception { assertEquals(0, unhealthyContainers.get(0).getActualReplicaCount().intValue()); - rec = unHealthyContainersTableHandle.fetchByContainerId(3L).get(0); - assertEquals("EMPTY_MISSING", rec.getContainerState()); - assertEquals(3, rec.getReplicaDelta().intValue()); - + // Check for MISSING state in container ID 7 rec = unHealthyContainersTableHandle.fetchByContainerId(7L).get(0); assertEquals("MISSING", rec.getContainerState()); assertEquals(3, rec.getReplicaDelta().intValue()); @@ -233,9 +243,7 @@ public void testRun() throws Exception { assertThat(taskStatus.getLastUpdatedTimestamp()) .isGreaterThan(currentTime); - // Now run the job again, to check that relevant records are updated or - // removed as appropriate. Need to adjust the return value for all the mocks - // Under replicated -> Delta goes from 2 to 1 + // Adjust the mock results and rerun to check for updates or removal of records when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(1L))) .thenReturn(getMockReplicas(1L, State.CLOSED, State.CLOSED)); @@ -244,7 +252,7 @@ public void testRun() throws Exception { .thenReturn(getMockReplicas(2L, State.CLOSED, State.CLOSED, State.CLOSED)); - // return 0 replicas for container ID 3 -> Still empty Missing + // Container 3 remains EMPTY_MISSING, but no DB insertion when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(3L))) .thenReturn(Collections.emptySet()); @@ -253,11 +261,16 @@ public void testRun() throws Exception { .thenReturn(getMockReplicas(4L, State.CLOSED, State.CLOSED, State.CLOSED, State.CLOSED)); - // Was mis-replicated - make it healthy now + // Convert container 7 which was MISSING to EMPTY_MISSING (not inserted into DB) + when(reconContainerMetadataManager.getKeyCountForContainer( + 7L)).thenReturn(0L); + placementMock.setMisRepWhenDnPresent(null); + // Ensure count is reduced after EMPTY_MISSING containers are not inserted LambdaTestUtils.await(60000, 1000, () -> - (unHealthyContainersTableHandle.count() == 4)); + (unHealthyContainersTableHandle.count() == 2)); + rec = unHealthyContainersTableHandle.fetchByContainerId(1L).get(0); assertEquals("UNDER_REPLICATED", rec.getContainerState()); assertEquals(1, rec.getReplicaDelta().intValue()); @@ -266,36 +279,21 @@ public void testRun() throws Exception { assertEquals(0, unHealthyContainersTableHandle.fetchByContainerId(2L).size()); - rec = unHealthyContainersTableHandle.fetchByContainerId(3L).get(0); - assertEquals("EMPTY_MISSING", rec.getContainerState()); - assertEquals(3, rec.getReplicaDelta().intValue()); - - rec = unHealthyContainersTableHandle.fetchByContainerId(7L).get(0); - assertEquals("MISSING", rec.getContainerState()); - assertEquals(3, rec.getReplicaDelta().intValue()); + // Assert that for container 7 no records exist in DB because it's now EMPTY_MISSING + assertEquals(0, + unHealthyContainersTableHandle.fetchByContainerId(7L).size()); rec = unHealthyContainersTableHandle.fetchByContainerId(4L).get(0); assertEquals("OVER_REPLICATED", rec.getContainerState()); assertEquals(-1, rec.getReplicaDelta().intValue()); - // This container is now healthy, it should not be in the table any more + // Ensure container 5 is now healthy and not in the table assertEquals(0, unHealthyContainersTableHandle.fetchByContainerId(5L).size()); - // Again make container Id 7 as empty which was missing as well, so in next - // container health task run, this container also should be deleted from - // UNHEALTHY_CONTAINERS table because we want to cleanup any existing - // EMPTY and MISSING containers from UNHEALTHY_CONTAINERS table. - when(reconContainerMetadataManager.getKeyCountForContainer(7L)).thenReturn(0L); - LambdaTestUtils.await(6000, 1000, () -> { - UnhealthyContainers emptyMissingContainer = unHealthyContainersTableHandle.fetchByContainerId(7L).get(0); - return ("EMPTY_MISSING".equals(emptyMissingContainer.getContainerState())); - }); - - // Just check once again that count doesn't change, only state of - // container 7 changes from MISSING to EMPTY_MISSING + // Just check once again that count remains consistent LambdaTestUtils.await(60000, 1000, () -> - (unHealthyContainersTableHandle.count() == 4)); + (unHealthyContainersTableHandle.count() == 2)); } @Test @@ -370,17 +368,12 @@ public void testDeletedContainer() throws Exception { reconContainerMetadataManager, new OzoneConfiguration()); containerHealthTask.start(); LambdaTestUtils.await(6000, 1000, () -> - (unHealthyContainersTableHandle.count() == 2)); + (unHealthyContainersTableHandle.count() == 1)); UnhealthyContainers rec = unHealthyContainersTableHandle.fetchByContainerId(1L).get(0); assertEquals("MISSING", rec.getContainerState()); assertEquals(3, rec.getReplicaDelta().intValue()); - rec = - unHealthyContainersTableHandle.fetchByContainerId(3L).get(0); - assertEquals("EMPTY_MISSING", rec.getContainerState()); - assertEquals(3, rec.getReplicaDelta().intValue()); - ReconTaskStatus taskStatus = reconTaskStatusDao.findById(containerHealthTask.getTaskName()); assertThat(taskStatus.getLastUpdatedTimestamp()) @@ -473,64 +466,106 @@ public void testAllContainerStateInsertions() { } @Test - public void testNegativeSizeContainers() throws Exception { - // Setup mock objects and test environment - UnhealthyContainersDao unhealthyContainersDao = + public void testMissingAndEmptyMissingContainerDeletion() throws Exception { + // Setup mock DAOs and managers + UnhealthyContainersDao unHealthyContainersTableHandle = getDao(UnhealthyContainersDao.class); ContainerHealthSchemaManager containerHealthSchemaManager = new ContainerHealthSchemaManager( getSchemaDefinition(ContainerSchemaDefinition.class), - unhealthyContainersDao); + unHealthyContainersTableHandle); ReconStorageContainerManagerFacade scmMock = mock(ReconStorageContainerManagerFacade.class); + MockPlacementPolicy placementMock = new MockPlacementPolicy(); ContainerManager containerManagerMock = mock(ContainerManager.class); StorageContainerServiceProvider scmClientMock = mock(StorageContainerServiceProvider.class); ReconContainerMetadataManager reconContainerMetadataManager = mock(ReconContainerMetadataManager.class); - MockPlacementPolicy placementMock = new MockPlacementPolicy(); + mock(ReconContainerMetadataManager.class); - // Mock container info setup - List mockContainers = getMockContainers(3); - when(scmMock.getContainerManager()).thenReturn(containerManagerMock); + // Create 2 containers. They start in CLOSED state in Recon. + List mockContainers = getMockContainers(2); when(scmMock.getScmServiceProvider()).thenReturn(scmClientMock); + when(scmMock.getContainerManager()).thenReturn(containerManagerMock); when(containerManagerMock.getContainers(any(ContainerID.class), anyInt())).thenReturn(mockContainers); + + // Mark both containers as initially CLOSED in Recon for (ContainerInfo c : mockContainers) { - when(containerManagerMock.getContainer( - c.containerID())).thenReturn(c); - when(scmClientMock.getContainerWithPipeline( - c.getContainerID())).thenReturn(new ContainerWithPipeline(c, null)); - when(containerManagerMock.getContainer(c.containerID()) - .getUsedBytes()).thenReturn(Long.valueOf(-10)); + when(containerManagerMock.getContainer(c.containerID())).thenReturn(c); } - // Verify the table is initially empty - assertThat(unhealthyContainersDao.findAll()).isEmpty(); + // Simulate SCM reporting the containers as DELETED + ContainerInfo deletedContainer1 = getMockDeletedContainer(1); + ContainerInfo deletedContainer2 = getMockDeletedContainer(2); + + when(scmClientMock.getContainerWithPipeline(1)) + .thenReturn(new ContainerWithPipeline(deletedContainer1, null)); + when(scmClientMock.getContainerWithPipeline(2)) + .thenReturn(new ContainerWithPipeline(deletedContainer2, null)); + + // Both containers start as CLOSED in Recon (MISSING or EMPTY_MISSING) + when(containerManagerMock.getContainer(ContainerID.valueOf(1L)).getState()) + .thenReturn(HddsProtos.LifeCycleState.CLOSED); + when(containerManagerMock.getContainer(ContainerID.valueOf(2L)).getState()) + .thenReturn(HddsProtos.LifeCycleState.CLOSED); - // Setup and start the container health task + // Replicas are empty, so both containers should be considered for deletion + when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(1L))) + .thenReturn(Collections.emptySet()); + when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(2L))) + .thenReturn(Collections.emptySet()); + + // Initialize UnhealthyContainers in DB (MISSING and EMPTY_MISSING) + // Create and set up the first UnhealthyContainer for a MISSING container + UnhealthyContainers container1 = new UnhealthyContainers(); + container1.setContainerId(1L); + container1.setContainerState("MISSING"); + container1.setExpectedReplicaCount(3); + container1.setActualReplicaCount(0); + container1.setReplicaDelta(3); + container1.setInStateSince(System.currentTimeMillis()); + + // Create and set up the second UnhealthyContainer for an EMPTY_MISSING container + UnhealthyContainers container2 = new UnhealthyContainers(); + container2.setContainerId(2L); + container2.setContainerState("MISSING"); + container2.setExpectedReplicaCount(3); + container2.setActualReplicaCount(0); + container2.setReplicaDelta(3); + container2.setInStateSince(System.currentTimeMillis()); + + unHealthyContainersTableHandle.insert(container1); + unHealthyContainersTableHandle.insert(container2); + + when(reconContainerMetadataManager.getKeyCountForContainer(1L)).thenReturn(5L); + when(reconContainerMetadataManager.getKeyCountForContainer(2L)).thenReturn(0L); + + // Start the container health task ReconTaskStatusDao reconTaskStatusDao = getDao(ReconTaskStatusDao.class); ReconTaskConfig reconTaskConfig = new ReconTaskConfig(); reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(2)); - ContainerHealthTask containerHealthTask = new ContainerHealthTask( - scmMock.getContainerManager(), scmMock.getScmServiceProvider(), - reconTaskStatusDao, - containerHealthSchemaManager, placementMock, reconTaskConfig, - reconContainerMetadataManager, - new OzoneConfiguration()); - containerHealthTask.start(); + ContainerHealthTask containerHealthTask = + new ContainerHealthTask(scmMock.getContainerManager(), + scmMock.getScmServiceProvider(), + reconTaskStatusDao, containerHealthSchemaManager, + placementMock, reconTaskConfig, + reconContainerMetadataManager, new OzoneConfiguration()); - // Wait for the task to identify unhealthy containers - LambdaTestUtils.await(6000, 1000, - () -> unhealthyContainersDao.count() == 3); + containerHealthTask.start(); - // Assert that all unhealthy containers have been identified as NEGATIVE_SIZE states - List negativeSizeContainers = - unhealthyContainersDao.fetchByContainerState("NEGATIVE_SIZE"); - assertThat(negativeSizeContainers).hasSize(3); + // Wait for the task to complete and ensure that updateContainerState is invoked for + // container IDs 1 and 2 to mark the containers as DELETED, since they are DELETED in SCM. + LambdaTestUtils.await(60000, 1000, () -> { + verify(containerManagerMock, times(1)) + .updateContainerState(ContainerID.valueOf(1L), HddsProtos.LifeCycleEvent.DELETE); + verify(containerManagerMock, times(1)) + .updateContainerState(ContainerID.valueOf(2L), HddsProtos.LifeCycleEvent.DELETE); + return true; + }); } - private Set getMockReplicas( long containerId, State...states) { Set replicas = new HashSet<>(); diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTaskRecordGenerator.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTaskRecordGenerator.java index 7d55e612bad4..4e9965638a17 100644 --- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTaskRecordGenerator.java +++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/fsck/TestContainerHealthTaskRecordGenerator.java @@ -127,6 +127,58 @@ public void testMissingRecordRetained() { )); } + @Test + public void testEmptyMissingRecordNotInsertedButLogged() { + // Create a container that is in EMPTY_MISSING state + Set replicas = new HashSet<>(); + ContainerHealthStatus status = new ContainerHealthStatus(emptyContainer, replicas, placementPolicy, + reconContainerMetadataManager, CONF); + + // Initialize stats map + Map> unhealthyContainerStateStatsMap = new HashMap<>(); + initializeUnhealthyContainerStateStatsMap(unhealthyContainerStateStatsMap); + + // Generate records for EMPTY_MISSING container + List records = ContainerHealthTask.ContainerHealthRecords.generateUnhealthyRecords( + status, (long) 345678, unhealthyContainerStateStatsMap); + + // Assert that no records are created for EMPTY_MISSING state + assertEquals(0, records.size()); + + // Assert that the EMPTY_MISSING state is logged + assertEquals(1, unhealthyContainerStateStatsMap.get(UnHealthyContainerStates.EMPTY_MISSING) + .getOrDefault(CONTAINER_COUNT, 0L)); + } + + @Test + public void testNegativeSizeRecordNotInsertedButLogged() { + // Simulate a container with NEGATIVE_SIZE state + when(container.getUsedBytes()).thenReturn(-10L); // Negative size + Set replicas = generateReplicas(container, CLOSED, CLOSED); + ContainerHealthStatus status = + new ContainerHealthStatus(container, replicas, placementPolicy, reconContainerMetadataManager, CONF); + + // Initialize stats map + Map> + unhealthyContainerStateStatsMap = new HashMap<>(); + initializeUnhealthyContainerStateStatsMap(unhealthyContainerStateStatsMap); + + // Generate records for NEGATIVE_SIZE container + List records = + ContainerHealthTask.ContainerHealthRecords.generateUnhealthyRecords( + status, (long) 123456, unhealthyContainerStateStatsMap); + + // Assert that none of the records are for negative. + records.forEach(record -> assertFalse(record.getContainerState() + .equals(UnHealthyContainerStates.NEGATIVE_SIZE.toString()))); + + + // Assert that the NEGATIVE_SIZE state is logged + assertEquals(1, unhealthyContainerStateStatsMap.get( + UnHealthyContainerStates.NEGATIVE_SIZE).getOrDefault(CONTAINER_COUNT, 0L)); + } + + @Test public void testUnderReplicatedRecordRetainedAndUpdated() { // under replicated container @@ -396,13 +448,9 @@ public void testCorrectRecordsGenerated() { status = new ContainerHealthStatus(emptyContainer, replicas, placementPolicy, reconContainerMetadataManager, CONF); - records = ContainerHealthTask.ContainerHealthRecords + ContainerHealthTask.ContainerHealthRecords .generateUnhealthyRecords(status, (long) 345678, unhealthyContainerStateStatsMap); - assertEquals(1, records.size()); - rec = records.get(0); - assertEquals(UnHealthyContainerStates.EMPTY_MISSING.toString(), - rec.getContainerState()); assertEquals(3, rec.getExpectedReplicaCount().intValue()); assertEquals(0, rec.getActualReplicaCount().intValue()); @@ -582,6 +630,8 @@ private void initializeUnhealthyContainerStateStatsMap( UnHealthyContainerStates.OVER_REPLICATED, new HashMap<>()); unhealthyContainerStateStatsMap.put( UnHealthyContainerStates.MIS_REPLICATED, new HashMap<>()); + unhealthyContainerStateStatsMap.put( + UnHealthyContainerStates.NEGATIVE_SIZE, new HashMap<>()); } private void logUnhealthyContainerStats( @@ -590,7 +640,7 @@ private void logUnhealthyContainerStats( // If any EMPTY_MISSING containers, then it is possible that such // containers got stuck in the closing state which never got // any replicas created on the datanodes. In this case, we log it as - // EMPTY, and insert as EMPTY_MISSING in UNHEALTHY_CONTAINERS table. + // EMPTY_MISSING containers, but dont add it to the unhealthy container table. unhealthyContainerStateStatsMap.entrySet().forEach(stateEntry -> { UnHealthyContainerStates unhealthyContainerState = stateEntry.getKey(); Map containerStateStatsMap = stateEntry.getValue();