|
@@ -19,19 +19,27 @@
|
|
package org.apache.hadoop.ozone.container.ozoneimpl;
|
|
package org.apache.hadoop.ozone.container.ozoneimpl;
|
|
|
|
|
|
|
|
|
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
|
+import com.google.common.primitives.Longs;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.conf.StorageUnit;
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
import org.apache.hadoop.hdds.HddsConfigKeys;
|
|
|
|
+import org.apache.hadoop.hdds.client.BlockID;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
|
|
|
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
|
|
+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.impl.ContainerSet;
|
|
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
|
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
|
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
|
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
|
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
|
|
|
|
+import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
|
|
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
|
|
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.RoundRobinVolumeChoosingPolicy;
|
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
|
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer;
|
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
|
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
|
|
|
|
+import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
|
|
import org.junit.Before;
|
|
import org.junit.Before;
|
|
import org.junit.Rule;
|
|
import org.junit.Rule;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
@@ -40,7 +48,9 @@ import org.mockito.Mockito;
|
|
|
|
|
|
import java.util.Random;
|
|
import java.util.Random;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
-
|
|
|
|
|
|
+import java.util.HashMap;
|
|
|
|
+import java.util.List;
|
|
|
|
+import java.util.ArrayList;
|
|
|
|
|
|
import static org.junit.Assert.assertEquals;
|
|
import static org.junit.Assert.assertEquals;
|
|
|
|
|
|
@@ -52,7 +62,6 @@ public class TestOzoneContainer {
|
|
@Rule
|
|
@Rule
|
|
public TemporaryFolder folder = new TemporaryFolder();
|
|
public TemporaryFolder folder = new TemporaryFolder();
|
|
|
|
|
|
-
|
|
|
|
private OzoneConfiguration conf;
|
|
private OzoneConfiguration conf;
|
|
private String scmId = UUID.randomUUID().toString();
|
|
private String scmId = UUID.randomUUID().toString();
|
|
private VolumeSet volumeSet;
|
|
private VolumeSet volumeSet;
|
|
@@ -60,6 +69,8 @@ public class TestOzoneContainer {
|
|
private KeyValueContainerData keyValueContainerData;
|
|
private KeyValueContainerData keyValueContainerData;
|
|
private KeyValueContainer keyValueContainer;
|
|
private KeyValueContainer keyValueContainer;
|
|
private final DatanodeDetails datanodeDetails = createDatanodeDetails();
|
|
private final DatanodeDetails datanodeDetails = createDatanodeDetails();
|
|
|
|
+ private HashMap<String, Long> commitSpaceMap; //RootDir -> committed space
|
|
|
|
+ private final int numTestContainers = 10;
|
|
|
|
|
|
@Before
|
|
@Before
|
|
public void setUp() throws Exception {
|
|
public void setUp() throws Exception {
|
|
@@ -68,6 +79,7 @@ public class TestOzoneContainer {
|
|
.getAbsolutePath());
|
|
.getAbsolutePath());
|
|
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
|
|
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS,
|
|
folder.newFolder().getAbsolutePath());
|
|
folder.newFolder().getAbsolutePath());
|
|
|
|
+ commitSpaceMap = new HashMap<String, Long>();
|
|
}
|
|
}
|
|
|
|
|
|
@Test
|
|
@Test
|
|
@@ -78,16 +90,32 @@ public class TestOzoneContainer {
|
|
// Format the volumes
|
|
// Format the volumes
|
|
for (HddsVolume volume : volumeSet.getVolumesList()) {
|
|
for (HddsVolume volume : volumeSet.getVolumesList()) {
|
|
volume.format(UUID.randomUUID().toString());
|
|
volume.format(UUID.randomUUID().toString());
|
|
|
|
+ commitSpaceMap.put(getVolumeKey(volume), Long.valueOf(0));
|
|
}
|
|
}
|
|
|
|
|
|
// Add containers to disk
|
|
// Add containers to disk
|
|
- for (int i=0; i<10; i++) {
|
|
|
|
|
|
+ for (int i = 0; i < numTestContainers; i++) {
|
|
|
|
+ long freeBytes = 0;
|
|
|
|
+ long volCommitBytes;
|
|
|
|
+ long maxCap = (long) StorageUnit.GB.toBytes(1);
|
|
|
|
+
|
|
|
|
+ HddsVolume myVolume;
|
|
|
|
+
|
|
keyValueContainerData = new KeyValueContainerData(i,
|
|
keyValueContainerData = new KeyValueContainerData(i,
|
|
- (long) StorageUnit.GB.toBytes(1), UUID.randomUUID().toString(),
|
|
|
|
|
|
+ maxCap, UUID.randomUUID().toString(),
|
|
datanodeDetails.getUuidString());
|
|
datanodeDetails.getUuidString());
|
|
keyValueContainer = new KeyValueContainer(
|
|
keyValueContainer = new KeyValueContainer(
|
|
keyValueContainerData, conf);
|
|
keyValueContainerData, conf);
|
|
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
|
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
|
|
|
+ myVolume = keyValueContainer.getContainerData().getVolume();
|
|
|
|
+
|
|
|
|
+ freeBytes = addBlocks(keyValueContainer, 2, 3);
|
|
|
|
+
|
|
|
|
+ // update our expectation of volume committed space in the map
|
|
|
|
+ volCommitBytes = commitSpaceMap.get(getVolumeKey(myVolume)).longValue();
|
|
|
|
+ Preconditions.checkState(freeBytes >= 0);
|
|
|
|
+ commitSpaceMap.put(getVolumeKey(myVolume),
|
|
|
|
+ Long.valueOf(volCommitBytes + freeBytes));
|
|
}
|
|
}
|
|
|
|
|
|
DatanodeStateMachine stateMachine = Mockito.mock(
|
|
DatanodeStateMachine stateMachine = Mockito.mock(
|
|
@@ -97,12 +125,65 @@ public class TestOzoneContainer {
|
|
Mockito.when(context.getParent()).thenReturn(stateMachine);
|
|
Mockito.when(context.getParent()).thenReturn(stateMachine);
|
|
// When OzoneContainer is started, the containers from disk should be
|
|
// When OzoneContainer is started, the containers from disk should be
|
|
// loaded into the containerSet.
|
|
// loaded into the containerSet.
|
|
|
|
+ // Also expected to initialize committed space for each volume.
|
|
OzoneContainer ozoneContainer = new
|
|
OzoneContainer ozoneContainer = new
|
|
OzoneContainer(datanodeDetails, conf, context, null);
|
|
OzoneContainer(datanodeDetails, conf, context, null);
|
|
|
|
+
|
|
ContainerSet containerset = ozoneContainer.getContainerSet();
|
|
ContainerSet containerset = ozoneContainer.getContainerSet();
|
|
- assertEquals(10, containerset.containerCount());
|
|
|
|
|
|
+ assertEquals(numTestContainers, containerset.containerCount());
|
|
|
|
+
|
|
|
|
+ verifyCommittedSpace(ozoneContainer);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ //verify committed space on each volume
|
|
|
|
+ private void verifyCommittedSpace(OzoneContainer oc) {
|
|
|
|
+ for (HddsVolume dnVol : oc.getVolumeSet().getVolumesList()) {
|
|
|
|
+ String key = getVolumeKey(dnVol);
|
|
|
|
+ long expectedCommit = commitSpaceMap.get(key).longValue();
|
|
|
|
+ long volumeCommitted = dnVol.getCommittedBytes();
|
|
|
|
+ assertEquals("Volume committed space not initialized correctly",
|
|
|
|
+ expectedCommit, volumeCommitted);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private long addBlocks(KeyValueContainer container,
|
|
|
|
+ int blocks, int chunksPerBlock) throws Exception {
|
|
|
|
+ String strBlock = "block";
|
|
|
|
+ String strChunk = "-chunkFile";
|
|
|
|
+ int datalen = 65536;
|
|
|
|
+ long usedBytes = 0;
|
|
|
|
+
|
|
|
|
+ long freeBytes = container.getContainerData().getMaxSize();
|
|
|
|
+ long containerId = container.getContainerData().getContainerID();
|
|
|
|
+ ContainerCache.ReferenceCountedDB db = BlockUtils.getDB(container
|
|
|
|
+ .getContainerData(), conf);
|
|
|
|
+
|
|
|
|
+ for (int bi = 0; bi < blocks; bi++) {
|
|
|
|
+ // Creating BlockData
|
|
|
|
+ BlockID blockID = new BlockID(containerId, bi);
|
|
|
|
+ BlockData blockData = new BlockData(blockID);
|
|
|
|
+ List<ContainerProtos.ChunkInfo> chunkList = new ArrayList<>();
|
|
|
|
+
|
|
|
|
+ chunkList.clear();
|
|
|
|
+ for (int ci = 0; ci < chunksPerBlock; ci++) {
|
|
|
|
+ String chunkName = strBlock + bi + strChunk + ci;
|
|
|
|
+ long offset = ci * datalen;
|
|
|
|
+ ChunkInfo info = new ChunkInfo(chunkName, offset, datalen);
|
|
|
|
+ usedBytes += datalen;
|
|
|
|
+ chunkList.add(info.getProtoBufMessage());
|
|
|
|
+ }
|
|
|
|
+ blockData.setChunks(chunkList);
|
|
|
|
+ db.getStore().put(Longs.toByteArray(blockID.getLocalID()),
|
|
|
|
+ blockData.getProtoBufMessage().toByteArray());
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ // remaining available capacity of the container
|
|
|
|
+ return (freeBytes - usedBytes);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private String getVolumeKey(HddsVolume volume) {
|
|
|
|
+ return volume.getHddsRootDir().getPath();
|
|
|
|
+ }
|
|
|
|
|
|
private DatanodeDetails createDatanodeDetails() {
|
|
private DatanodeDetails createDatanodeDetails() {
|
|
Random random = new Random();
|
|
Random random = new Random();
|