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 all 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 @@ -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;
Expand Down Expand Up @@ -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;

Expand Down Expand Up @@ -236,16 +238,27 @@ 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);
});

// Check if EMPTY_MISSING containers are not added to the DB and their count is logged
Map<ContainerSchemaDefinition.UnHealthyContainerStates, Map<String, Long>>
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.
Expand All @@ -272,7 +285,17 @@ public void testEmptyMissingContainerDownNode() throws Exception {
ContainerSchemaDefinition.UnHealthyContainerStates.
EMPTY_MISSING,
0, 1000);
return (allEmptyMissingContainers.isEmpty());


Map<ContainerSchemaDefinition.UnHealthyContainerStates, Map<String, Long>>
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;
Copy link
Contributor

Choose a reason for hiding this comment

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

Shouldn't we check the CONTAINER_COUNT for MISSING state after we added keys in container ? We did already for EMPTY_MISSING above.. Let's do in next PR.

});

// Now remove keys from container. This data is used to
Expand All @@ -283,16 +306,25 @@ 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);

Map<ContainerSchemaDefinition.UnHealthyContainerStates, Map<String, Long>>
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.
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
Loading