Skip to content

Commit 9569015

Browse files
supratimdekaarp7
authored andcommitted
HDDS-1511. Space tracking for Open Containers in HDDS Volumes. Contributed by Supratim Deka (#812)
1 parent 570fa2d commit 9569015

File tree

4 files changed

+107
-0
lines changed

4 files changed

+107
-0
lines changed

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerData.java

Lines changed: 63 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -71,6 +71,8 @@ public abstract class ContainerData {
7171

7272
private final long maxSize;
7373

74+
private boolean committedSpace;
75+
7476
//ID of the pipeline where this container is created
7577
private String originPipelineId;
7678
//ID of the datanode where this container is created
@@ -184,7 +186,23 @@ public synchronized ContainerDataProto.State getState() {
184186
* @param state
185187
*/
186188
public synchronized void setState(ContainerDataProto.State state) {
189+
ContainerDataProto.State oldState = this.state;
187190
this.state = state;
191+
192+
if ((oldState == ContainerDataProto.State.OPEN) &&
193+
(state != oldState)) {
194+
releaseCommitSpace();
195+
}
196+
197+
/**
198+
* commit space when container transitions (back) to Open.
199+
* when? perhaps closing a container threw an exception
200+
*/
201+
if ((state == ContainerDataProto.State.OPEN) &&
202+
(state != oldState)) {
203+
Preconditions.checkState(getMaxSize() > 0);
204+
commitSpace();
205+
}
188206
}
189207

190208
/**
@@ -280,6 +298,41 @@ public synchronized void closeContainer() {
280298
setState(ContainerDataProto.State.CLOSED);
281299
}
282300

301+
private void releaseCommitSpace() {
302+
long unused = getMaxSize() - getBytesUsed();
303+
304+
// only if container size < max size
305+
if (unused > 0 && committedSpace) {
306+
getVolume().incCommittedBytes(0 - unused);
307+
}
308+
committedSpace = false;
309+
}
310+
311+
/**
312+
* add available space in the container to the committed space in the volume.
313+
* available space is the number of bytes remaining till max capacity.
314+
*/
315+
public void commitSpace() {
316+
long unused = getMaxSize() - getBytesUsed();
317+
ContainerDataProto.State myState = getState();
318+
HddsVolume cVol;
319+
320+
//we don't expect duplicate calls
321+
Preconditions.checkState(!committedSpace);
322+
323+
// Only Open Containers have Committed Space
324+
if (myState != ContainerDataProto.State.OPEN) {
325+
return;
326+
}
327+
328+
// junit tests do not always set up volume
329+
cVol = getVolume();
330+
if (unused > 0 && (cVol != null)) {
331+
cVol.incCommittedBytes(unused);
332+
committedSpace = true;
333+
}
334+
}
335+
283336
/**
284337
* Get the number of bytes read from the container.
285338
* @return the number of bytes read from the container.
@@ -321,10 +374,20 @@ public long getWriteBytes() {
321374

322375
/**
323376
* Increase the number of bytes write into the container.
377+
* Also decrement committed bytes against the bytes written.
324378
* @param bytes the number of bytes write into the container.
325379
*/
326380
public void incrWriteBytes(long bytes) {
381+
long unused = getMaxSize() - getBytesUsed();
382+
327383
this.writeBytes.addAndGet(bytes);
384+
385+
// only if container size < max size
386+
if (committedSpace && unused > 0) {
387+
//with this write, container size might breach max size
388+
long decrement = Math.min(bytes, unused);
389+
this.getVolume().incCommittedBytes(0 - decrement);
390+
}
328391
}
329392

330393
/**

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/ContainerSet.java

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -68,6 +68,8 @@ public boolean addContainer(Container container) throws
6868
if(containerMap.putIfAbsent(containerId, container) == null) {
6969
LOG.debug("Container with container Id {} is added to containerMap",
7070
containerId);
71+
// wish we could have done this from ContainerData.setState
72+
container.getContainerData().commitSpace();
7173
return true;
7274
} else {
7375
LOG.warn("Container already exists with container Id {}", containerId);

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -44,6 +44,7 @@
4444
import java.io.IOException;
4545
import java.util.Properties;
4646
import java.util.UUID;
47+
import java.util.concurrent.atomic.AtomicLong;
4748

4849
/**
4950
* HddsVolume represents volume in a datanode. {@link VolumeSet} maintains a
@@ -85,6 +86,7 @@ public class HddsVolume
8586
private String datanodeUuid; // id of the DataNode
8687
private long cTime; // creation time of the file system state
8788
private int layoutVersion; // layout version of the storage data
89+
private final AtomicLong committedBytes; // till Open containers become full
8890

8991
/**
9092
* Run a check on the current volume to determine if it is healthy.
@@ -168,6 +170,7 @@ private HddsVolume(Builder b) throws IOException {
168170
.storageType(b.storageType)
169171
.configuredCapacity(b.configuredCapacity);
170172
this.volumeInfo = volumeBuilder.build();
173+
this.committedBytes = new AtomicLong(0);
171174

172175
LOG.info("Creating Volume: " + this.hddsRootDir + " of storage type : " +
173176
b.storageType + " and capacity : " + volumeInfo.getCapacity());
@@ -181,6 +184,7 @@ private HddsVolume(Builder b) throws IOException {
181184
volumeInfo = null;
182185
storageID = UUID.randomUUID().toString();
183186
state = VolumeState.FAILED;
187+
committedBytes = null;
184188
}
185189
}
186190

@@ -421,6 +425,23 @@ public enum VolumeState {
421425
NOT_INITIALIZED
422426
}
423427

428+
/**
429+
* add "delta" bytes to committed space in the volume.
430+
* @param delta bytes to add to committed space counter
431+
* @return bytes of committed space
432+
*/
433+
public long incCommittedBytes(long delta) {
434+
return committedBytes.addAndGet(delta);
435+
}
436+
437+
/**
438+
* return the committed space in the volume.
439+
* @return bytes of committed space
440+
*/
441+
public long getCommittedBytes() {
442+
return committedBytes.get();
443+
}
444+
424445
/**
425446
* Only for testing. Do not use otherwise.
426447
*/

hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/container/common/impl/TestContainerPersistence.java

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -162,14 +162,25 @@ private DispatcherContext getDispatcherContext() {
162162

163163
private Container addContainer(ContainerSet cSet, long cID)
164164
throws IOException {
165+
long commitBytesBefore = 0;
166+
long commitBytesAfter = 0;
167+
long commitIncrement = 0;
165168
KeyValueContainerData data = new KeyValueContainerData(cID,
166169
ContainerTestHelper.CONTAINER_MAX_SIZE, UUID.randomUUID().toString(),
167170
UUID.randomUUID().toString());
168171
data.addMetadata("VOLUME", "shire");
169172
data.addMetadata("owner)", "bilbo");
170173
KeyValueContainer container = new KeyValueContainer(data, conf);
171174
container.create(volumeSet, volumeChoosingPolicy, SCM_ID);
175+
commitBytesBefore = container.getContainerData()
176+
.getVolume().getCommittedBytes();
172177
cSet.addContainer(container);
178+
commitBytesAfter = container.getContainerData()
179+
.getVolume().getCommittedBytes();
180+
commitIncrement = commitBytesAfter - commitBytesBefore;
181+
// did we commit space for the new container?
182+
Assert.assertTrue(commitIncrement ==
183+
ContainerTestHelper.CONTAINER_MAX_SIZE);
173184
return container;
174185
}
175186

@@ -328,6 +339,9 @@ public void testListContainer() throws IOException {
328339

329340
private ChunkInfo writeChunkHelper(BlockID blockID) throws IOException {
330341
final int datalen = 1024;
342+
long commitBytesBefore = 0;
343+
long commitBytesAfter = 0;
344+
long commitDecrement = 0;
331345
long testContainerID = blockID.getContainerID();
332346
Container container = containerSet.getContainer(testContainerID);
333347
if (container == null) {
@@ -337,8 +351,15 @@ private ChunkInfo writeChunkHelper(BlockID blockID) throws IOException {
337351
blockID.getLocalID(), 0, 0, datalen);
338352
byte[] data = getData(datalen);
339353
setDataChecksum(info, data);
354+
commitBytesBefore = container.getContainerData()
355+
.getVolume().getCommittedBytes();
340356
chunkManager.writeChunk(container, blockID, info, ByteBuffer.wrap(data),
341357
getDispatcherContext());
358+
commitBytesAfter = container.getContainerData()
359+
.getVolume().getCommittedBytes();
360+
commitDecrement = commitBytesBefore - commitBytesAfter;
361+
// did we decrement commit bytes by the amount of data we wrote?
362+
Assert.assertTrue(commitDecrement == info.getLen());
342363
return info;
343364

344365
}

0 commit comments

Comments
 (0)