|
@@ -34,7 +34,7 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
|
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
|
import org.apache.hadoop.utils.MetadataKeyFilters;
|
|
|
-import org.apache.hadoop.utils.MetadataStore;
|
|
|
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache.ReferenceCountedDB;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Before;
|
|
|
import org.junit.Test;
|
|
@@ -109,30 +109,31 @@ public class TestKeyValueBlockIterator {
|
|
|
createContainerWithBlocks(containerID, normalBlocks, deletedBlocks);
|
|
|
String containerPath = new File(containerData.getMetadataPath())
|
|
|
.getParent();
|
|
|
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
- containerID, new File(containerPath));
|
|
|
-
|
|
|
- int counter = 0;
|
|
|
- while(keyValueBlockIterator.hasNext()) {
|
|
|
- BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
- assertEquals(blockData.getLocalID(), counter++);
|
|
|
- }
|
|
|
-
|
|
|
- assertFalse(keyValueBlockIterator.hasNext());
|
|
|
-
|
|
|
- keyValueBlockIterator.seekToFirst();
|
|
|
- counter = 0;
|
|
|
- while(keyValueBlockIterator.hasNext()) {
|
|
|
- BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
- assertEquals(blockData.getLocalID(), counter++);
|
|
|
- }
|
|
|
- assertFalse(keyValueBlockIterator.hasNext());
|
|
|
-
|
|
|
- try {
|
|
|
- keyValueBlockIterator.nextBlock();
|
|
|
- } catch (NoSuchElementException ex) {
|
|
|
- GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
- "for ContainerID " + containerID, ex);
|
|
|
+ try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
+ containerID, new File(containerPath))) {
|
|
|
+
|
|
|
+ int counter = 0;
|
|
|
+ while (keyValueBlockIterator.hasNext()) {
|
|
|
+ BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
+ assertEquals(blockData.getLocalID(), counter++);
|
|
|
+ }
|
|
|
+
|
|
|
+ assertFalse(keyValueBlockIterator.hasNext());
|
|
|
+
|
|
|
+ keyValueBlockIterator.seekToFirst();
|
|
|
+ counter = 0;
|
|
|
+ while (keyValueBlockIterator.hasNext()) {
|
|
|
+ BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
+ assertEquals(blockData.getLocalID(), counter++);
|
|
|
+ }
|
|
|
+ assertFalse(keyValueBlockIterator.hasNext());
|
|
|
+
|
|
|
+ try {
|
|
|
+ keyValueBlockIterator.nextBlock();
|
|
|
+ } catch (NoSuchElementException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
+ "for ContainerID " + containerID, ex);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -142,17 +143,18 @@ public class TestKeyValueBlockIterator {
|
|
|
createContainerWithBlocks(containerID, 2, 0);
|
|
|
String containerPath = new File(containerData.getMetadataPath())
|
|
|
.getParent();
|
|
|
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
- containerID, new File(containerPath));
|
|
|
- long blockID = 0L;
|
|
|
- assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
- assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
-
|
|
|
- try {
|
|
|
- keyValueBlockIterator.nextBlock();
|
|
|
- } catch (NoSuchElementException ex) {
|
|
|
- GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
- "for ContainerID " + containerID, ex);
|
|
|
+ try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
+ containerID, new File(containerPath))) {
|
|
|
+ long blockID = 0L;
|
|
|
+ assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+ assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+
|
|
|
+ try {
|
|
|
+ keyValueBlockIterator.nextBlock();
|
|
|
+ } catch (NoSuchElementException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
+ "for ContainerID " + containerID, ex);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -162,42 +164,41 @@ public class TestKeyValueBlockIterator {
|
|
|
createContainerWithBlocks(containerID, 2, 0);
|
|
|
String containerPath = new File(containerData.getMetadataPath())
|
|
|
.getParent();
|
|
|
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
- containerID, new File(containerPath));
|
|
|
- long blockID = 0L;
|
|
|
-
|
|
|
- // Even calling multiple times hasNext() should not move entry forward.
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
-
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
-
|
|
|
- keyValueBlockIterator.seekToLast();
|
|
|
- assertTrue(keyValueBlockIterator.hasNext());
|
|
|
- assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
-
|
|
|
- keyValueBlockIterator.seekToFirst();
|
|
|
- blockID = 0L;
|
|
|
- assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
- assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
-
|
|
|
- try {
|
|
|
- keyValueBlockIterator.nextBlock();
|
|
|
- } catch (NoSuchElementException ex) {
|
|
|
- GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
- "for ContainerID " + containerID, ex);
|
|
|
+ try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
+ containerID, new File(containerPath))) {
|
|
|
+ long blockID = 0L;
|
|
|
+
|
|
|
+ // Even calling multiple times hasNext() should not move entry forward.
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+
|
|
|
+ keyValueBlockIterator.seekToLast();
|
|
|
+ assertTrue(keyValueBlockIterator.hasNext());
|
|
|
+ assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+
|
|
|
+ keyValueBlockIterator.seekToFirst();
|
|
|
+ blockID = 0L;
|
|
|
+ assertEquals(blockID++, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+ assertEquals(blockID, keyValueBlockIterator.nextBlock().getLocalID());
|
|
|
+
|
|
|
+ try {
|
|
|
+ keyValueBlockIterator.nextBlock();
|
|
|
+ } catch (NoSuchElementException ex) {
|
|
|
+ GenericTestUtils.assertExceptionContains("Block Iterator reached end " +
|
|
|
+ "for ContainerID " + containerID, ex);
|
|
|
+ }
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -208,14 +209,15 @@ public class TestKeyValueBlockIterator {
|
|
|
createContainerWithBlocks(containerId, normalBlocks, deletedBlocks);
|
|
|
String containerPath = new File(containerData.getMetadataPath())
|
|
|
.getParent();
|
|
|
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
+ try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
containerId, new File(containerPath), MetadataKeyFilters
|
|
|
- .getDeletingKeyFilter());
|
|
|
+ .getDeletingKeyFilter())) {
|
|
|
|
|
|
- int counter = 5;
|
|
|
- while(keyValueBlockIterator.hasNext()) {
|
|
|
- BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
- assertEquals(blockData.getLocalID(), counter++);
|
|
|
+ int counter = 5;
|
|
|
+ while (keyValueBlockIterator.hasNext()) {
|
|
|
+ BlockData blockData = keyValueBlockIterator.nextBlock();
|
|
|
+ assertEquals(blockData.getLocalID(), counter++);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -226,11 +228,12 @@ public class TestKeyValueBlockIterator {
|
|
|
createContainerWithBlocks(containerId, 0, 5);
|
|
|
String containerPath = new File(containerData.getMetadataPath())
|
|
|
.getParent();
|
|
|
- KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
- containerId, new File(containerPath));
|
|
|
- //As all blocks are deleted blocks, blocks does not match with normal key
|
|
|
- // filter.
|
|
|
- assertFalse(keyValueBlockIterator.hasNext());
|
|
|
+ try(KeyValueBlockIterator keyValueBlockIterator = new KeyValueBlockIterator(
|
|
|
+ containerId, new File(containerPath))) {
|
|
|
+ //As all blocks are deleted blocks, blocks does not match with normal key
|
|
|
+ // filter.
|
|
|
+ assertFalse(keyValueBlockIterator.hasNext());
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -251,27 +254,30 @@ public class TestKeyValueBlockIterator {
|
|
|
container = new KeyValueContainer(containerData, conf);
|
|
|
container.create(volumeSet, new RoundRobinVolumeChoosingPolicy(), UUID
|
|
|
.randomUUID().toString());
|
|
|
- MetadataStore metadataStore = BlockUtils.getDB(containerData, conf);
|
|
|
-
|
|
|
- List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
|
|
|
- ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
|
|
|
- chunkList.add(info.getProtoBufMessage());
|
|
|
-
|
|
|
- for (int i=0; i<normalBlocks; i++) {
|
|
|
- BlockID blockID = new BlockID(containerId, i);
|
|
|
- BlockData blockData = new BlockData(blockID);
|
|
|
- blockData.setChunks(chunkList);
|
|
|
- metadataStore.put(Longs.toByteArray(blockID.getLocalID()), blockData
|
|
|
- .getProtoBufMessage().toByteArray());
|
|
|
- }
|
|
|
-
|
|
|
- for (int i=normalBlocks; i<deletedBlocks; i++) {
|
|
|
- BlockID blockID = new BlockID(containerId, i);
|
|
|
- BlockData blockData = new BlockData(blockID);
|
|
|
- blockData.setChunks(chunkList);
|
|
|
- metadataStore.put(DFSUtil.string2Bytes(OzoneConsts
|
|
|
- .DELETING_KEY_PREFIX + blockID.getLocalID()), blockData
|
|
|
- .getProtoBufMessage().toByteArray());
|
|
|
+ try(ReferenceCountedDB metadataStore = BlockUtils.getDB(containerData,
|
|
|
+ conf)) {
|
|
|
+
|
|
|
+ List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
|
|
|
+ ChunkInfo info = new ChunkInfo("chunkfile", 0, 1024);
|
|
|
+ chunkList.add(info.getProtoBufMessage());
|
|
|
+
|
|
|
+ for (int i = 0; i < normalBlocks; i++) {
|
|
|
+ BlockID blockID = new BlockID(containerId, i);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ metadataStore.getStore().put(Longs.toByteArray(blockID.getLocalID()),
|
|
|
+ blockData
|
|
|
+ .getProtoBufMessage().toByteArray());
|
|
|
+ }
|
|
|
+
|
|
|
+ for (int i = normalBlocks; i < deletedBlocks; i++) {
|
|
|
+ BlockID blockID = new BlockID(containerId, i);
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
+ metadataStore.getStore().put(DFSUtil.string2Bytes(OzoneConsts
|
|
|
+ .DELETING_KEY_PREFIX + blockID.getLocalID()), blockData
|
|
|
+ .getProtoBufMessage().toByteArray());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|