|
@@ -71,6 +71,8 @@ public abstract class ContainerData {
|
|
|
|
|
|
private final long maxSize;
|
|
|
|
|
|
+ private boolean committedSpace;
|
|
|
+
|
|
|
//ID of the pipeline where this container is created
|
|
|
private String originPipelineId;
|
|
|
//ID of the datanode where this container is created
|
|
@@ -184,7 +186,23 @@ public abstract class ContainerData {
|
|
|
* @param state
|
|
|
*/
|
|
|
public synchronized void setState(ContainerDataProto.State state) {
|
|
|
+ ContainerDataProto.State oldState = this.state;
|
|
|
this.state = state;
|
|
|
+
|
|
|
+ if ((oldState == ContainerDataProto.State.OPEN) &&
|
|
|
+ (state != oldState)) {
|
|
|
+ releaseCommitSpace();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * commit space when container transitions (back) to Open.
|
|
|
+ * when? perhaps closing a container threw an exception
|
|
|
+ */
|
|
|
+ if ((state == ContainerDataProto.State.OPEN) &&
|
|
|
+ (state != oldState)) {
|
|
|
+ Preconditions.checkState(getMaxSize() > 0);
|
|
|
+ commitSpace();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -280,6 +298,41 @@ public abstract class ContainerData {
|
|
|
setState(ContainerDataProto.State.CLOSED);
|
|
|
}
|
|
|
|
|
|
+ private void releaseCommitSpace() {
|
|
|
+ long unused = getMaxSize() - getBytesUsed();
|
|
|
+
|
|
|
+ // only if container size < max size
|
|
|
+ if (unused > 0 && committedSpace) {
|
|
|
+ getVolume().incCommittedBytes(0 - unused);
|
|
|
+ }
|
|
|
+ committedSpace = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * add available space in the container to the committed space in the volume.
|
|
|
+ * available space is the number of bytes remaining till max capacity.
|
|
|
+ */
|
|
|
+ public void commitSpace() {
|
|
|
+ long unused = getMaxSize() - getBytesUsed();
|
|
|
+ ContainerDataProto.State myState = getState();
|
|
|
+ HddsVolume cVol;
|
|
|
+
|
|
|
+ //we don't expect duplicate calls
|
|
|
+ Preconditions.checkState(!committedSpace);
|
|
|
+
|
|
|
+ // Only Open Containers have Committed Space
|
|
|
+ if (myState != ContainerDataProto.State.OPEN) {
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ // junit tests do not always set up volume
|
|
|
+ cVol = getVolume();
|
|
|
+ if (unused > 0 && (cVol != null)) {
|
|
|
+ cVol.incCommittedBytes(unused);
|
|
|
+ committedSpace = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get the number of bytes read from the container.
|
|
|
* @return the number of bytes read from the container.
|
|
@@ -321,10 +374,20 @@ public abstract class ContainerData {
|
|
|
|
|
|
/**
|
|
|
* Increase the number of bytes write into the container.
|
|
|
+ * Also decrement committed bytes against the bytes written.
|
|
|
* @param bytes the number of bytes write into the container.
|
|
|
*/
|
|
|
public void incrWriteBytes(long bytes) {
|
|
|
+ long unused = getMaxSize() - getBytesUsed();
|
|
|
+
|
|
|
this.writeBytes.addAndGet(bytes);
|
|
|
+
|
|
|
+ // only if container size < max size
|
|
|
+ if (committedSpace && unused > 0) {
|
|
|
+ //with this write, container size might breach max size
|
|
|
+ long decrement = Math.min(bytes, unused);
|
|
|
+ this.getVolume().incCommittedBytes(0 - decrement);
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
/**
|