|
@@ -45,17 +45,17 @@ import org.apache.hadoop.test.GenericTestUtils.LogCapturer;
|
|
|
import org.apache.hadoop.utils.BackgroundService;
|
|
|
import org.apache.hadoop.utils.MetadataKeyFilters;
|
|
|
import org.apache.hadoop.utils.MetadataStore;
|
|
|
+import org.junit.AfterClass;
|
|
|
import org.junit.Assert;
|
|
|
import org.junit.Test;
|
|
|
import org.junit.BeforeClass;
|
|
|
-import org.junit.Before;
|
|
|
-import org.junit.After;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.charset.Charset;
|
|
|
+import java.util.Iterator;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.UUID;
|
|
@@ -72,35 +72,28 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|
|
/**
|
|
|
* Tests to test block deleting service.
|
|
|
*/
|
|
|
-// TODO: Fix BlockDeletingService to work with new StorageLayer
|
|
|
public class TestBlockDeletingService {
|
|
|
|
|
|
private static final Logger LOG =
|
|
|
LoggerFactory.getLogger(TestBlockDeletingService.class);
|
|
|
|
|
|
private static File testRoot;
|
|
|
- private static File containersDir;
|
|
|
- private static File chunksDir;
|
|
|
+ private static String scmId;
|
|
|
+ private static String clusterID;
|
|
|
|
|
|
@BeforeClass
|
|
|
- public static void init() {
|
|
|
+ public static void init() throws IOException {
|
|
|
testRoot = GenericTestUtils
|
|
|
.getTestDir(TestBlockDeletingService.class.getSimpleName());
|
|
|
- chunksDir = new File(testRoot, "chunks");
|
|
|
- containersDir = new File(testRoot, "containers");
|
|
|
- }
|
|
|
-
|
|
|
- @Before
|
|
|
- public void setup() throws IOException {
|
|
|
- if (chunksDir.exists()) {
|
|
|
- FileUtils.deleteDirectory(chunksDir);
|
|
|
+ if (testRoot.exists()) {
|
|
|
+ FileUtils.cleanDirectory(testRoot);
|
|
|
}
|
|
|
+ scmId = UUID.randomUUID().toString();
|
|
|
+ clusterID = UUID.randomUUID().toString();
|
|
|
}
|
|
|
|
|
|
- @After
|
|
|
- public void cleanup() throws IOException {
|
|
|
- FileUtils.deleteDirectory(chunksDir);
|
|
|
- FileUtils.deleteDirectory(containersDir);
|
|
|
+ @AfterClass
|
|
|
+ public static void cleanup() throws IOException {
|
|
|
FileUtils.deleteDirectory(testRoot);
|
|
|
}
|
|
|
|
|
@@ -111,14 +104,15 @@ public class TestBlockDeletingService {
|
|
|
*/
|
|
|
private void createToDeleteBlocks(ContainerSet containerSet,
|
|
|
Configuration conf, int numOfContainers, int numOfBlocksPerContainer,
|
|
|
- int numOfChunksPerBlock, File chunkDir) throws IOException {
|
|
|
+ int numOfChunksPerBlock) throws IOException {
|
|
|
for (int x = 0; x < numOfContainers; x++) {
|
|
|
+ conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, testRoot.getAbsolutePath());
|
|
|
long containerID = ContainerTestHelper.getTestContainerID();
|
|
|
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());
|
|
|
+ container.create(new VolumeSet(scmId, clusterID, conf),
|
|
|
+ new RoundRobinVolumeChoosingPolicy(), scmId);
|
|
|
containerSet.addContainer(container);
|
|
|
data = (KeyValueContainerData) containerSet.getContainer(
|
|
|
containerID).getContainerData();
|
|
@@ -133,7 +127,7 @@ public class TestBlockDeletingService {
|
|
|
for (int k = 0; k<numOfChunksPerBlock; k++) {
|
|
|
// offset doesn't matter here
|
|
|
String chunkName = blockID.getLocalID() + "_chunk_" + k;
|
|
|
- File chunk = new File(chunkDir, chunkName);
|
|
|
+ File chunk = new File(data.getChunksPath(), chunkName);
|
|
|
FileUtils.writeStringToFile(chunk, "a chunk",
|
|
|
Charset.defaultCharset());
|
|
|
LOG.info("Creating file {}", chunk.getAbsolutePath());
|
|
@@ -193,7 +187,7 @@ public class TestBlockDeletingService {
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
|
|
ContainerSet containerSet = new ContainerSet();
|
|
|
- createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 1, 3, 1);
|
|
|
|
|
|
BlockDeletingServiceTestImpl svc =
|
|
|
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
|
|
@@ -208,12 +202,13 @@ public class TestBlockDeletingService {
|
|
|
MetadataStore meta = KeyUtils.getDB(
|
|
|
(KeyValueContainerData) containerData.get(0), conf);
|
|
|
Map<Long, Container> containerMap = containerSet.getContainerMap();
|
|
|
+ long transactionId = containerMap.get(containerData.get(0).getContainerID())
|
|
|
+ .getContainerData().getDeleteTransactionId();
|
|
|
|
|
|
|
|
|
// Number of deleted blocks in container should be equal to 0 before
|
|
|
// block delete
|
|
|
- // TODO : Implement deleteTransactionID in ContainerData.
|
|
|
-// Assert.assertEquals(0, transactionId);
|
|
|
+ Assert.assertEquals(0, transactionId);
|
|
|
|
|
|
// Ensure there are 3 blocks under deletion and 0 deleted blocks
|
|
|
Assert.assertEquals(3, getUnderDeletionBlocksCount(meta));
|
|
@@ -247,7 +242,7 @@ public class TestBlockDeletingService {
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 10);
|
|
|
ContainerSet containerSet = new ContainerSet();
|
|
|
// Create 1 container with 100 blocks
|
|
|
- createToDeleteBlocks(containerSet, conf, 1, 100, 1, chunksDir);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 1, 100, 1);
|
|
|
|
|
|
BlockDeletingServiceTestImpl service =
|
|
|
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
|
|
@@ -275,12 +270,13 @@ public class TestBlockDeletingService {
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
|
|
ContainerSet containerSet = new ContainerSet();
|
|
|
- createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 1, 3, 1);
|
|
|
|
|
|
// set timeout value as 1ns to trigger timeout behavior
|
|
|
long timeout = 1;
|
|
|
- BlockDeletingService svc =
|
|
|
- new BlockDeletingService(containerSet, 1000, timeout, conf);
|
|
|
+ BlockDeletingService svc = new BlockDeletingService(containerSet,
|
|
|
+ TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.NANOSECONDS,
|
|
|
+ conf);
|
|
|
svc.start();
|
|
|
|
|
|
LogCapturer log = LogCapturer.captureLogs(BackgroundService.LOG);
|
|
@@ -299,8 +295,10 @@ public class TestBlockDeletingService {
|
|
|
|
|
|
// test for normal case that doesn't have timeout limitation
|
|
|
timeout = 0;
|
|
|
- createToDeleteBlocks(containerSet, conf, 1, 3, 1, chunksDir);
|
|
|
- svc = new BlockDeletingService(containerSet, 1000, timeout, conf);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 1, 3, 1);
|
|
|
+ svc = new BlockDeletingService(containerSet,
|
|
|
+ TimeUnit.MILLISECONDS.toNanos(1000), timeout, TimeUnit.MILLISECONDS,
|
|
|
+ conf);
|
|
|
svc.start();
|
|
|
|
|
|
// get container meta data
|
|
@@ -347,7 +345,7 @@ public class TestBlockDeletingService {
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 1);
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 1);
|
|
|
ContainerSet containerSet = new ContainerSet();
|
|
|
- createToDeleteBlocks(containerSet, conf, 2, 1, 10, chunksDir);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 2, 1, 10);
|
|
|
|
|
|
BlockDeletingServiceTestImpl service =
|
|
|
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
|
|
@@ -357,7 +355,12 @@ public class TestBlockDeletingService {
|
|
|
GenericTestUtils.waitFor(() -> service.isStarted(), 100, 3000);
|
|
|
// 1st interval processes 1 container 1 block and 10 chunks
|
|
|
deleteAndWait(service, 1);
|
|
|
- Assert.assertEquals(10, chunksDir.listFiles().length);
|
|
|
+ Assert.assertEquals(10, getNumberOfChunksInContainers(containerSet));
|
|
|
+ deleteAndWait(service, 2);
|
|
|
+ deleteAndWait(service, 3);
|
|
|
+ deleteAndWait(service, 4);
|
|
|
+ deleteAndWait(service, 5);
|
|
|
+ Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
|
|
|
} finally {
|
|
|
service.shutdown();
|
|
|
}
|
|
@@ -383,10 +386,10 @@ public class TestBlockDeletingService {
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_CONTAINER_LIMIT_PER_INTERVAL, 10);
|
|
|
conf.setInt(OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER, 2);
|
|
|
ContainerSet containerSet = new ContainerSet();
|
|
|
- createToDeleteBlocks(containerSet, conf, 5, 3, 1, chunksDir);
|
|
|
+ createToDeleteBlocks(containerSet, conf, 5, 3, 1);
|
|
|
|
|
|
// Make sure chunks are created
|
|
|
- Assert.assertEquals(15, chunksDir.listFiles().length);
|
|
|
+ Assert.assertEquals(15, getNumberOfChunksInContainers(containerSet));
|
|
|
|
|
|
BlockDeletingServiceTestImpl service =
|
|
|
new BlockDeletingServiceTestImpl(containerSet, 1000, conf);
|
|
@@ -399,13 +402,26 @@ public class TestBlockDeletingService {
|
|
|
// number of containers = 5
|
|
|
// each interval will at most runDeletingTasks 5 * 2 = 10 blocks
|
|
|
deleteAndWait(service, 1);
|
|
|
- Assert.assertEquals(5, chunksDir.listFiles().length);
|
|
|
+ Assert.assertEquals(5, getNumberOfChunksInContainers(containerSet));
|
|
|
|
|
|
// There is only 5 blocks left to runDeletingTasks
|
|
|
deleteAndWait(service, 2);
|
|
|
- Assert.assertEquals(0, chunksDir.listFiles().length);
|
|
|
+ Assert.assertEquals(0, getNumberOfChunksInContainers(containerSet));
|
|
|
} finally {
|
|
|
service.shutdown();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private int getNumberOfChunksInContainers(ContainerSet containerSet) {
|
|
|
+ Iterator<Container> containerIterator = containerSet.getContainerIterator();
|
|
|
+ int numChunks = 0;
|
|
|
+ while (containerIterator.hasNext()) {
|
|
|
+ Container container = containerIterator.next();
|
|
|
+ File chunkDir = FileUtils.getFile(
|
|
|
+ ((KeyValueContainerData) container.getContainerData())
|
|
|
+ .getChunksPath());
|
|
|
+ numChunks += chunkDir.listFiles().length;
|
|
|
+ }
|
|
|
+ return numChunks;
|
|
|
+ }
|
|
|
}
|