|
@@ -22,10 +22,12 @@ import org.apache.hadoop.hdds.client.BlockID;
|
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
|
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
|
|
+import org.apache.hadoop.ozone.OzoneConsts;
|
|
|
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
|
|
|
-import org.apache.hadoop.ozone.container.common.helpers.KeyData;
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.helpers.ChunkUtils;
|
|
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
|
|
import org.apache.hadoop.ozone.container.common.volume.RoundRobinVolumeChoosingPolicy;
|
|
|
+import org.apache.hadoop.ozone.container.common.volume.VolumeIOStats;
|
|
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
|
import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerImpl;
|
|
|
import org.apache.hadoop.test.GenericTestUtils;
|
|
@@ -37,13 +39,9 @@ import org.mockito.Mockito;
|
|
|
|
|
|
import java.io.File;
|
|
|
import java.util.Arrays;
|
|
|
-import java.util.LinkedList;
|
|
|
-import java.util.List;
|
|
|
import java.util.UUID;
|
|
|
|
|
|
-import static org.junit.Assert.assertEquals;
|
|
|
-import static org.junit.Assert.assertTrue;
|
|
|
-import static org.junit.Assert.fail;
|
|
|
+import static org.junit.Assert.*;
|
|
|
import static org.mockito.ArgumentMatchers.anyList;
|
|
|
import static org.mockito.ArgumentMatchers.anyLong;
|
|
|
import static org.mockito.Mockito.mock;
|
|
@@ -57,21 +55,21 @@ public class TestChunkManagerImpl {
|
|
|
private String scmId = UUID.randomUUID().toString();
|
|
|
private VolumeSet volumeSet;
|
|
|
private RoundRobinVolumeChoosingPolicy volumeChoosingPolicy;
|
|
|
+ private HddsVolume hddsVolume;
|
|
|
private KeyValueContainerData keyValueContainerData;
|
|
|
private KeyValueContainer keyValueContainer;
|
|
|
- private KeyData keyData;
|
|
|
private BlockID blockID;
|
|
|
private ChunkManagerImpl chunkManager;
|
|
|
private ChunkInfo chunkInfo;
|
|
|
private byte[] data;
|
|
|
+
|
|
|
@Rule
|
|
|
public TemporaryFolder folder = new TemporaryFolder();
|
|
|
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
|
config = new OzoneConfiguration();
|
|
|
-
|
|
|
- HddsVolume hddsVolume = new HddsVolume.Builder(folder.getRoot()
|
|
|
+ hddsVolume = new HddsVolume.Builder(folder.getRoot()
|
|
|
.getAbsolutePath()).conf(config).datanodeUuid(UUID.randomUUID()
|
|
|
.toString()).build();
|
|
|
|
|
@@ -83,22 +81,15 @@ public class TestChunkManagerImpl {
|
|
|
|
|
|
keyValueContainerData = new KeyValueContainerData(1L);
|
|
|
|
|
|
- keyValueContainer = new KeyValueContainer(
|
|
|
- keyValueContainerData, config);
|
|
|
+ keyValueContainer = new KeyValueContainer(keyValueContainerData, config);
|
|
|
|
|
|
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
|
|
|
|
|
data = "testing write chunks".getBytes();
|
|
|
// Creating KeyData
|
|
|
blockID = new BlockID(1L, 1L);
|
|
|
- keyData = new KeyData(blockID);
|
|
|
- keyData.addMetadata("VOLUME", "ozone");
|
|
|
- keyData.addMetadata("OWNER", "hdfs");
|
|
|
- List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
|
|
.getLocalID(), 0), 0, data.length);
|
|
|
- chunkList.add(chunkInfo.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
|
|
|
// Create a ChunkManager object.
|
|
|
chunkManager = new ChunkManagerImpl();
|
|
@@ -113,16 +104,38 @@ public class TestChunkManagerImpl {
|
|
|
assertTrue(chunksPath.exists());
|
|
|
// Initially chunks folder should be empty.
|
|
|
assertTrue(chunksPath.listFiles().length == 0);
|
|
|
+
|
|
|
+ // As no chunks are written to the volume writeBytes should be 0
|
|
|
+ checkWriteIOStats(0, 0);
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.WRITE_DATA);
|
|
|
// Now a chunk file is being written with Stage WRITE_DATA, so it should
|
|
|
// create a temporary chunk file.
|
|
|
assertTrue(chunksPath.listFiles().length == 1);
|
|
|
+
|
|
|
+ File chunkFile = ChunkUtils.getChunkFile(keyValueContainerData, chunkInfo);
|
|
|
+ File tempChunkFile = new File(chunkFile.getParent(),
|
|
|
+ chunkFile.getName() +
|
|
|
+ OzoneConsts.CONTAINER_CHUNK_NAME_DELIMITER +
|
|
|
+ OzoneConsts.CONTAINER_TEMPORARY_CHUNK_PREFIX);
|
|
|
+
|
|
|
+ // As chunk write stage is WRITE_DATA, temp chunk file will be created.
|
|
|
+ assertTrue(tempChunkFile.exists());
|
|
|
+
|
|
|
+ checkWriteIOStats(data.length, 1);
|
|
|
+
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.COMMIT_DATA);
|
|
|
+
|
|
|
+ checkWriteIOStats(data.length, 1);
|
|
|
+
|
|
|
// Old temp file should have been renamed to chunk file.
|
|
|
assertTrue(chunksPath.listFiles().length == 1);
|
|
|
|
|
|
+ // As commit happened, chunk file should exist.
|
|
|
+ assertTrue(chunkFile.exists());
|
|
|
+ assertFalse(tempChunkFile.exists());
|
|
|
+
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -131,13 +144,12 @@ public class TestChunkManagerImpl {
|
|
|
long randomLength = 200L;
|
|
|
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
|
|
.getLocalID(), 0), 0, randomLength);
|
|
|
- List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
- chunkList.add(chunkInfo.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.WRITE_DATA);
|
|
|
fail("testWriteChunkIncorrectLength failed");
|
|
|
} catch (StorageContainerException ex) {
|
|
|
+ // As we got an exception, writeBytes should be 0.
|
|
|
+ checkWriteIOStats(0, 0);
|
|
|
GenericTestUtils.assertExceptionContains("data array does not match " +
|
|
|
"the length ", ex);
|
|
|
assertEquals(ContainerProtos.Result.INVALID_WRITE_SIZE, ex.getResult());
|
|
@@ -152,21 +164,29 @@ public class TestChunkManagerImpl {
|
|
|
assertTrue(chunksPath.exists());
|
|
|
// Initially chunks folder should be empty.
|
|
|
assertTrue(chunksPath.listFiles().length == 0);
|
|
|
+ checkWriteIOStats(0, 0);
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.COMBINED);
|
|
|
- // Now a chunk file is being written with Stage WRITE_DATA, so it should
|
|
|
- // create a temporary chunk file.
|
|
|
+ // Now a chunk file is being written with Stage COMBINED_DATA, so it should
|
|
|
+ // create a chunk file.
|
|
|
assertTrue(chunksPath.listFiles().length == 1);
|
|
|
+ File chunkFile = ChunkUtils.getChunkFile(keyValueContainerData, chunkInfo);
|
|
|
+ assertTrue(chunkFile.exists());
|
|
|
+ checkWriteIOStats(data.length, 1);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
|
public void testReadChunk() throws Exception {
|
|
|
+ checkWriteIOStats(0, 0);
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.COMBINED);
|
|
|
+ checkWriteIOStats(data.length, 1);
|
|
|
+ checkReadIOStats(0, 0);
|
|
|
byte[] expectedData = chunkManager.readChunk(keyValueContainer, blockID,
|
|
|
chunkInfo);
|
|
|
assertEquals(expectedData.length, data.length);
|
|
|
assertTrue(Arrays.equals(expectedData, data));
|
|
|
+ checkReadIOStats(data.length, 1);
|
|
|
}
|
|
|
|
|
|
@Test
|
|
@@ -187,9 +207,6 @@ public class TestChunkManagerImpl {
|
|
|
long randomLength = 200L;
|
|
|
chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
|
|
.getLocalID(), 0), 0, randomLength);
|
|
|
- List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
- chunkList.add(chunkInfo.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
chunkManager.deleteChunk(keyValueContainer, blockID, chunkInfo);
|
|
|
fail("testDeleteChunkUnsupportedRequest");
|
|
|
} catch (StorageContainerException ex) {
|
|
@@ -205,9 +222,6 @@ public class TestChunkManagerImpl {
|
|
|
.getLocalID(), 0), 0, data.length);
|
|
|
//Setting checksum to some value.
|
|
|
chunkInfo.setChecksum("some garbage");
|
|
|
- List<ContainerProtos.ChunkInfo> chunkList = new LinkedList<>();
|
|
|
- chunkList.add(chunkInfo.getProtoBufMessage());
|
|
|
- keyData.setChunks(chunkList);
|
|
|
chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
ContainerProtos.Stage.COMBINED);
|
|
|
fail("testWriteChunkChecksumMismatch failed");
|
|
@@ -231,5 +245,46 @@ public class TestChunkManagerImpl {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testWriteAndReadChunkMultipleTimes() throws Exception {
|
|
|
+ for (int i=0; i<100; i++) {
|
|
|
+ chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
|
|
+ .getLocalID(), i), 0, data.length);
|
|
|
+ chunkManager.writeChunk(keyValueContainer, blockID, chunkInfo, data,
|
|
|
+ ContainerProtos.Stage.COMBINED);
|
|
|
+ }
|
|
|
+ checkWriteIOStats(data.length*100, 100);
|
|
|
+ assertTrue(hddsVolume.getVolumeIOStats().getWriteTime() > 0);
|
|
|
+
|
|
|
+ for (int i=0; i<100; i++) {
|
|
|
+ chunkInfo = new ChunkInfo(String.format("%d.data.%d", blockID
|
|
|
+ .getLocalID(), i), 0, data.length);
|
|
|
+ chunkManager.readChunk(keyValueContainer, blockID, chunkInfo);
|
|
|
+ }
|
|
|
+ checkReadIOStats(data.length*100, 100);
|
|
|
+ assertTrue(hddsVolume.getVolumeIOStats().getReadTime() > 0);
|
|
|
+ }
|
|
|
+
|
|
|
|
|
|
+ /**
|
|
|
+ * Check WriteIO stats.
|
|
|
+ * @param length
|
|
|
+ * @param opCount
|
|
|
+ */
|
|
|
+ private void checkWriteIOStats(long length, long opCount) {
|
|
|
+ VolumeIOStats volumeIOStats = hddsVolume.getVolumeIOStats();
|
|
|
+ assertEquals(length, volumeIOStats.getWriteBytes());
|
|
|
+ assertEquals(opCount, volumeIOStats.getWriteOpCount());
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Check ReadIO stats.
|
|
|
+ * @param length
|
|
|
+ * @param opCount
|
|
|
+ */
|
|
|
+ private void checkReadIOStats(long length, long opCount) {
|
|
|
+ VolumeIOStats volumeIOStats = hddsVolume.getVolumeIOStats();
|
|
|
+ assertEquals(length, volumeIOStats.getReadBytes());
|
|
|
+ assertEquals(opCount, volumeIOStats.getReadOpCount());
|
|
|
+ }
|
|
|
}
|