|
@@ -19,7 +19,6 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
|
|
|
|
|
|
import com.google.common.primitives.Longs;
|
|
|
import org.apache.commons.lang3.RandomStringUtils;
|
|
|
-import org.apache.hadoop.hdds.client.BlockID;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationFactor;
|
|
|
import org.apache.hadoop.hdds.client.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
@@ -29,6 +28,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
+import org.apache.hadoop.ozone.OzoneTestUtils;
|
|
|
import org.apache.hadoop.ozone.client.ObjectStore;
|
|
|
import org.apache.hadoop.ozone.client.OzoneBucket;
|
|
|
import org.apache.hadoop.ozone.client.OzoneClientFactory;
|
|
@@ -40,7 +40,6 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
|
|
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
|
|
import org.apache.hadoop.ozone.om.OzoneManager;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
|
|
|
-import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
|
|
|
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
|
|
|
import org.apache.hadoop.ozone.ozShell.TestOzoneShell;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -53,13 +52,15 @@ import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.util.*;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
-import java.util.function.Consumer;
|
|
|
|
|
|
-import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
|
|
|
+import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_HEARTBEAT_INTERVAL;
|
|
|
+import static org.apache.hadoop.ozone
|
|
|
+ .OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
|
|
|
|
|
|
public class TestBlockDeletion {
|
|
|
private static OzoneConfiguration conf = null;
|
|
|
private static ObjectStore store;
|
|
|
+ private static MiniOzoneCluster cluster = null;
|
|
|
private static ContainerSet dnContainerSet = null;
|
|
|
private static StorageContainerManager scm = null;
|
|
|
private static OzoneManager om = null;
|
|
@@ -81,9 +82,10 @@ public class TestBlockDeletion {
|
|
|
conf.setQuietMode(false);
|
|
|
conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 100,
|
|
|
TimeUnit.MILLISECONDS);
|
|
|
+ conf.setTimeDuration(HDDS_HEARTBEAT_INTERVAL, 200,
|
|
|
+ TimeUnit.MILLISECONDS);
|
|
|
|
|
|
- MiniOzoneCluster cluster =
|
|
|
- MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
|
|
+ cluster = MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
|
|
cluster.waitForClusterToBeReady();
|
|
|
store = OzoneClientFactory.getRpcClient(conf).getObjectStore();
|
|
|
dnContainerSet = cluster.getHddsDatanodes().get(0)
|
|
@@ -127,6 +129,14 @@ public class TestBlockDeletion {
|
|
|
matchContainerTransactionIds();
|
|
|
om.deleteKey(keyArgs);
|
|
|
Thread.sleep(5000);
|
|
|
+ // The blocks should not be deleted in the DN as the container is open
|
|
|
+ Assert.assertTrue(!verifyBlocksDeleted(omKeyLocationInfoGroupList));
|
|
|
+
|
|
|
+ // close the containers which hold the blocks for the key
|
|
|
+ Assert
|
|
|
+ .assertTrue(
|
|
|
+ OzoneTestUtils.closeContainers(omKeyLocationInfoGroupList, scm));
|
|
|
+ Thread.sleep(5000);
|
|
|
// The blocks should be deleted in the DN.
|
|
|
Assert.assertTrue(verifyBlocksDeleted(omKeyLocationInfoGroupList));
|
|
|
|
|
@@ -157,7 +167,7 @@ public class TestBlockDeletion {
|
|
|
private boolean verifyBlocksCreated(
|
|
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
|
|
throws IOException {
|
|
|
- return performOperationOnKeyContainers((blockID) -> {
|
|
|
+ return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
|
|
|
try {
|
|
|
MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
|
|
|
dnContainerSet.getContainer(blockID.getContainerID())
|
|
@@ -172,7 +182,7 @@ public class TestBlockDeletion {
|
|
|
private boolean verifyBlocksDeleted(
|
|
|
List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
|
|
throws IOException {
|
|
|
- return performOperationOnKeyContainers((blockID) -> {
|
|
|
+ return OzoneTestUtils.performOperationOnKeyContainers((blockID) -> {
|
|
|
try {
|
|
|
MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
|
|
|
dnContainerSet.getContainer(blockID.getContainerID())
|
|
@@ -188,25 +198,4 @@ public class TestBlockDeletion {
|
|
|
}
|
|
|
}, omKeyLocationInfoGroups);
|
|
|
}
|
|
|
-
|
|
|
- private boolean performOperationOnKeyContainers(Consumer<BlockID> consumer,
|
|
|
- List<OmKeyLocationInfoGroup> omKeyLocationInfoGroups)
|
|
|
- throws IOException {
|
|
|
-
|
|
|
- try {
|
|
|
- for (OmKeyLocationInfoGroup omKeyLocationInfoGroup :
|
|
|
- omKeyLocationInfoGroups) {
|
|
|
- List<OmKeyLocationInfo> omKeyLocationInfos =
|
|
|
- omKeyLocationInfoGroup.getLocationList();
|
|
|
- for (OmKeyLocationInfo omKeyLocationInfo : omKeyLocationInfos) {
|
|
|
- BlockID blockID = omKeyLocationInfo.getBlockID();
|
|
|
- consumer.accept(blockID);
|
|
|
- }
|
|
|
- }
|
|
|
- } catch (Error e) {
|
|
|
- e.printStackTrace();
|
|
|
- return false;
|
|
|
- }
|
|
|
- return true;
|
|
|
- }
|
|
|
}
|