Skip to content

Commit

Permalink
HDDS-10370. Recon - Handle the pre-existing missing empty containers …
Browse files Browse the repository at this point in the history
…in clusters. (apache#6255)

(cherry picked from commit e0bf7b4)
Change-Id: I72150151e39f2bfc29d9f39673782c1afd5353ad
  • Loading branch information
devmadhuu authored and jojochuang committed Mar 7, 2024
1 parent b69f6d8 commit 2157700
Show file tree
Hide file tree
Showing 5 changed files with 99 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -69,11 +69,11 @@ public void init() throws Exception {
conf.set(HDDS_PIPELINE_REPORT_INTERVAL, "5s");

ReconTaskConfig taskConfig = conf.getObject(ReconTaskConfig.class);
taskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(15));
taskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(10));
conf.setFromObject(taskConfig);

conf.set("ozone.scm.stale.node.interval", "6s");
conf.set("ozone.scm.dead.node.interval", "10s");
conf.set("ozone.scm.dead.node.interval", "8s");
cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1)
.includeRecon(true).build();
cluster.waitForClusterToBeReady();
Expand Down Expand Up @@ -246,6 +246,7 @@ public void testEmptyMissingContainerDownNode() throws Exception {
return (allEmptyMissingContainers.size() == 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.
try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
Expand Down Expand Up @@ -274,6 +275,26 @@ public void testEmptyMissingContainerDownNode() throws Exception {
return (allEmptyMissingContainers.isEmpty());
});

// Now remove keys from container. This data is used to
// identify if container is empty in terms of keys mapped to container.
try (RDBBatchOperation rdbBatchOperation = new RDBBatchOperation()) {
reconContainerMetadataManager
.batchStoreContainerKeyCounts(rdbBatchOperation, containerID, 0L);
reconContainerMetadataManager.commitBatchOperation(rdbBatchOperation);
}

// Check existing container state in UNHEALTHY_CONTAINER table
// will be updated as EMPTY_MISSING
LambdaTestUtils.await(25000, 1000, () -> {
List<UnhealthyContainers> allEmptyMissingContainers =
reconContainerManager.getContainerSchemaManager()
.getUnhealthyContainers(
ContainerSchemaDefinition.UnHealthyContainerStates.
EMPTY_MISSING,
0, 1000);
return (allEmptyMissingContainers.size() == 1);
});

// Now restart the cluster and verify the container is no longer missing.
cluster.restartHddsDatanode(pipeline.getFirstNode(), true);
LambdaTestUtils.await(25000, 1000, () -> {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -397,6 +397,9 @@ public Response getUnhealthyContainers(
summary = containerHealthSchemaManager.getUnhealthyContainersSummary();
List<UnhealthyContainers> containers = containerHealthSchemaManager
.getUnhealthyContainers(internalState, offset, limit);
containers.stream()
.filter(
container -> !container.getContainerState().equals(UnHealthyContainerStates.EMPTY_MISSING.toString()));
for (UnhealthyContainers c : containers) {
long containerID = c.getContainerId();
ContainerInfo containerInfo =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,6 +57,7 @@
import static org.apache.hadoop.ozone.recon.ReconConstants.DEFAULT_FETCH_COUNT;
import static org.apache.hadoop.ozone.recon.ReconConstants.TOTAL_KEYS;
import static org.apache.hadoop.ozone.recon.ReconConstants.TOTAL_USED_BYTES;
import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.EMPTY_MISSING;


/**
Expand Down Expand Up @@ -295,6 +296,8 @@ private long processExistingDBRecords(long currentTime,
rec.update();
}
} else {
LOG.info("DELETED existing unhealthy container record...for Container: {}",
currentContainer.getContainerID());
rec.delete();
}
} catch (ContainerNotFoundException cnf) {
Expand Down Expand Up @@ -430,7 +433,7 @@ public static boolean retainOrUpdateRecord(
boolean returnValue = false;
switch (UnHealthyContainerStates.valueOf(rec.getContainerState())) {
case MISSING:
returnValue = container.isMissing();
returnValue = container.isMissing() && !container.isEmpty();
break;
case MIS_REPLICATED:
returnValue = keepMisReplicatedRecord(container, rec);
Expand Down Expand Up @@ -495,10 +498,10 @@ public static List<UnhealthyContainers> generateUnhealthyRecords(
"starting with **Container State Stats:**");
}
records.add(
recordForState(container, UnHealthyContainerStates.EMPTY_MISSING,
recordForState(container, EMPTY_MISSING,
time));
populateContainerStats(container,
UnHealthyContainerStates.EMPTY_MISSING,
EMPTY_MISSING,
unhealthyContainerStateStatsMap);
}
// A container cannot have any other records if it is missing so return
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,13 +34,18 @@
import org.jooq.DSLContext;
import org.jooq.Record;
import org.jooq.SelectQuery;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.util.List;

/**
* Provide a high level API to access the Container Schema.
*/
@Singleton
public class ContainerHealthSchemaManager {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerHealthSchemaManager.class);

private final UnhealthyContainersDao unhealthyContainersDao;
private final ContainerSchemaDefinition containerSchemaDefinition;
Expand Down Expand Up @@ -113,6 +118,12 @@ public Cursor<UnhealthyContainersRecord> getAllUnhealthyRecordsCursor() {
}

public void insertUnhealthyContainerRecords(List<UnhealthyContainers> recs) {
if (LOG.isDebugEnabled()) {
recs.forEach(rec -> {
LOG.debug("rec.getContainerId() : {}, rec.getContainerState(): {} ", rec.getContainerId(),
rec.getContainerState());
});
}
unhealthyContainersDao.insert(recs);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hadoop.ozone.recon.fsck;

import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE;
import static org.assertj.core.api.Assertions.assertThat;
import static org.hadoop.ozone.recon.schema.ContainerSchemaDefinition.UnHealthyContainerStates.ALL_REPLICAS_UNHEALTHY;
import static org.junit.jupiter.api.Assertions.assertEquals;
Expand All @@ -38,6 +39,7 @@
import java.util.UUID;

import org.apache.hadoop.hdds.client.RatisReplicationConfig;
import org.apache.hadoop.hdds.client.ReplicatedReplicationConfig;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.MockDatanodeDetails;
Expand All @@ -49,6 +51,7 @@
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.TestContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementStatusDefault;
import org.apache.hadoop.ozone.recon.persistence.ContainerHealthSchemaManager;
Expand Down Expand Up @@ -110,38 +113,61 @@ public void testRun() throws Exception {
when(scmClientMock.getContainerWithPipeline(c.getContainerID()))
.thenReturn(new ContainerWithPipeline(c, null));
}

ReplicatedReplicationConfig replicationConfig = RatisReplicationConfig.getInstance(THREE);
// Under replicated
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(1L)))
ContainerInfo containerInfo1 =
TestContainerInfo.newBuilderForTest().setContainerID(1).setReplicationConfig(replicationConfig).build();
when(containerManagerMock.getContainer(ContainerID.valueOf(1L))).thenReturn(containerInfo1);
when(containerManagerMock.getContainerReplicas(containerInfo1.containerID()))
.thenReturn(getMockReplicas(1L, State.CLOSED, State.UNHEALTHY));

// return all UNHEALTHY replicas for container ID 2 -> UNDER_REPLICATED
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(2L)))
ContainerInfo containerInfo2 =
TestContainerInfo.newBuilderForTest().setContainerID(2).setReplicationConfig(replicationConfig).build();
when(containerManagerMock.getContainer(ContainerID.valueOf(2L))).thenReturn(containerInfo2);
when(containerManagerMock.getContainerReplicas(containerInfo2.containerID()))
.thenReturn(getMockReplicas(2L, State.UNHEALTHY));

// return 0 replicas for container ID 3 -> Missing
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(3L)))
// return 0 replicas for container ID 3 -> Empty Missing
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
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(4L)))
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));

