Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 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 @@ -160,6 +160,10 @@ public boolean isEmpty() {
return numKeys == 0;
}

public boolean isEmptyMissing() {
return numKeys == 0 && numReplicas == 0;
}

private ContainerPlacementStatus getPlacementStatus(
PlacementPolicy policy, int repFactor) {
List<DatanodeDetails> dns = healthyReplicas.stream()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,6 +256,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 +278,44 @@ 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, EMPTY_MISSING, UNDER_REPLICATED,
// OVER_REPLICATED, MIS_REPLICATED can have their unhealthy states changed or retained.
if (!ContainerHealthRecords.retainOrUpdateRecord(currentContainer, rec)) {
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());
rec.delete();
continue;
Copy link
Contributor

Choose a reason for hiding this comment

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

This can be removed because we want to proceed ahead for other unhealthy states. IMO once we remove EMPTY_MISSING switch case from there, we don't need this.

}

// If the container is marked as MISSING and it's deleted in SCM, remove the record
if (currentContainer.isMissing() && containerDeletedInSCM(currentContainer.getContainer())) {
rec.delete();
}

// If the container is in the EMPTY_MISSING state, delete the record
if (currentContainer.isEmptyMissing()) {
rec.delete();
}
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;
}
Expand Down Expand Up @@ -349,6 +364,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 +385,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 Down Expand Up @@ -435,6 +465,9 @@ public static boolean retainOrUpdateRecord(
case MISSING:
returnValue = container.isMissing() && !container.isEmpty();
break;
case EMPTY_MISSING:
Copy link
Contributor

Choose a reason for hiding this comment

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

If we were never storing NEGATIVE_SIZE containers in UnhealthyContainers table, then what was the motive behind adding this in UnHealthyContainerStates enum eralier ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for pointing this out @devmadhuu

This change of Negative Sized was added by this :- #6178
The Negative sized containers do get added to the table, but they do have a bug in their implementation :-

Iteration 1 of ContainerHealthTask:

  • What happens: The checkAndProcessContainers method fetches all containers, including the ones with negative sizes, since processedContainers is empty initially.
  • Result: The negative-sized containers are processed and added to the UNHEALTHY_CONTAINERS table by handleNegativeSizedContainers.

Iteration 2:

  • What happens: The processExistingDBRecords method processes the previously added negative-sized containers. When it checks the records in the table, retainOrUpdateRecord returns false for negative-sized containers because the logic for keeping negative-sized records isn't present.
  • Result: The negative-sized containers are deleted from the UNHEALTHY_CONTAINERS table, and the container is added to processedContainers. As a result, checkAndProcessContainers skips this container because it’s in the processedContainers set.

Iteration 3:

  • What happens: In this iteration, since there are no records for negative-sized containers in the UNHEALTHY_CONTAINERS table (due to their deletion in the previous iteration), the processExistingDBRecords method does not delete anything.
  • Result: The negative-sized containers are not present in the table, so checkAndProcessContainers processes them again and adds them back to the UNHEALTHY_CONTAINERS table.

Iteration 4:

  • What happens: The process from Iteration 2 is repeated, where the processExistingDBRecords method deletes the negative-sized containers again because retainOrUpdateRecord still returns false.
  • Result: The container gets deleted again, and the loop continues.

Conclusion:

This behaviour creates a cycle where the negative-sized containers are added in one iteration and deleted in the next. This is due to the absence of logic in generateUnhealthyRecords to retain negative-sized containers. Adding proper handling for negative-sized containers in generateUnhealthyRecords should prevent this cycle.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks @ArafatKhan2198 for your explanation above. My point is do we need to insert NEGATIVE_SIZE and EMPTY_MISSING containers in UnhealthyContainers table because they are anyway getting deleted in next iteration. There can be two ways to handle this:

  1. Either DO NOT insert both these types containers in UnhealthyContainers table because they are getting deleted anyways in next iteration and we have logs to know the information on current count of both these types of containers with each task run. You can check the correctness of information in logs. This will make things simpler because no need to take care of constraint violation exception in case user moves to next higher version. And any existing MISSING type old containers which have actually zero keys needs to be cleaned up, so that Recon will not report such empty missing containers.
  2. Or, Insert both these types of containers in UnhealthyContainers table which makes things little complex in terms of management of this data which is not being shown over Recon UI. Because such containers will continue to be lying in UnhealthyContainers table unless until we introduce some mechanism to clean up such containers periodically and then when and how to do cleanup of such containers makes things little overly complex.

By having solution 1, we also need to make sure about not to miss the corner edge case of updating container states of such containers after verification from SCM.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I agree @devmadhuu with the first solution! Based on my experience with customer issues, backward compatibility can be a significant challenge. We are anyways printing out the logs for it so no point in storing them.

Copy link
Contributor

Choose a reason for hiding this comment

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

@ArafatKhan2198 @devmadhuu
There is one more changes done, rename of one filed ALL_REPLICAS_BAD which also can cause compatibility issue.
IMO, current we can re-create constraint dropping and re-creating for this.

But we need support Schema upgrade feature for compatibility scenario for this DB.

Copy link
Contributor

Choose a reason for hiding this comment

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

Pls explain the reason why we need to add this explicit EMPTY_MISSING switch case here because anyway we are deleting EMPTY_MISSING later at line here

returnValue = container.isMissing() && container.isEmpty();
break;
case MIS_REPLICATED:
returnValue = keepMisReplicatedRecord(container, rec);
break;
Expand Down Expand Up @@ -495,7 +528,7 @@ public static List<UnhealthyContainers> generateUnhealthyRecords(

LOG.debug("Empty container {} is missing. Kindly check the " +
"consolidated container stats per UNHEALTHY state logged as " +
"starting with **Container State Stats:**");
"starting with **Container State Stats:**", container.getContainerID());

records.add(
recordForState(container, EMPTY_MISSING,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@
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_BAD;
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.times;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertNull;
Expand All @@ -31,6 +33,7 @@
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;


import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
Expand Down Expand Up @@ -530,6 +533,106 @@ public void testNegativeSizeContainers() throws Exception {
assertThat(negativeSizeContainers).hasSize(3);
}

@Test
public void testMissingAndEmptyMissingContainerDeletion() throws Exception {
// Setup mock DAOs and managers
UnhealthyContainersDao unHealthyContainersTableHandle =
getDao(UnhealthyContainersDao.class);
ContainerHealthSchemaManager containerHealthSchemaManager =
new ContainerHealthSchemaManager(
getSchemaDefinition(ContainerSchemaDefinition.class),
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);
mock(ReconContainerMetadataManager.class);

// Create 2 containers. They start in CLOSED state in Recon.
List<ContainerInfo> 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);
}

// 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);

// 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("EMPTY_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();

// 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<ContainerReplica> getMockReplicas(
long containerId, State...states) {
Expand Down