Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-11389. Incorrect number of deleted containers shown in Recon UI. #7149

Merged
merged 14 commits into from
Sep 11, 2024
Merged
Show file tree
Hide file tree
Changes from 12 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -236,14 +236,16 @@ 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<UnhealthyContainers> allEmptyMissingContainers =
reconContainerManager.getContainerSchemaManager()
.getUnhealthyContainers(
ContainerSchemaDefinition.UnHealthyContainerStates.
EMPTY_MISSING,
0, 1000);
return (allEmptyMissingContainers.size() == 1);
return (allEmptyMissingContainers.size() == 0);
});


Expand Down Expand Up @@ -283,16 +285,16 @@ 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<UnhealthyContainers> allEmptyMissingContainers =
reconContainerManager.getContainerSchemaManager()
.getUnhealthyContainers(
ContainerSchemaDefinition.UnHealthyContainerStates.
EMPTY_MISSING,
0, 1000);
return (allEmptyMissingContainers.size() == 1);
return (allEmptyMissingContainers.size() == 0);
});

// Now restart the cluster and verify the container is no longer missing.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down Expand Up @@ -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.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -408,13 +408,18 @@ public Response getUnhealthyContainers(
summary = containerHealthSchemaManager.getUnhealthyContainersSummary();
List<UnhealthyContainers> containers = containerHealthSchemaManager
.getUnhealthyContainers(internalState, offset, limit);
List<UnhealthyContainers> 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<UnhealthyContainers> 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));
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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.
*/
Expand Down Expand Up @@ -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;
}
}

Expand All @@ -105,10 +96,6 @@ public long getMisReplicatedCount() {
return misReplicatedCount;
}

public long getNegativeSizeCount() {
return negativeSizeCount;
}

public Collection<UnhealthyContainerMetadata> getContainers() {
return containers;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -188,7 +189,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 in unhealthy container statistics but do not add it to the table.
unhealthyContainerStateStatsMap.entrySet().forEach(stateEntry -> {
UnHealthyContainerStates unhealthyContainerState = stateEntry.getKey();
Map<String, Long> containerStateStatsMap = stateEntry.getValue();
Expand Down Expand Up @@ -256,6 +257,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
Expand All @@ -273,34 +279,41 @@ 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,
unhealthyContainerStateCountMap);
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: {}",
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Move this log after rec.delete

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done!

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()) {
ArafatKhan2198 marked this conversation as resolved.
Show resolved Hide resolved
rec.update();
}
} catch (ContainerNotFoundException cnf) {
// If the container is not found, delete the record and reset currentContainer
rec.delete();
currentContainer = null;
}
Expand All @@ -326,13 +339,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;
}
Expand All @@ -349,6 +355,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 =
Expand All @@ -358,13 +376,16 @@ 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())
.size() == 0
) {
containerManager.updateContainerState(containerInfo.containerID(),
HddsProtos.LifeCycleEvent.CLEANUP);
LOG.info("Successfully Deleted container {} from Recon.", containerInfo.containerID());
}
return true;
}
Expand All @@ -380,28 +401,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<UnHealthyContainerStates, Map<String, Long>>
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,
sumitagrawl marked this conversation as resolved.
Show resolved Hide resolved
UnHealthyContainerStates.NEGATIVE_SIZE, currentTime);
List<UnhealthyContainers> 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<UnHealthyContainerStates, Map<String, Long>>
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);
}

/**
Expand Down Expand Up @@ -492,22 +535,21 @@ public static List<UnhealthyContainers> 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())) {
Expand Down Expand Up @@ -650,4 +692,15 @@ private static void populateContainerStats(
(value + container.getContainer().getUsedBytes()));
}
}

/**
* Expose the logger for testing purposes.
*
* @return the logger instance
*/
@VisibleForTesting
public Logger getLogger() {
return LOG;
}

}
Loading
Loading