|
@@ -29,20 +29,20 @@ import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerExcep
|
|
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.container.ContainerTestHelper;
|
|
|
+import org.apache.hadoop.ozone.container.common.helpers.BlockData;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
|
|
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.Container;
|
|
|
import org.apache.hadoop.ozone.container.common.interfaces.VolumeChoosingPolicy;
|
|
|
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;
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
|
|
|
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
|
|
|
-import org.apache.hadoop.ozone.container.keyvalue.impl.KeyManagerImpl;
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.impl.BlockManagerImpl;
|
|
|
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.interfaces.BlockManager;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.utils.MetadataStore;
|
|
|
import org.junit.After;
|
|
@@ -97,7 +97,7 @@ public class TestContainerPersistence {
|
|
|
private static ContainerSet containerSet;
|
|
|
private static VolumeSet volumeSet;
|
|
|
private static VolumeChoosingPolicy volumeChoosingPolicy;
|
|
|
- private static KeyManager keyManager;
|
|
|
+ private static BlockManager blockManager;
|
|
|
private static ChunkManager chunkManager;
|
|
|
@Rule
|
|
|
public ExpectedException exception = ExpectedException.none();
|
|
@@ -126,7 +126,7 @@ public class TestContainerPersistence {
|
|
|
public void setupPaths() throws IOException {
|
|
|
containerSet = new ContainerSet();
|
|
|
volumeSet = new VolumeSet(DATANODE_UUID, conf);
|
|
|
- keyManager = new KeyManagerImpl(conf);
|
|
|
+ blockManager = new BlockManagerImpl(conf);
|
|
|
chunkManager = new ChunkManagerImpl();
|
|
|
|
|
|
for (String dir : conf.getStrings(DFS_DATANODE_DATA_DIR_KEY)) {
|
|
@@ -152,15 +152,15 @@ public class TestContainerPersistence {
|
|
|
return ContainerTestHelper.getTestContainerID();
|
|
|
}
|
|
|
|
|
|
- private Container addContainer(ContainerSet containerSet, long containerID)
|
|
|
+ private Container addContainer(ContainerSet cSet, long cID)
|
|
|
throws IOException {
|
|
|
- KeyValueContainerData data = new KeyValueContainerData(containerID,
|
|
|
+ KeyValueContainerData data = new KeyValueContainerData(cID,
|
|
|
ContainerTestHelper.CONTAINER_MAX_SIZE);
|
|
|
data.addMetadata("VOLUME", "shire");
|
|
|
data.addMetadata("owner)", "bilbo");
|
|
|
KeyValueContainer container = new KeyValueContainer(data, conf);
|
|
|
container.create(volumeSet, volumeChoosingPolicy, SCM_ID);
|
|
|
- containerSet.addContainer(container);
|
|
|
+ cSet.addContainer(container);
|
|
|
return container;
|
|
|
}
|
|
|
|
|
@@ -184,7 +184,7 @@ public class TestContainerPersistence {
|
|
|
|
|
|
MetadataStore store = null;
|
|
|
try {
|
|
|
- store = KeyUtils.getDB(kvData, conf);
|
|
|
+ store = BlockUtils.getDB(kvData, conf);
|
|
|
Assert.assertNotNull(store);
|
|
|
} finally {
|
|
|
if (store != null) {
|
|
@@ -227,19 +227,19 @@ public class TestContainerPersistence {
|
|
|
Assert.assertFalse(containerSet.getContainerMap()
|
|
|
.containsKey(testContainerID1));
|
|
|
|
|
|
- // Adding key to a deleted container should fail.
|
|
|
+ // Adding block to a deleted container should fail.
|
|
|
exception.expect(StorageContainerException.class);
|
|
|
exception.expectMessage("Error opening DB.");
|
|
|
BlockID blockID1 = ContainerTestHelper.getTestBlockID(testContainerID1);
|
|
|
- KeyData someKey1 = new KeyData(blockID1);
|
|
|
+ BlockData someKey1 = new BlockData(blockID1);
|
|
|
someKey1.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
|
|
|
- keyManager.putKey(container1, someKey1);
|
|
|
+ blockManager.putBlock(container1, someKey1);
|
|
|
|
|
|
// Deleting a non-empty container should fail.
|
|
|
BlockID blockID2 = ContainerTestHelper.getTestBlockID(testContainerID2);
|
|
|
- KeyData someKey2 = new KeyData(blockID2);
|
|
|
+ BlockData someKey2 = new BlockData(blockID2);
|
|
|
someKey2.setChunks(new LinkedList<ContainerProtos.ChunkInfo>());
|
|
|
- keyManager.putKey(container2, someKey2);
|
|
|
+ blockManager.putBlock(container2, someKey2);
|
|
|
|
|
|
exception.expect(StorageContainerException.class);
|
|
|
exception.expectMessage(
|
|
@@ -325,7 +325,8 @@ public class TestContainerPersistence {
|
|
|
if (container == null) {
|
|
|
container = addContainer(containerSet, testContainerID);
|
|
|
}
|
|
|
- ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
|
|
|
+ ChunkInfo info = getChunk(
|
|
|
+ blockID.getLocalID(), 0, 0, datalen);
|
|
|
byte[] data = getData(datalen);
|
|
|
setDataChecksum(info, data);
|
|
|
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
|
|
@@ -348,8 +349,8 @@ public class TestContainerPersistence {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Writes many chunks of the same key into different chunk files and verifies
|
|
|
- * that we have that data in many files.
|
|
|
+ * Writes many chunks of the same block into different chunk files and
|
|
|
+ * verifies that we have that data in many files.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
* @throws NoSuchAlgorithmException
|
|
@@ -425,7 +426,8 @@ public class TestContainerPersistence {
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
- ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
|
|
|
+ ChunkInfo info = getChunk(
|
|
|
+ blockID.getLocalID(), 0, 0, datalen);
|
|
|
byte[] data = getData(datalen);
|
|
|
setDataChecksum(info, data);
|
|
|
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
|
|
@@ -456,7 +458,8 @@ public class TestContainerPersistence {
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
- ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
|
|
|
+ ChunkInfo info = getChunk(
|
|
|
+ blockID.getLocalID(), 0, 0, datalen);
|
|
|
byte[] data = getData(datalen);
|
|
|
setDataChecksum(info, data);
|
|
|
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
|
|
@@ -500,7 +503,8 @@ public class TestContainerPersistence {
|
|
|
for (int x = 0; x < chunkCount; x++) {
|
|
|
// we are writing to the same chunk file but at different offsets.
|
|
|
long offset = x * datalen;
|
|
|
- ChunkInfo info = getChunk(blockID.getLocalID(), 0, offset, datalen);
|
|
|
+ ChunkInfo info = getChunk(
|
|
|
+ blockID.getLocalID(), 0, offset, datalen);
|
|
|
byte[] data = getData(datalen);
|
|
|
oldSha.update(data);
|
|
|
setDataChecksum(info, data);
|
|
@@ -531,7 +535,8 @@ public class TestContainerPersistence {
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
- ChunkInfo info = getChunk(blockID.getLocalID(), 0, 0, datalen);
|
|
|
+ ChunkInfo info = getChunk(
|
|
|
+ blockID.getLocalID(), 0, 0, datalen);
|
|
|
byte[] data = getData(datalen);
|
|
|
setDataChecksum(info, data);
|
|
|
chunkManager.writeChunk(container, blockID, info, data, COMBINED);
|
|
@@ -542,37 +547,38 @@ public class TestContainerPersistence {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Tests a put key and read key.
|
|
|
+ * Tests a put block and read block.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
* @throws NoSuchAlgorithmException
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testPutKey() throws IOException, NoSuchAlgorithmException {
|
|
|
+ public void testPutBlock() throws IOException, NoSuchAlgorithmException {
|
|
|
long testContainerID = getTestContainerID();
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
ChunkInfo info = writeChunkHelper(blockID);
|
|
|
- KeyData keyData = new KeyData(blockID);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
chunkList.add(info.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
- keyManager.putKey(container, keyData);
|
|
|
- KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ blockManager.putBlock(container, blockData);
|
|
|
+ BlockData readBlockData = blockManager.
|
|
|
+ getBlock(container, blockData.getBlockID());
|
|
|
ChunkInfo readChunk =
|
|
|
- ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(0));
|
|
|
+ ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(0));
|
|
|
Assert.assertEquals(info.getChecksum(), readChunk.getChecksum());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Tests a put key and read key.
|
|
|
+ * Tests a put block and read block.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
* @throws NoSuchAlgorithmException
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testPutKeyWithLotsOfChunks() throws IOException,
|
|
|
+ public void testPutBlockWithLotsOfChunks() throws IOException,
|
|
|
NoSuchAlgorithmException {
|
|
|
final int chunkCount = 2;
|
|
|
final int datalen = 1024;
|
|
@@ -603,66 +609,67 @@ public class TestContainerPersistence {
|
|
|
long writeCount = container.getContainerData().getWriteCount();
|
|
|
Assert.assertEquals(chunkCount, writeCount);
|
|
|
|
|
|
- KeyData keyData = new KeyData(blockID);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
List<ContainerProtos.ChunkInfo> chunkProtoList = new LinkedList<>();
|
|
|
for (ChunkInfo i : chunkList) {
|
|
|
chunkProtoList.add(i.getProtoBufMessage());
|
|
|
}
|
|
|
- keyData.setChunks(chunkProtoList);
|
|
|
- keyManager.putKey(container, keyData);
|
|
|
- KeyData readKeyData = keyManager.getKey(container, keyData.getBlockID());
|
|
|
+ blockData.setChunks(chunkProtoList);
|
|
|
+ blockManager.putBlock(container, blockData);
|
|
|
+ BlockData readBlockData = blockManager.
|
|
|
+ getBlock(container, blockData.getBlockID());
|
|
|
ChunkInfo lastChunk = chunkList.get(chunkList.size() - 1);
|
|
|
ChunkInfo readChunk =
|
|
|
- ChunkInfo.getFromProtoBuf(readKeyData.getChunks().get(readKeyData
|
|
|
+ ChunkInfo.getFromProtoBuf(readBlockData.getChunks().get(readBlockData
|
|
|
.getChunks().size() - 1));
|
|
|
Assert.assertEquals(lastChunk.getChecksum(), readChunk.getChecksum());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Deletes a key and tries to read it back.
|
|
|
+ * Deletes a block and tries to read it back.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
* @throws NoSuchAlgorithmException
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testDeleteKey() throws IOException, NoSuchAlgorithmException {
|
|
|
+ public void testDeleteBlock() throws IOException, NoSuchAlgorithmException {
|
|
|
long testContainerID = getTestContainerID();
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
ChunkInfo info = writeChunkHelper(blockID);
|
|
|
- KeyData keyData = new KeyData(blockID);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
chunkList.add(info.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
- keyManager.putKey(container, keyData);
|
|
|
- keyManager.deleteKey(container, blockID);
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ blockManager.putBlock(container, blockData);
|
|
|
+ blockManager.deleteBlock(container, blockID);
|
|
|
exception.expect(StorageContainerException.class);
|
|
|
- exception.expectMessage("Unable to find the key.");
|
|
|
- keyManager.getKey(container, keyData.getBlockID());
|
|
|
+ exception.expectMessage("Unable to find the block.");
|
|
|
+ blockManager.getBlock(container, blockData.getBlockID());
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Tries to Deletes a key twice.
|
|
|
+ * Tries to Deletes a block twice.
|
|
|
*
|
|
|
* @throws IOException
|
|
|
* @throws NoSuchAlgorithmException
|
|
|
*/
|
|
|
@Test
|
|
|
- public void testDeleteKeyTwice() throws IOException,
|
|
|
+ public void testDeleteBlockTwice() throws IOException,
|
|
|
NoSuchAlgorithmException {
|
|
|
long testContainerID = getTestContainerID();
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
BlockID blockID = ContainerTestHelper.getTestBlockID(testContainerID);
|
|
|
ChunkInfo info = writeChunkHelper(blockID);
|
|
|
- KeyData keyData = new KeyData(blockID);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
chunkList.add(info.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
- keyManager.putKey(container, keyData);
|
|
|
- keyManager.deleteKey(container, blockID);
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ blockManager.putBlock(container, blockData);
|
|
|
+ blockManager.deleteBlock(container, blockID);
|
|
|
exception.expect(StorageContainerException.class);
|
|
|
- exception.expectMessage("Unable to find the key.");
|
|
|
- keyManager.deleteKey(container, blockID);
|
|
|
+ exception.expectMessage("Unable to find the block.");
|
|
|
+ blockManager.deleteBlock(container, blockID);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -722,8 +729,9 @@ public class TestContainerPersistence {
|
|
|
try {
|
|
|
container.update(newMetadata, false);
|
|
|
} catch (StorageContainerException ex) {
|
|
|
- Assert.assertEquals("Updating a closed container without force option " +
|
|
|
- "is not allowed. ContainerID: " + testContainerID, ex.getMessage());
|
|
|
+ Assert.assertEquals("Updating a closed container without " +
|
|
|
+ "force option is not allowed. ContainerID: " +
|
|
|
+ testContainerID, ex.getMessage());
|
|
|
}
|
|
|
|
|
|
// Update with force flag, it should be success.
|
|
@@ -741,53 +749,55 @@ public class TestContainerPersistence {
|
|
|
|
|
|
}
|
|
|
|
|
|
- private KeyData writeKeyHelper(BlockID blockID)
|
|
|
+ private BlockData writeBlockHelper(BlockID blockID)
|
|
|
throws IOException, NoSuchAlgorithmException {
|
|
|
ChunkInfo info = writeChunkHelper(blockID);
|
|
|
- KeyData keyData = new KeyData(blockID);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
chunkList.add(info.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
- return keyData;
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ return blockData;
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
- public void testListKey() throws Exception {
|
|
|
+ public void testListBlock() throws Exception {
|
|
|
long testContainerID = getTestContainerID();
|
|
|
Container container = addContainer(containerSet, testContainerID);
|
|
|
- List<BlockID> expectedKeys = new ArrayList<>();
|
|
|
+ List<BlockID> expectedBlocks = new ArrayList<>();
|
|
|
for (int i = 0; i < 10; i++) {
|
|
|
BlockID blockID = new BlockID(testContainerID, i);
|
|
|
- expectedKeys.add(blockID);
|
|
|
- KeyData kd = writeKeyHelper(blockID);
|
|
|
- keyManager.putKey(container, kd);
|
|
|
+ expectedBlocks.add(blockID);
|
|
|
+ BlockData kd = writeBlockHelper(blockID);
|
|
|
+ blockManager.putBlock(container, kd);
|
|
|
}
|
|
|
|
|
|
- // List all keys
|
|
|
- List<KeyData> result = keyManager.listKey(container, 0, 100);
|
|
|
+ // List all blocks
|
|
|
+ List<BlockData> result = blockManager.listBlock(
|
|
|
+ container, 0, 100);
|
|
|
Assert.assertEquals(10, result.size());
|
|
|
|
|
|
int index = 0;
|
|
|
for (int i = index; i < result.size(); i++) {
|
|
|
- KeyData data = result.get(i);
|
|
|
+ BlockData data = result.get(i);
|
|
|
Assert.assertEquals(testContainerID, data.getContainerID());
|
|
|
- Assert.assertEquals(expectedKeys.get(i).getLocalID(), data.getLocalID());
|
|
|
+ Assert.assertEquals(expectedBlocks.get(i).getLocalID(),
|
|
|
+ data.getLocalID());
|
|
|
index++;
|
|
|
}
|
|
|
|
|
|
- // List key with startKey filter
|
|
|
- long k6 = expectedKeys.get(6).getLocalID();
|
|
|
- result = keyManager.listKey(container, k6, 100);
|
|
|
+ // List block with startBlock filter
|
|
|
+ long k6 = expectedBlocks.get(6).getLocalID();
|
|
|
+ result = blockManager.listBlock(container, k6, 100);
|
|
|
|
|
|
Assert.assertEquals(4, result.size());
|
|
|
for (int i = 6; i < 10; i++) {
|
|
|
- Assert.assertEquals(expectedKeys.get(i).getLocalID(),
|
|
|
+ Assert.assertEquals(expectedBlocks.get(i).getLocalID(),
|
|
|
result.get(i - 6).getLocalID());
|
|
|
}
|
|
|
|
|
|
// Count must be >0
|
|
|
exception.expect(IllegalArgumentException.class);
|
|
|
exception.expectMessage("Count must be a positive number.");
|
|
|
- keyManager.listKey(container, 0, -1);
|
|
|
+ blockManager.listBlock(container, 0, -1);
|
|
|
}
|
|
|
}
|