// Mis-replicated
ContainerInfo containerInfo5 =
TestContainerInfo.newBuilderForTest().setContainerID(5).setReplicationConfig(replicationConfig).build();
when(containerManagerMock.getContainer(ContainerID.valueOf(5L))).thenReturn(containerInfo5);
Set<ContainerReplica> misReplicas = getMockReplicas(5L,
State.CLOSED, State.CLOSED, State.CLOSED);
placementMock.setMisRepWhenDnPresent(
misReplicas.iterator().next().getDatanodeDetails().getUuid());
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(5L)))
when(containerManagerMock.getContainerReplicas(containerInfo5.containerID()))
.thenReturn(misReplicas);

// Return 3 Healthy -> Healthy container
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(6L)))
ContainerInfo containerInfo6 =
TestContainerInfo.newBuilderForTest().setContainerID(6).setReplicationConfig(replicationConfig).build();
when(containerManagerMock.getContainer(ContainerID.valueOf(6L))).thenReturn(containerInfo6);
when(containerManagerMock.getContainerReplicas(containerInfo6.containerID()))
.thenReturn(getMockReplicas(6L,
State.CLOSED, State.CLOSED, State.CLOSED));

// return 0 replicas for container ID 7 -> EMPTY_MISSING
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(7L)))
// return 0 replicas for container ID 7 -> MISSING
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());

