errose28 commented on code in PR #7065:
URL: https://github.com/apache/ozone/pull/7065#discussion_r1729266600
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java:
##########
@@ -337,12 +345,9 @@ public void
testCloseInvalidContainer(ContainerLayoutVersion layoutVersion)
.build();
dispatcher.dispatch(closeContainerRequest, null);
- when(handler.handleCloseContainer(any(), any()))
- .thenCallRealMethod();
- doCallRealMethod().when(handler).closeContainer(any());
// Closing invalid container should return error response.
ContainerProtos.ContainerCommandResponseProto response =
- handler.handleCloseContainer(closeContainerRequest, container);
+ keyValueHandler.handleCloseContainer(closeContainerRequest, container);
Review Comment:
After this we should check that the tree file is present, since it looks
like it will get generated even on this invalid state transition which I think
is fine.
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java:
##########
@@ -218,22 +227,31 @@ void testNPEFromPutBlock() throws IOException {
@Test
public void testMarkContainerUnhealthyInFailedVolume() throws IOException {
KeyValueHandler handler = getDummyHandler();
- KeyValueContainerData mockContainerData =
mock(KeyValueContainerData.class);
- HddsVolume mockVolume = mock(HddsVolume.class);
- when(mockContainerData.getVolume()).thenReturn(mockVolume);
+ OzoneConfiguration conf = new OzoneConfiguration();
+ KeyValueContainerData kvData = new KeyValueContainerData(1L,
+ ContainerLayoutVersion.FILE_PER_BLOCK,
+ (long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
+ UUID.randomUUID().toString());
+ kvData.setMetadataPath(tempDir.toString());
+ kvData.setDbFile(dbFile.toFile());
+ HddsVolume hddsVolume = new
HddsVolume.Builder(tempDir.toString()).conf(conf)
+
.clusterID(UUID.randomUUID().toString()).datanodeUuid(UUID.randomUUID().toString())
+ .volumeSet(mock(MutableVolumeSet.class))
+ .build();
+ kvData.setVolume(hddsVolume);
KeyValueContainer container = new KeyValueContainer(
- mockContainerData, new OzoneConfiguration());
+ kvData, new OzoneConfiguration());
// When volume is failed, the call to mark the container unhealthy should
// be ignored.
- when(mockVolume.isFailed()).thenReturn(true);
+ hddsVolume.setState(StorageVolume.VolumeState.FAILED);
Review Comment:
In this case we should probably not bother generating the tree file and the
test should check this. Volume failure supersedes container failures.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +183,95 @@ public void testCloseClosedContainer()
StorageContainerManager scm = cluster.getStorageContainerManager();
// Pick any container on the cluster and close it via client
ContainerInfo container = scm.getContainerManager().getContainers().get(0);
+ // Checksum file doesn't exist before container close
+ List<HddsDatanodeService> hddsDatanodes = cluster.getHddsDatanodes();
+ for (HddsDatanodeService hddsDatanode: hddsDatanodes) {
+ assertFalse(containerChecksumFileExists(hddsDatanode, container));
+ }
+ // Close container
OzoneTestUtils.closeContainer(scm, container);
+
+ // Checksum file exists after container close
+ for (HddsDatanodeService hddsDatanode: hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
checkContainerCloseInDatanode(hddsDatanode, container), 100, 5000);
+ assertTrue(containerChecksumFileExists(hddsDatanode, container));
+ }
+
assertThrows(IOException.class,
() -> cluster.getStorageContainerLocationClient()
.closeContainer(container.getContainerID()),
"Container " + container.getContainerID() + " already closed");
}
+ @Test
+ public void testContainerChecksumForClosedContainer() throws Exception {
+ // Create some keys to write data into the open containers
+ TestDataUtil.createKey(bucket, "key1", ReplicationFactor.THREE,
+ ReplicationType.RATIS, "this is the content");
+ StorageContainerManager scm = cluster.getStorageContainerManager();
+
+ ContainerInfo containerInfo1 =
scm.getContainerManager().getContainers().get(0);
+ // Checksum file doesn't exist before container close
+ List<HddsDatanodeService> hddsDatanodes = cluster.getHddsDatanodes();
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ assertFalse(containerChecksumFileExists(hddsDatanode, containerInfo1));
+ }
+ // Close container.
+ OzoneTestUtils.closeContainer(scm, containerInfo1);
+ ContainerProtos.ContainerChecksumInfo prevExpectedChecksumInfo1 = null;
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
checkContainerCloseInDatanode(hddsDatanode, containerInfo1), 100, 5000);
+ assertTrue(containerChecksumFileExists(hddsDatanode, containerInfo1));
+ OzoneContainer ozoneContainer =
hddsDatanode.getDatanodeStateMachine().getContainer();
+ Container<?> container1 =
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo containerChecksumInfo =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
Review Comment:
We can overload `ContainerMerkleTreeTestUtils#readChecksumFile` with
parameters similar to
`ContainerMerkleTreeTestUtils#containerChecksumFileExists` to avoid having this
big block twice in the test.
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandlerWithUnhealthyContainer.java:
##########
@@ -218,22 +227,31 @@ void testNPEFromPutBlock() throws IOException {
@Test
public void testMarkContainerUnhealthyInFailedVolume() throws IOException {
KeyValueHandler handler = getDummyHandler();
- KeyValueContainerData mockContainerData =
mock(KeyValueContainerData.class);
- HddsVolume mockVolume = mock(HddsVolume.class);
- when(mockContainerData.getVolume()).thenReturn(mockVolume);
+ OzoneConfiguration conf = new OzoneConfiguration();
+ KeyValueContainerData kvData = new KeyValueContainerData(1L,
+ ContainerLayoutVersion.FILE_PER_BLOCK,
+ (long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
+ UUID.randomUUID().toString());
+ kvData.setMetadataPath(tempDir.toString());
+ kvData.setDbFile(dbFile.toFile());
+ HddsVolume hddsVolume = new
HddsVolume.Builder(tempDir.toString()).conf(conf)
+
.clusterID(UUID.randomUUID().toString()).datanodeUuid(UUID.randomUUID().toString())
+ .volumeSet(mock(MutableVolumeSet.class))
+ .build();
+ kvData.setVolume(hddsVolume);
KeyValueContainer container = new KeyValueContainer(
- mockContainerData, new OzoneConfiguration());
+ kvData, new OzoneConfiguration());
// When volume is failed, the call to mark the container unhealthy should
// be ignored.
- when(mockVolume.isFailed()).thenReturn(true);
+ hddsVolume.setState(StorageVolume.VolumeState.FAILED);
handler.markContainerUnhealthy(container,
ContainerTestUtils.getUnhealthyScanResult());
verify(mockIcrSender, never()).send(any());
// When volume is healthy, ICR should be sent when container is marked
// unhealthy.
- when(mockVolume.isFailed()).thenReturn(false);
+ hddsVolume.setState(StorageVolume.VolumeState.NORMAL);
Review Comment:
In this part of the test we can check that the tree file is generated for
the unhealthy container on the healthy volume.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +183,95 @@ public void testCloseClosedContainer()
StorageContainerManager scm = cluster.getStorageContainerManager();
// Pick any container on the cluster and close it via client
ContainerInfo container = scm.getContainerManager().getContainers().get(0);
+ // Checksum file doesn't exist before container close
+ List<HddsDatanodeService> hddsDatanodes = cluster.getHddsDatanodes();
+ for (HddsDatanodeService hddsDatanode: hddsDatanodes) {
+ assertFalse(containerChecksumFileExists(hddsDatanode, container));
+ }
+ // Close container
OzoneTestUtils.closeContainer(scm, container);
+
+ // Checksum file exists after container close
+ for (HddsDatanodeService hddsDatanode: hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
checkContainerCloseInDatanode(hddsDatanode, container), 100, 5000);
+ assertTrue(containerChecksumFileExists(hddsDatanode, container));
+ }
+
assertThrows(IOException.class,
() -> cluster.getStorageContainerLocationClient()
.closeContainer(container.getContainerID()),
"Container " + container.getContainerID() + " already closed");
}
+ @Test
+ public void testContainerChecksumForClosedContainer() throws Exception {
+ // Create some keys to write data into the open containers
+ TestDataUtil.createKey(bucket, "key1", ReplicationFactor.THREE,
+ ReplicationType.RATIS, "this is the content");
+ StorageContainerManager scm = cluster.getStorageContainerManager();
+
+ ContainerInfo containerInfo1 =
scm.getContainerManager().getContainers().get(0);
+ // Checksum file doesn't exist before container close
+ List<HddsDatanodeService> hddsDatanodes = cluster.getHddsDatanodes();
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ assertFalse(containerChecksumFileExists(hddsDatanode, containerInfo1));
+ }
+ // Close container.
+ OzoneTestUtils.closeContainer(scm, containerInfo1);
+ ContainerProtos.ContainerChecksumInfo prevExpectedChecksumInfo1 = null;
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
checkContainerCloseInDatanode(hddsDatanode, containerInfo1), 100, 5000);
+ assertTrue(containerChecksumFileExists(hddsDatanode, containerInfo1));
+ OzoneContainer ozoneContainer =
hddsDatanode.getDatanodeStateMachine().getContainer();
+ Container<?> container1 =
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo containerChecksumInfo =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+ if (prevExpectedChecksumInfo1 != null) {
Review Comment:
Need this extra assertion here and for the same block later in this method.
Otherwise the test could still pass if `readChecksumFile` returns null every
time.
```suggestion
assertNotNull(containerChecksumInfo);
if (prevExpectedChecksumInfo1 != null) {
```
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeTestUtils.java:
##########
@@ -120,4 +123,15 @@ public static ContainerProtos.ContainerChecksumInfo
readChecksumFile(ContainerDa
return ContainerProtos.ContainerChecksumInfo.parseFrom(inStream);
}
}
+
+ /**
+ * This function checks whether the container checksum file exists.
+ */
+ public static boolean containerChecksumFileExists(HddsDatanodeService
hddsDatanode,
+ ContainerInfo
containerInfo) {
+ OzoneContainer ozoneContainer =
hddsDatanode.getDatanodeStateMachine().getContainer();
+ Container container =
ozoneContainer.getController().getContainer(containerInfo.getContainerID());
+ File containerChecksumFile =
ContainerChecksumTreeManager.getContainerChecksumFile(container.getContainerData());
+ return containerChecksumFile.exists();
Review Comment:
`ContainerChecksumTreeManager#checksumFileExists` can be static and called
here instead.
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/keyvalue/TestKeyValueHandler.java:
##########
@@ -488,4 +493,28 @@ private static ContainerCommandRequestProto
createContainerRequest(
.setContainerID(containerID).setPipelineID(UUID.randomUUID().toString())
.build();
}
+
+ private KeyValueHandler createKeyValueHandler() throws IOException {
+ final String clusterId = UUID.randomUUID().toString();
+ final String datanodeId = UUID.randomUUID().toString();
+ final ConfigurationSource conf = new OzoneConfiguration();
+ final ContainerSet containerSet = new ContainerSet(1000);
+ final MutableVolumeSet volumeSet = mock(MutableVolumeSet.class);
+
+ HddsVolume hddsVolume = new
HddsVolume.Builder(tempDir.toString()).conf(conf)
+ .clusterID(clusterId).datanodeUuid(datanodeId)
+ .volumeSet(volumeSet)
+ .build();
+ hddsVolume.format(clusterId);
+ hddsVolume.createWorkingDir(clusterId, null);
+ hddsVolume.createTmpDirs(clusterId);
+
when(volumeSet.getVolumesList()).thenReturn(Collections.singletonList(hddsVolume));
+ final ContainerMetrics metrics = ContainerMetrics.create(conf);
+ final AtomicInteger icrReceived = new AtomicInteger(0);
+ final KeyValueHandler kvHandler = new KeyValueHandler(conf,
+ datanodeId, containerSet, volumeSet, metrics,
+ c -> icrReceived.incrementAndGet(), new
ContainerChecksumTreeManager(conf));
Review Comment:
These lines can be replaced with [this helper
method](https://github.com/apache/ozone/blob/3355701853c10f47c0acafbef502a3d73e48926a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/ContainerTestUtils.java#L205)
to avoid redundant placeholder variables.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]