HDDS-251. Integrate BlockDeletingService in KeyValueHandler. Contributed by Lokesh Jain
This commit is contained in:
parent
4523cc5637
commit
0927bc4f76
@ -231,8 +231,9 @@ public final class ScmConfigKeys {
|
||||
"ozone.scm.container.provision_batch_size";
|
||||
public static final int OZONE_SCM_CONTAINER_PROVISION_BATCH_SIZE_DEFAULT = 20;
|
||||
|
||||
public static final String OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY =
|
||||
"ozone.scm.container.deletion-choosing.policy";
|
||||
public static final String
|
||||
OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY =
|
||||
"ozone.scm.keyvalue.container.deletion-choosing.policy";
|
||||
|
||||
public static final String OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT =
|
||||
"ozone.scm.container.creation.lease.timeout";
|
||||
|
@ -541,13 +541,13 @@
|
||||
<description>The port number of the Ozone SCM client service.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.scm.container.deletion-choosing.policy</name>
|
||||
<name>ozone.scm.keyvalue.container.deletion-choosing.policy</name>
|
||||
<value>
|
||||
org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy
|
||||
</value>
|
||||
<tag>OZONE, MANAGEMENT</tag>
|
||||
<description>
|
||||
The policy used for choosing desire containers for block deletion.
|
||||
The policy used for choosing desired keyvalue containers for block deletion.
|
||||
Datanode selects some containers to process block deletion
|
||||
in a certain interval defined by ozone.block.deleting.service.interval.
|
||||
The number of containers to process in each interval is defined
|
||||
|
@ -30,6 +30,8 @@
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers
|
||||
.StorageContainerException;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||
import org.apache.hadoop.ozone.container.common
|
||||
.interfaces.ContainerDeletionChoosingPolicy;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -247,9 +249,15 @@ private HddsProtos.LifeCycleState getState(ContainerData containerData)
|
||||
return state;
|
||||
}
|
||||
|
||||
// TODO: Implement BlockDeletingService
|
||||
public List<ContainerData> chooseContainerForBlockDeletion(
|
||||
int count) throws StorageContainerException {
|
||||
return null;
|
||||
public List<ContainerData> chooseContainerForBlockDeletion(int count,
|
||||
ContainerDeletionChoosingPolicy deletionPolicy)
|
||||
throws StorageContainerException {
|
||||
Map<Long, ContainerData> containerDataMap = containerMap.entrySet().stream()
|
||||
.filter(e -> e.getValue().getContainerType()
|
||||
== ContainerProtos.ContainerType.KeyValueContainer)
|
||||
.collect(Collectors.toMap(Map.Entry::getKey,
|
||||
e -> e.getValue().getContainerData()));
|
||||
return deletionPolicy
|
||||
.chooseContainerForBlockDeletion(count, containerDataMap);
|
||||
}
|
||||
}
|
||||
|
@ -68,7 +68,7 @@ void update(Map<String, String> metaData, boolean forceUpdate)
|
||||
* @return ContainerData - Container Data.
|
||||
* @throws StorageContainerException
|
||||
*/
|
||||
ContainerData getContainerData() throws StorageContainerException;
|
||||
ContainerData getContainerData();
|
||||
|
||||
/**
|
||||
* Get the Container Lifecycle state.
|
||||
|
@ -28,7 +28,6 @@
|
||||
* This interface is used for choosing desired containers for
|
||||
* block deletion.
|
||||
*/
|
||||
// TODO: Fix ContainerDeletionChoosingPolicy to work with new StorageLayer
|
||||
public interface ContainerDeletionChoosingPolicy {
|
||||
|
||||
/**
|
||||
|
@ -62,6 +62,8 @@
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.interfaces.KeyManager;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.statemachine
|
||||
.background.BlockDeletingService;
|
||||
import org.apache.hadoop.util.AutoCloseableLock;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -71,6 +73,7 @@
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
@ -90,6 +93,14 @@
|
||||
|
||||
import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.Stage;
|
||||
import static org.apache.hadoop.ozone
|
||||
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
|
||||
import static org.apache.hadoop.ozone
|
||||
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT;
|
||||
import static org.apache.hadoop.ozone
|
||||
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
|
||||
import static org.apache.hadoop.ozone
|
||||
.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
|
||||
|
||||
/**
|
||||
* Handler for KeyValue Container type.
|
||||
@ -102,6 +113,7 @@ public class KeyValueHandler extends Handler {
|
||||
private final ContainerType containerType;
|
||||
private final KeyManager keyManager;
|
||||
private final ChunkManager chunkManager;
|
||||
private final BlockDeletingService blockDeletingService;
|
||||
private VolumeChoosingPolicy volumeChoosingPolicy;
|
||||
private final int maxContainerSizeGB;
|
||||
private final AutoCloseableLock handlerLock;
|
||||
@ -113,6 +125,18 @@ public KeyValueHandler(Configuration config, ContainerSet contSet,
|
||||
containerType = ContainerType.KeyValueContainer;
|
||||
keyManager = new KeyManagerImpl(config);
|
||||
chunkManager = new ChunkManagerImpl();
|
||||
long svcInterval = config
|
||||
.getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL,
|
||||
OZONE_BLOCK_DELETING_SERVICE_INTERVAL_DEFAULT,
|
||||
TimeUnit.MILLISECONDS);
|
||||
long serviceTimeout = config
|
||||
.getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_TIMEOUT,
|
||||
OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT,
|
||||
TimeUnit.MILLISECONDS);
|
||||
this.blockDeletingService =
|
||||
new BlockDeletingService(containerSet, svcInterval, serviceTimeout,
|
||||
config);
|
||||
blockDeletingService.start();
|
||||
// TODO: Add supoort for different volumeChoosingPolicies.
|
||||
volumeChoosingPolicy = new RoundRobinVolumeChoosingPolicy();
|
||||
maxContainerSizeGB = config.getInt(ScmConfigKeys
|
||||
|
@ -19,10 +19,14 @@
|
||||
package org.apache.hadoop.ozone.container.keyvalue.statemachine.background;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||
import org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDeletionChoosingPolicy;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.ratis.shaded.com.google.protobuf
|
||||
.InvalidProtocolBufferException;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
@ -69,6 +73,7 @@ public class BlockDeletingService extends BackgroundService{
|
||||
LoggerFactory.getLogger(BlockDeletingService.class);
|
||||
|
||||
ContainerSet containerSet;
|
||||
private ContainerDeletionChoosingPolicy containerDeletionPolicy;
|
||||
private final Configuration conf;
|
||||
|
||||
// Throttle number of blocks to delete per task,
|
||||
@ -89,6 +94,10 @@ public BlockDeletingService(ContainerSet containerSet,
|
||||
TimeUnit.MILLISECONDS, BLOCK_DELETING_SERVICE_CORE_POOL_SIZE,
|
||||
serviceTimeout);
|
||||
this.containerSet = containerSet;
|
||||
containerDeletionPolicy = ReflectionUtils.newInstance(conf.getClass(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
TopNOrderedContainerDeletionChoosingPolicy.class,
|
||||
ContainerDeletionChoosingPolicy.class), conf);
|
||||
this.conf = conf;
|
||||
this.blockLimitPerTask = conf.getInt(
|
||||
OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER,
|
||||
@ -110,7 +119,7 @@ public BackgroundTaskQueue getTasks() {
|
||||
// The chosen result depends on what container deletion policy is
|
||||
// configured.
|
||||
containers = containerSet.chooseContainerForBlockDeletion(
|
||||
containerLimitPerInterval);
|
||||
containerLimitPerInterval, containerDeletionPolicy);
|
||||
LOG.info("Plan to choose {} containers for block deletion, "
|
||||
+ "actually returns {} valid containers.",
|
||||
containerLimitPerInterval, containers.size());
|
||||
|
@ -21,17 +21,16 @@
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.client.BlockID;
|
||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
||||
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
|
||||
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
@ -47,7 +46,6 @@
|
||||
import org.apache.hadoop.utils.MetadataKeyFilters;
|
||||
import org.apache.hadoop.utils.MetadataStore;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Before;
|
||||
@ -58,9 +56,9 @@
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
||||
@ -75,7 +73,6 @@
|
||||
* Tests to test block deleting service.
|
||||
*/
|
||||
// TODO: Fix BlockDeletingService to work with new StorageLayer
|
||||
@Ignore
|
||||
public class TestBlockDeletingService {
|
||||
|
||||
private static final Logger LOG =
|
||||
@ -120,6 +117,8 @@ private void createToDeleteBlocks(ContainerSet containerSet,
|
||||
KeyValueContainerData data = new KeyValueContainerData(containerID,
|
||||
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
|
||||
Container container = new KeyValueContainer(data, conf);
|
||||
container.create(new VolumeSet(UUID.randomUUID().toString(), conf),
|
||||
new RoundRobinVolumeChoosingPolicy(), UUID.randomUUID().toString());
|
||||
containerSet.addContainer(container);
|
||||
data = (KeyValueContainerData) containerSet.getContainer(
|
||||
containerID).getContainerData();
|
||||
@ -188,6 +187,9 @@ private int getDeletedBlocksCount(MetadataStore db) throws IOException {
|
||||
@Test
|
||||
public void testBlockDeletion() throws Exception {
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
||||
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
||||
ContainerSet containerSet = new ContainerSet();
|
||||
@ -236,6 +238,9 @@ public void testBlockDeletion() throws Exception {
|
||||
@Test
|
||||
public void testShutdownService() throws Exception {
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
conf.setTimeDuration(OZONE_BLOCK_DELETING_SERVICE_INTERVAL, 500,
|
||||
TimeUnit.MILLISECONDS);
|
||||
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
||||
@ -264,6 +269,9 @@ public void testShutdownService() throws Exception {
|
||||
@Test
|
||||
public void testBlockDeletionTimeout() throws Exception {
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
||||
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
||||
ContainerSet containerSet = new ContainerSet();
|
||||
@ -333,6 +341,9 @@ public void testContainerThrottle() throws Exception {
|
||||
// 1 block from 1 container can be deleted.
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
// Process 1 container per interval
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
|
||||
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
|
||||
ContainerSet containerSet = new ContainerSet();
|
||||
@ -366,6 +377,9 @@ public void testBlockThrottle() throws Exception {
|
||||
// per container can be actually deleted. So it requires 2 waves
|
||||
// to cleanup all blocks.
|
||||
Configuration conf = new OzoneConfiguration();
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
||||
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
||||
ContainerSet containerSet = new ContainerSet();
|
||||
|
@ -27,29 +27,22 @@
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.lang3.RandomUtils;
|
||||
import org.apache.hadoop.hdds.scm.TestUtils;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
||||
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDeletionChoosingPolicy;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
||||
import org.apache.hadoop.ozone.container.keyvalue.helpers.KeyUtils;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.utils.MetadataStore;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
/**
|
||||
* The class for testing container deletion choosing policy.
|
||||
*/
|
||||
@Ignore
|
||||
public class TestContainerDeletionChoosingPolicy {
|
||||
private static String path;
|
||||
private static ContainerSet containerSet;
|
||||
@ -73,7 +66,8 @@ public void testRandomChoosingPolicy() throws IOException {
|
||||
}
|
||||
Assert.assertTrue(containerDir.mkdirs());
|
||||
|
||||
conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
RandomContainerDeletionChoosingPolicy.class.getName());
|
||||
List<StorageLocation> pathLists = new LinkedList<>();
|
||||
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
|
||||
@ -89,15 +83,17 @@ public void testRandomChoosingPolicy() throws IOException {
|
||||
containerSet.getContainerMap().containsKey(data.getContainerID()));
|
||||
}
|
||||
|
||||
List<ContainerData> result0 = containerSet
|
||||
.chooseContainerForBlockDeletion(5);
|
||||
ContainerDeletionChoosingPolicy deletionPolicy =
|
||||
new RandomContainerDeletionChoosingPolicy();
|
||||
List<ContainerData> result0 =
|
||||
containerSet.chooseContainerForBlockDeletion(5, deletionPolicy);
|
||||
Assert.assertEquals(5, result0.size());
|
||||
|
||||
// test random choosing
|
||||
List<ContainerData> result1 = containerSet
|
||||
.chooseContainerForBlockDeletion(numContainers);
|
||||
.chooseContainerForBlockDeletion(numContainers, deletionPolicy);
|
||||
List<ContainerData> result2 = containerSet
|
||||
.chooseContainerForBlockDeletion(numContainers);
|
||||
.chooseContainerForBlockDeletion(numContainers, deletionPolicy);
|
||||
|
||||
boolean hasShuffled = false;
|
||||
for (int i = 0; i < numContainers; i++) {
|
||||
@ -118,12 +114,12 @@ public void testTopNOrderedChoosingPolicy() throws IOException {
|
||||
}
|
||||
Assert.assertTrue(containerDir.mkdirs());
|
||||
|
||||
conf.set(ScmConfigKeys.OZONE_SCM_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
conf.set(
|
||||
ScmConfigKeys.OZONE_SCM_KEY_VALUE_CONTAINER_DELETION_CHOOSING_POLICY,
|
||||
TopNOrderedContainerDeletionChoosingPolicy.class.getName());
|
||||
List<StorageLocation> pathLists = new LinkedList<>();
|
||||
pathLists.add(StorageLocation.parse(containerDir.getAbsolutePath()));
|
||||
containerSet = new ContainerSet();
|
||||
DatanodeDetails datanodeDetails = TestUtils.getDatanodeDetails();
|
||||
|
||||
int numContainers = 10;
|
||||
Random random = new Random();
|
||||
@ -131,38 +127,28 @@ public void testTopNOrderedChoosingPolicy() throws IOException {
|
||||
// create [numContainers + 1] containers
|
||||
for (int i = 0; i <= numContainers; i++) {
|
||||
long containerId = RandomUtils.nextLong();
|
||||
KeyValueContainerData data = new KeyValueContainerData(new Long(i),
|
||||
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
|
||||
KeyValueContainerData data =
|
||||
new KeyValueContainerData(new Long(containerId),
|
||||
ContainerTestHelper.CONTAINER_MAX_SIZE_GB);
|
||||
if (i != numContainers) {
|
||||
int deletionBlocks = random.nextInt(numContainers) + 1;
|
||||
data.incrPendingDeletionBlocks(deletionBlocks);
|
||||
name2Count.put(containerId, deletionBlocks);
|
||||
}
|
||||
KeyValueContainer container = new KeyValueContainer(data, conf);
|
||||
containerSet.addContainer(container);
|
||||
Assert.assertTrue(
|
||||
containerSet.getContainerMap().containsKey(containerId));
|
||||
|
||||
// don't create deletion blocks in the last container.
|
||||
if (i == numContainers) {
|
||||
break;
|
||||
}
|
||||
|
||||
// create random number of deletion blocks and write to container db
|
||||
int deletionBlocks = random.nextInt(numContainers) + 1;
|
||||
// record <ContainerName, DeletionCount> value
|
||||
name2Count.put(containerId, deletionBlocks);
|
||||
for (int j = 0; j <= deletionBlocks; j++) {
|
||||
MetadataStore metadata = KeyUtils.getDB(data, conf);
|
||||
String blk = "blk" + i + "-" + j;
|
||||
byte[] blkBytes = DFSUtil.string2Bytes(blk);
|
||||
metadata.put(
|
||||
DFSUtil.string2Bytes(OzoneConsts.DELETING_KEY_PREFIX + blk),
|
||||
blkBytes);
|
||||
}
|
||||
}
|
||||
|
||||
List<ContainerData> result0 = containerSet
|
||||
.chooseContainerForBlockDeletion(5);
|
||||
ContainerDeletionChoosingPolicy deletionPolicy =
|
||||
new TopNOrderedContainerDeletionChoosingPolicy();
|
||||
List<ContainerData> result0 =
|
||||
containerSet.chooseContainerForBlockDeletion(5, deletionPolicy);
|
||||
Assert.assertEquals(5, result0.size());
|
||||
|
||||
List<ContainerData> result1 = containerSet
|
||||
.chooseContainerForBlockDeletion(numContainers + 1);
|
||||
.chooseContainerForBlockDeletion(numContainers + 1, deletionPolicy);
|
||||
// the empty deletion blocks container should not be chosen
|
||||
Assert.assertEquals(numContainers, result1.size());
|
||||
|
||||
|
@ -47,7 +47,6 @@
|
||||
import org.apache.hadoop.utils.MetadataStore;
|
||||
import org.junit.Assert;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
@ -58,11 +57,10 @@
|
||||
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_BLOCK_DELETING_SERVICE_INTERVAL;
|
||||
|
||||
@Ignore("Need to be fixed according to ContainerIO")
|
||||
public class TestBlockDeletion {
|
||||
private static OzoneConfiguration conf = null;
|
||||
private static ObjectStore store;
|
||||
private static ContainerSet dnContainerManager = null;
|
||||
private static ContainerSet dnContainerSet = null;
|
||||
private static StorageContainerManager scm = null;
|
||||
private static OzoneManager om = null;
|
||||
private static Set<Long> containerIdsWithDeletedBlocks;
|
||||
@ -88,7 +86,7 @@ public static void init() throws Exception {
|
||||
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).build();
|
||||
cluster.waitForClusterToBeReady();
|
||||
store = OzoneClientFactory.getRpcClient(conf).getObjectStore();
|
||||
dnContainerManager = cluster.getHddsDatanodes().get(0)
|
||||
dnContainerSet = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine().getContainer().getContainerSet();
|
||||
om = cluster.getOzoneManager();
|
||||
scm = cluster.getStorageContainerManager();
|
||||
@ -140,7 +138,7 @@ public void testBlockDeletion()
|
||||
|
||||
private void matchContainerTransactionIds() throws IOException {
|
||||
List<ContainerData> containerDataList = new ArrayList<>();
|
||||
dnContainerManager.listContainer(0, 10000, containerDataList);
|
||||
dnContainerSet.listContainer(0, 10000, containerDataList);
|
||||
for (ContainerData containerData : containerDataList) {
|
||||
long containerId = containerData.getContainerID();
|
||||
if (containerIdsWithDeletedBlocks.contains(containerId)) {
|
||||
@ -150,7 +148,7 @@ private void matchContainerTransactionIds() throws IOException {
|
||||
Assert.assertEquals(
|
||||
scm.getContainerInfo(containerId).getDeleteTransactionId(), 0);
|
||||
}
|
||||
Assert.assertEquals(dnContainerManager.getContainer(containerId)
|
||||
Assert.assertEquals(dnContainerSet.getContainer(containerId)
|
||||
.getContainerData().getDeleteTransactionId(),
|
||||
scm.getContainerInfo(containerId).getDeleteTransactionId());
|
||||
}
|
||||
@ -162,7 +160,7 @@ private boolean verifyBlocksCreated(
|
||||
return performOperationOnKeyContainers((blockID) -> {
|
||||
try {
|
||||
MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
|
||||
dnContainerManager.getContainer(blockID.getContainerID())
|
||||
dnContainerSet.getContainer(blockID.getContainerID())
|
||||
.getContainerData(), conf);
|
||||
Assert.assertNotNull(db.get(Longs.toByteArray(blockID.getLocalID())));
|
||||
} catch (IOException e) {
|
||||
@ -177,7 +175,7 @@ private boolean verifyBlocksDeleted(
|
||||
return performOperationOnKeyContainers((blockID) -> {
|
||||
try {
|
||||
MetadataStore db = KeyUtils.getDB((KeyValueContainerData)
|
||||
dnContainerManager.getContainer(blockID.getContainerID())
|
||||
dnContainerSet.getContainer(blockID.getContainerID())
|
||||
.getContainerData(), conf);
|
||||
Assert.assertNull(db.get(Longs.toByteArray(blockID.getLocalID())));
|
||||
Assert.assertNull(db.get(DFSUtil.string2Bytes(
|
||||
|
@ -252,17 +252,13 @@ public void testCloseContainerViaRatis() throws IOException,
|
||||
private Boolean isContainerClosed(MiniOzoneCluster cluster, long containerID,
|
||||
DatanodeDetails datanode) {
|
||||
ContainerData containerData;
|
||||
try {
|
||||
for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes())
|
||||
if (datanode.equals(datanodeService.getDatanodeDetails())) {
|
||||
containerData =
|
||||
datanodeService.getDatanodeStateMachine().getContainer()
|
||||
.getContainerSet().getContainer(containerID).getContainerData();
|
||||
return !containerData.isOpen();
|
||||
}
|
||||
} catch (StorageContainerException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
for (HddsDatanodeService datanodeService : cluster.getHddsDatanodes())
|
||||
if (datanode.equals(datanodeService.getDatanodeDetails())) {
|
||||
containerData =
|
||||
datanodeService.getDatanodeStateMachine().getContainer()
|
||||
.getContainerSet().getContainer(containerID).getContainerData();
|
||||
return !containerData.isOpen();
|
||||
}
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
@ -102,14 +102,10 @@ public void test() throws IOException, TimeoutException, InterruptedException,
|
||||
private Boolean isContainerClosed(MiniOzoneCluster cluster,
|
||||
long containerID) {
|
||||
ContainerData containerData;
|
||||
try {
|
||||
containerData = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine().getContainer().getContainerSet()
|
||||
.getContainer(containerID).getContainerData();
|
||||
return !containerData.isOpen();
|
||||
} catch (StorageContainerException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
containerData = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine().getContainer().getContainerSet()
|
||||
.getContainer(containerID).getContainerData();
|
||||
return !containerData.isOpen();
|
||||
}
|
||||
|
||||
}
|
@ -131,13 +131,10 @@ public void testContainerReportKeyWrite() throws Exception {
|
||||
|
||||
private static ContainerData getContainerData(long containerID) {
|
||||
ContainerData containerData;
|
||||
try {
|
||||
ContainerSet containerManager = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine().getContainer().getContainerSet();
|
||||
containerData = containerManager.getContainer(containerID).getContainerData();
|
||||
} catch (StorageContainerException e) {
|
||||
throw new AssertionError(e);
|
||||
}
|
||||
ContainerSet containerManager = cluster.getHddsDatanodes().get(0)
|
||||
.getDatanodeStateMachine().getContainer().getContainerSet();
|
||||
containerData =
|
||||
containerManager.getContainer(containerID).getContainerData();
|
||||
return containerData;
|
||||
}
|
||||
}
|
@ -67,7 +67,6 @@
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Ignore;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.rules.Timeout;
|
||||
@ -663,7 +662,6 @@ private int countOmKeys(OzoneManager om) throws IOException {
|
||||
}
|
||||
|
||||
@Test
|
||||
@Ignore("Needs to be fixed for new SCM and Storage design")
|
||||
public void testDeleteKey() throws Exception {
|
||||
OzoneManager ozoneManager = ozoneCluster.getOzoneManager();
|
||||
// To avoid interference from other test cases,
|
||||
|
Loading…
Reference in New Issue
Block a user