List<UnhealthyContainers> all = unHealthyContainersTableHandle.findAll();
Expand All @@ -150,7 +176,7 @@ public void testRun() throws Exception {
long currentTime = System.currentTimeMillis();
ReconTaskStatusDao reconTaskStatusDao = getDao(ReconTaskStatusDao.class);
ReconTaskConfig reconTaskConfig = new ReconTaskConfig();
reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(2));
reconTaskConfig.setMissingContainerTaskInterval(Duration.ofSeconds(5));
when(reconContainerMetadataManager.getKeyCountForContainer(
7L)).thenReturn(5L);
ContainerHealthTask containerHealthTask =
Expand Down Expand Up @@ -215,7 +241,7 @@ public void testRun() throws Exception {
.thenReturn(getMockReplicas(2L,
State.CLOSED, State.CLOSED, State.CLOSED));

// return 0 replicas for container ID 3 -> Still Missing
// return 0 replicas for container ID 3 -> Still empty Missing
when(containerManagerMock.getContainerReplicas(ContainerID.valueOf(3L)))
.thenReturn(Collections.emptySet());

Expand All @@ -227,7 +253,7 @@ public void testRun() throws Exception {
// Was mis-replicated - make it healthy now
placementMock.setMisRepWhenDnPresent(null);

LambdaTestUtils.await(6000, 1000, () ->
LambdaTestUtils.await(60000, 1000, () ->
(unHealthyContainersTableHandle.count() == 4));
rec = unHealthyContainersTableHandle.fetchByContainerId(1L).get(0);
assertEquals("UNDER_REPLICATED", rec.getContainerState());
Expand All @@ -252,6 +278,21 @@ public void testRun() throws Exception {
// This container is now healthy, it should not be in the table any more
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
LambdaTestUtils.await(60000, 1000, () ->
(unHealthyContainersTableHandle.count() == 4));
}

@Test
Expand Down Expand Up @@ -423,9 +464,9 @@ private List<ContainerInfo> getMockContainers(int num) {
when(c.getContainerID()).thenReturn((long)i);
when(c.getReplicationConfig())
.thenReturn(RatisReplicationConfig.getInstance(
HddsProtos.ReplicationFactor.THREE));
THREE));
when(c.getReplicationFactor())
.thenReturn(HddsProtos.ReplicationFactor.THREE);
.thenReturn(THREE);
when(c.getState()).thenReturn(HddsProtos.LifeCycleState.CLOSED);
when(c.containerID()).thenReturn(ContainerID.valueOf(i));
containers.add(c);
Expand All @@ -438,7 +479,7 @@ private ContainerInfo getMockDeletedContainer(int containerID) {
when(c.getContainerID()).thenReturn((long)containerID);
when(c.getReplicationConfig())
.thenReturn(RatisReplicationConfig
.getInstance(HddsProtos.ReplicationFactor.THREE));
.getInstance(THREE));
when(c.containerID()).thenReturn(ContainerID.valueOf(containerID));
when(c.getState()).thenReturn(HddsProtos.LifeCycleState.DELETED);
return c;
Expand Down

0 comments on commit 2157700

Please sign in to comment.