errose28 commented on code in PR #7065:
URL: https://github.com/apache/ozone/pull/7065#discussion_r1723680978
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -116,8 +129,18 @@ public void
testReplicasAreReportedForClosedContainerAfterRestart()
// Pick any container on the cluster, get its pipeline, close it and then
// wait for the container to close
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));
+ }
OzoneTestUtils.closeContainer(scm, container);
+ // Checksum file exists after container close
+ for (HddsDatanodeService hddsDatanode: hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
waitForContainerChecksumToExist(hddsDatanode, container), 100, 5000);
+ }
Review Comment:
`OzoneTestUtils#closeContainer` waits for the container to close on SCM. It
would be better to explicitly wait for the container to close on the datanode
here, then check that the checksum file exists. This ensures that the file is
written synchronously on close, which is a stronger guarantee than the test
currently provides.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +182,98 @@ 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(() ->
waitForContainerChecksumToExist(hddsDatanode, container), 100, 5000);
+ }
+
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);
+ // Build expected merkle tree for one datanode.
+ HddsDatanodeService hddsDatanodeService =
cluster.getHddsDatanodes().get(0);
+ OzoneContainer ozoneContainer =
hddsDatanodeService.getDatanodeStateMachine().getContainer();
+ KeyValueContainer container1 = (KeyValueContainer)
ozoneContainer.getController()
+ .getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo expectedChecksumInfo1 =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
Review Comment:
The file may not exist at this point, since we didn't wait for datanodes to
close the container yet. Same for the next container in the test.
##########
hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/checksum/ContainerMerkleTreeTestUtils.java:
##########
@@ -87,8 +91,8 @@ public static void
assertTreesSortedAndMatch(ContainerProtos.ContainerMerkleTree
* "bytesPerChecksum" amount of data and are assumed to be contiguous.
* @return The ChunkInfo proto object built from this information.
*/
- public static ChunkInfo buildChunk(ConfigurationSource config, int
indexInBlock, ByteBuffer... chunkChecksums)
- throws IOException {
+ public static ContainerProtos.ChunkInfo buildChunk(ConfigurationSource
config, int indexInBlock,
+ ByteBuffer...
chunkChecksums) {
Review Comment:
nit. Better to indent this and the other change 4 spaces below the method
like others in this class. Technically checkstyle doesn't complain though.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +182,98 @@ 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(() ->
waitForContainerChecksumToExist(hddsDatanode, container), 100, 5000);
+ }
+
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);
+ // Build expected merkle tree for one datanode.
+ HddsDatanodeService hddsDatanodeService =
cluster.getHddsDatanodes().get(0);
+ OzoneContainer ozoneContainer =
hddsDatanodeService.getDatanodeStateMachine().getContainer();
+ KeyValueContainer container1 = (KeyValueContainer)
ozoneContainer.getController()
+ .getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo expectedChecksumInfo1 =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
waitForContainerChecksumToExist(hddsDatanode, containerInfo1), 100, 5000);
+ container1 = (KeyValueContainer)
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
Review Comment:
Can remove the cast, make `container1` have type `Container<?>`
```suggestion
container1 =
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
```
##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java:
##########
@@ -1202,6 +1218,7 @@ public void closeContainer(Container container)
}
container.close();
ContainerLogger.logClosed(container.getContainerData());
+ createContainerMerkleTree(container);
Review Comment:
Similar to Ritesh's existing comment, this method will exit if the container
has moved from open to unhealthy and won't generate the file. We need tests
that we always get a tree when the container leaves the open state. This might
be easier as unit tests, unless we can inject a failure into the existing
integration test.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +182,98 @@ 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(() ->
waitForContainerChecksumToExist(hddsDatanode, container), 100, 5000);
+ }
+
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);
+ // Build expected merkle tree for one datanode.
+ HddsDatanodeService hddsDatanodeService =
cluster.getHddsDatanodes().get(0);
+ OzoneContainer ozoneContainer =
hddsDatanodeService.getDatanodeStateMachine().getContainer();
+ KeyValueContainer container1 = (KeyValueContainer)
ozoneContainer.getController()
+ .getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo expectedChecksumInfo1 =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
waitForContainerChecksumToExist(hddsDatanode, containerInfo1), 100, 5000);
+ container1 = (KeyValueContainer)
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo containerChecksumInfo =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+
ContainerMerkleTreeTestUtils.assertTreesSortedAndMatch(expectedChecksumInfo1.getContainerMerkleTree(),
+ containerChecksumInfo.getContainerMerkleTree());
Review Comment:
Equality is transitive, so instead of saving the first tree and comparing
each one to that (which will duplicate wait code outside of the loop which
currently has a bug), we can save the last replica we saw and compare the
current one to the previous one if the previous one is present. Then check that
we saw 3 replicas.
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOzoneContainerWithTLS.java:
##########
@@ -279,7 +280,7 @@ public void testGetContainerMerkleTree() throws IOException
{
ContainerProtocolCalls.getContainerMerkleTree(client,
containerId, containerToken);
// Getting container merkle tree with valid container token
- assertEquals(response.getContainerMerkleTree(), ByteString.EMPTY);
+ assertNotEquals(response.getContainerMerkleTree(), ByteString.EMPTY);
Review Comment:
nit.
```suggestion
assertFalse(response.getContainerMerkleTree().isEmpty());
```
##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java:
##########
@@ -597,5 +597,4 @@ public BlockDeletingService getBlockDeletingService() {
public ReplicationServer getReplicationServer() {
return replicationServer;
}
-
Review Comment:
nit. whitespace diff
##########
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestCloseContainer.java:
##########
@@ -159,11 +182,98 @@ 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(() ->
waitForContainerChecksumToExist(hddsDatanode, container), 100, 5000);
+ }
+
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);
+ // Build expected merkle tree for one datanode.
+ HddsDatanodeService hddsDatanodeService =
cluster.getHddsDatanodes().get(0);
+ OzoneContainer ozoneContainer =
hddsDatanodeService.getDatanodeStateMachine().getContainer();
+ KeyValueContainer container1 = (KeyValueContainer)
ozoneContainer.getController()
+ .getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo expectedChecksumInfo1 =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
waitForContainerChecksumToExist(hddsDatanode, containerInfo1), 100, 5000);
+ container1 = (KeyValueContainer)
ozoneContainer.getController().getContainer(containerInfo1.getContainerID());
+ ContainerProtos.ContainerChecksumInfo containerChecksumInfo =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container1.getContainerData());
+
ContainerMerkleTreeTestUtils.assertTreesSortedAndMatch(expectedChecksumInfo1.getContainerMerkleTree(),
+ containerChecksumInfo.getContainerMerkleTree());
+ }
+
+ // Create 2nd container and check the checksum doesn't match with 1st
container
+ TestDataUtil.createKey(bucket, "key2", ReplicationFactor.THREE,
+ ReplicationType.RATIS, "this is the different content");
+ ContainerInfo containerInfo2 =
scm.getContainerManager().getContainers().get(1);
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ assertFalse(containerChecksumFileExists(hddsDatanode, containerInfo2));
+ }
+
+ // Close container.
+ OzoneTestUtils.closeContainer(scm, containerInfo2);
+ // Build expected merkle tree for one datanode.
+ KeyValueContainer container2 = (KeyValueContainer)
ozoneContainer.getController()
+ .getContainer(containerInfo2.getContainerID());
+ ContainerProtos.ContainerChecksumInfo expectedChecksumInfo2 =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container2.getContainerData());
+
+ // Container merkle tree for different container should not match.
+ assertNotEquals(expectedChecksumInfo1.getContainerID(),
expectedChecksumInfo2.getContainerID());
+
assertNotEquals(expectedChecksumInfo1.getContainerMerkleTree().getDataChecksum(),
+ expectedChecksumInfo2.getContainerMerkleTree().getDataChecksum());
+
+ // Checksum file exists after container close and matches the expected
container
+ // merkle tree for all the datanodes
+ for (HddsDatanodeService hddsDatanode : hddsDatanodes) {
+ GenericTestUtils.waitFor(() ->
waitForContainerChecksumToExist(hddsDatanode, containerInfo2), 100, 5000);
+ container2 = (KeyValueContainer)
ozoneContainer.getController().getContainer(containerInfo2.getContainerID());
+ ContainerProtos.ContainerChecksumInfo containerChecksumInfo =
ContainerMerkleTreeTestUtils.readChecksumFile(
+ container2.getContainerData());
+
ContainerMerkleTreeTestUtils.assertTreesSortedAndMatch(expectedChecksumInfo2.getContainerMerkleTree(),
+ containerChecksumInfo.getContainerMerkleTree());
+ }
+ }
+
+ private boolean waitForContainerChecksumToExist(HddsDatanodeService
hddsDatanode,
Review Comment:
There is no wait in this method currently. Probably better to have this wait
for container close and then check that the file exists.
--
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]