|
@@ -22,13 +22,14 @@ import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.base.Preconditions;
|
|
import com.google.common.base.Preconditions;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.commons.io.FileUtils;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
-import org.apache.hadoop.fs.StorageType;
|
|
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
.StorageContainerException;
|
|
.StorageContainerException;
|
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
|
|
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.protocol.datanode.proto.ContainerProtos;
|
|
|
|
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|
|
|
+ .ContainerLifeCycleState;
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
.StorageContainerDatanodeProtocolProtos;
|
|
.StorageContainerDatanodeProtocolProtos;
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
@@ -39,8 +40,6 @@ import org.apache.hadoop.hdds.protocol.proto
|
|
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
|
|
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
import org.apache.hadoop.hdds.protocol.proto
|
|
.StorageContainerDatanodeProtocolProtos.SCMStorageReport;
|
|
.StorageContainerDatanodeProtocolProtos.SCMStorageReport;
|
|
-import org.apache.hadoop.hdds.protocol.proto
|
|
|
|
- .StorageContainerDatanodeProtocolProtos.StorageTypeProto;
|
|
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
import org.apache.hadoop.ozone.OzoneConsts;
|
|
@@ -116,7 +115,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
|
|
|
// TODO: consider primitive collection like eclipse-collections
|
|
// TODO: consider primitive collection like eclipse-collections
|
|
// to avoid autoboxing overhead
|
|
// to avoid autoboxing overhead
|
|
- private final ConcurrentSkipListMap<Long, ContainerStatus>
|
|
|
|
|
|
+ private final ConcurrentSkipListMap<Long, ContainerData>
|
|
containerMap = new ConcurrentSkipListMap<>();
|
|
containerMap = new ConcurrentSkipListMap<>();
|
|
|
|
|
|
// Use a non-fair RW lock for better throughput, we may revisit this decision
|
|
// Use a non-fair RW lock for better throughput, we may revisit this decision
|
|
@@ -246,19 +245,20 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
// when loading the info we get a null, this often means last time
|
|
// when loading the info we get a null, this often means last time
|
|
// SCM was ending up at some middle phase causing that the metadata
|
|
// SCM was ending up at some middle phase causing that the metadata
|
|
// was not populated. Such containers are marked as inactive.
|
|
// was not populated. Such containers are marked as inactive.
|
|
- containerMap.put(containerID, new ContainerStatus(null));
|
|
|
|
|
|
+ ContainerData cData = new ContainerData(containerID, conf,
|
|
|
|
+ ContainerLifeCycleState.INVALID);
|
|
|
|
+ containerMap.put(containerID, cData);
|
|
return;
|
|
return;
|
|
}
|
|
}
|
|
containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
|
|
containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
|
|
|
|
|
|
- ContainerStatus containerStatus = new ContainerStatus(containerData);
|
|
|
|
// Initialize pending deletion blocks count in in-memory
|
|
// Initialize pending deletion blocks count in in-memory
|
|
// container status.
|
|
// container status.
|
|
MetadataStore metadata = KeyUtils.getDB(containerData, conf);
|
|
MetadataStore metadata = KeyUtils.getDB(containerData, conf);
|
|
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
|
|
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
|
|
.getSequentialRangeKVs(null, Integer.MAX_VALUE,
|
|
.getSequentialRangeKVs(null, Integer.MAX_VALUE,
|
|
MetadataKeyFilters.getDeletingKeyFilter());
|
|
MetadataKeyFilters.getDeletingKeyFilter());
|
|
- containerStatus.incrPendingDeletionBlocks(underDeletionBlocks.size());
|
|
|
|
|
|
+ containerData.incrPendingDeletionBlocks(underDeletionBlocks.size());
|
|
|
|
|
|
List<Map.Entry<byte[], byte[]>> liveKeys = metadata
|
|
List<Map.Entry<byte[], byte[]>> liveKeys = metadata
|
|
.getRangeKVs(null, Integer.MAX_VALUE,
|
|
.getRangeKVs(null, Integer.MAX_VALUE,
|
|
@@ -277,9 +277,9 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
return 0L;
|
|
return 0L;
|
|
}
|
|
}
|
|
}).sum();
|
|
}).sum();
|
|
- containerStatus.setBytesUsed(bytesUsed);
|
|
|
|
|
|
+ containerData.setBytesUsed(bytesUsed);
|
|
|
|
|
|
- containerMap.put(containerID, containerStatus);
|
|
|
|
|
|
+ containerMap.put(containerID, containerData);
|
|
} catch (IOException ex) {
|
|
} catch (IOException ex) {
|
|
LOG.error("read failed for file: {} ex: {}", containerName,
|
|
LOG.error("read failed for file: {} ex: {}", containerName,
|
|
ex.getMessage());
|
|
ex.getMessage());
|
|
@@ -287,7 +287,9 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
// TODO : Add this file to a recovery Queue.
|
|
// TODO : Add this file to a recovery Queue.
|
|
|
|
|
|
// Remember that this container is busted and we cannot use it.
|
|
// Remember that this container is busted and we cannot use it.
|
|
- containerMap.put(containerID, new ContainerStatus(null));
|
|
|
|
|
|
+ ContainerData cData = new ContainerData(containerID, conf,
|
|
|
|
+ ContainerLifeCycleState.INVALID);
|
|
|
|
+ containerMap.put(containerID, cData);
|
|
throw new StorageContainerException("Unable to read container info",
|
|
throw new StorageContainerException("Unable to read container info",
|
|
UNABLE_TO_READ_METADATA_DB);
|
|
UNABLE_TO_READ_METADATA_DB);
|
|
} finally {
|
|
} finally {
|
|
@@ -456,18 +458,19 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
UNCLOSED_CONTAINER_IO);
|
|
UNCLOSED_CONTAINER_IO);
|
|
}
|
|
}
|
|
|
|
|
|
- ContainerStatus status = containerMap.get(containerID);
|
|
|
|
- if (status == null) {
|
|
|
|
|
|
+ ContainerData containerData = containerMap.get(containerID);
|
|
|
|
+ if (containerData == null) {
|
|
LOG.debug("No such container. ID: {}", containerID);
|
|
LOG.debug("No such container. ID: {}", containerID);
|
|
throw new StorageContainerException("No such container. ID : " +
|
|
throw new StorageContainerException("No such container. ID : " +
|
|
containerID, CONTAINER_NOT_FOUND);
|
|
containerID, CONTAINER_NOT_FOUND);
|
|
}
|
|
}
|
|
- if (status.getContainer() == null) {
|
|
|
|
|
|
+
|
|
|
|
+ if(!containerData.isValid()) {
|
|
LOG.debug("Invalid container data. ID: {}", containerID);
|
|
LOG.debug("Invalid container data. ID: {}", containerID);
|
|
throw new StorageContainerException("Invalid container data. Name : " +
|
|
throw new StorageContainerException("Invalid container data. Name : " +
|
|
containerID, CONTAINER_NOT_FOUND);
|
|
containerID, CONTAINER_NOT_FOUND);
|
|
}
|
|
}
|
|
- ContainerUtils.removeContainer(status.getContainer(), conf, forceDelete);
|
|
|
|
|
|
+ ContainerUtils.removeContainer(containerData, conf, forceDelete);
|
|
containerMap.remove(containerID);
|
|
containerMap.remove(containerID);
|
|
} catch (StorageContainerException e) {
|
|
} catch (StorageContainerException e) {
|
|
throw e;
|
|
throw e;
|
|
@@ -509,7 +512,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
|
|
|
readLock();
|
|
readLock();
|
|
try {
|
|
try {
|
|
- ConcurrentNavigableMap<Long, ContainerStatus> map;
|
|
|
|
|
|
+ ConcurrentNavigableMap<Long, ContainerData> map;
|
|
if (startContainerID == 0) {
|
|
if (startContainerID == 0) {
|
|
map = containerMap.tailMap(containerMap.firstKey(), true);
|
|
map = containerMap.tailMap(containerMap.firstKey(), true);
|
|
} else {
|
|
} else {
|
|
@@ -517,9 +520,9 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
}
|
|
}
|
|
|
|
|
|
int currentCount = 0;
|
|
int currentCount = 0;
|
|
- for (ContainerStatus entry : map.values()) {
|
|
|
|
|
|
+ for (ContainerData entry : map.values()) {
|
|
if (currentCount < count) {
|
|
if (currentCount < count) {
|
|
- data.add(entry.getContainer());
|
|
|
|
|
|
+ data.add(entry);
|
|
currentCount++;
|
|
currentCount++;
|
|
} else {
|
|
} else {
|
|
return;
|
|
return;
|
|
@@ -546,7 +549,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
throw new StorageContainerException("Unable to find the container. ID: "
|
|
throw new StorageContainerException("Unable to find the container. ID: "
|
|
+ containerID, CONTAINER_NOT_FOUND);
|
|
+ containerID, CONTAINER_NOT_FOUND);
|
|
}
|
|
}
|
|
- ContainerData cData = containerMap.get(containerID).getContainer();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerID);
|
|
if (cData == null) {
|
|
if (cData == null) {
|
|
throw new StorageContainerException("Invalid container data. ID: "
|
|
throw new StorageContainerException("Invalid container data. ID: "
|
|
+ containerID, CONTAINER_INTERNAL_ERROR);
|
|
+ containerID, CONTAINER_INTERNAL_ERROR);
|
|
@@ -584,8 +587,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
// I/O failure, this allows us to take quick action in case of container
|
|
// I/O failure, this allows us to take quick action in case of container
|
|
// issues.
|
|
// issues.
|
|
|
|
|
|
- ContainerStatus status = new ContainerStatus(containerData);
|
|
|
|
- containerMap.put(containerID, status);
|
|
|
|
|
|
+ containerMap.put(containerID, containerData);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -614,7 +616,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
|
|
|
try {
|
|
try {
|
|
Path location = locationManager.getContainerPath();
|
|
Path location = locationManager.getContainerPath();
|
|
- ContainerData orgData = containerMap.get(containerID).getContainer();
|
|
|
|
|
|
+ ContainerData orgData = containerMap.get(containerID);
|
|
if (orgData == null) {
|
|
if (orgData == null) {
|
|
// updating a invalid container
|
|
// updating a invalid container
|
|
throw new StorageContainerException("Update a container with invalid" +
|
|
throw new StorageContainerException("Update a container with invalid" +
|
|
@@ -652,8 +654,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
}
|
|
}
|
|
|
|
|
|
// Update the in-memory map
|
|
// Update the in-memory map
|
|
- ContainerStatus newStatus = new ContainerStatus(data);
|
|
|
|
- containerMap.replace(containerID, newStatus);
|
|
|
|
|
|
+ containerMap.replace(containerID, data);
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
// Restore the container file from backup
|
|
// Restore the container file from backup
|
|
if(containerFileBK != null && containerFileBK.exists() && deleted) {
|
|
if(containerFileBK != null && containerFileBK.exists() && deleted) {
|
|
@@ -699,17 +700,12 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public boolean isOpen(long containerID) throws StorageContainerException {
|
|
public boolean isOpen(long containerID) throws StorageContainerException {
|
|
- final ContainerStatus status = containerMap.get(containerID);
|
|
|
|
- if (status == null) {
|
|
|
|
- throw new StorageContainerException(
|
|
|
|
- "Container status not found: " + containerID, CONTAINER_NOT_FOUND);
|
|
|
|
- }
|
|
|
|
- final ContainerData cData = status.getContainer();
|
|
|
|
- if (cData == null) {
|
|
|
|
|
|
+ final ContainerData containerData = containerMap.get(containerID);
|
|
|
|
+ if (containerData == null) {
|
|
throw new StorageContainerException(
|
|
throw new StorageContainerException(
|
|
"Container not found: " + containerID, CONTAINER_NOT_FOUND);
|
|
"Container not found: " + containerID, CONTAINER_NOT_FOUND);
|
|
}
|
|
}
|
|
- return cData.isOpen();
|
|
|
|
|
|
+ return containerData.isOpen();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -727,7 +723,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
|
|
|
|
|
|
|
|
@VisibleForTesting
|
|
@VisibleForTesting
|
|
- public ConcurrentSkipListMap<Long, ContainerStatus> getContainerMap() {
|
|
|
|
|
|
+ public ConcurrentSkipListMap<Long, ContainerData> getContainerMap() {
|
|
return containerMap;
|
|
return containerMap;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -847,9 +843,9 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
// And we can never get the exact state since close might happen
|
|
// And we can never get the exact state since close might happen
|
|
// after we iterate a point.
|
|
// after we iterate a point.
|
|
return containerMap.entrySet().stream()
|
|
return containerMap.entrySet().stream()
|
|
- .filter(containerStatus ->
|
|
|
|
- !containerStatus.getValue().getContainer().isOpen())
|
|
|
|
- .map(containerStatus -> containerStatus.getValue().getContainer())
|
|
|
|
|
|
+ .filter(containerData ->
|
|
|
|
+ !containerData.getValue().isOpen())
|
|
|
|
+ .map(containerData -> containerData.getValue())
|
|
.collect(Collectors.toList());
|
|
.collect(Collectors.toList());
|
|
}
|
|
}
|
|
|
|
|
|
@@ -865,7 +861,7 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
// No need for locking since containerMap is a ConcurrentSkipListMap
|
|
// No need for locking since containerMap is a ConcurrentSkipListMap
|
|
// And we can never get the exact state since close might happen
|
|
// And we can never get the exact state since close might happen
|
|
// after we iterate a point.
|
|
// after we iterate a point.
|
|
- List<ContainerStatus> containers = containerMap.values().stream()
|
|
|
|
|
|
+ List<ContainerData> containers = containerMap.values().stream()
|
|
.collect(Collectors.toList());
|
|
.collect(Collectors.toList());
|
|
|
|
|
|
ContainerReportsRequestProto.Builder crBuilder =
|
|
ContainerReportsRequestProto.Builder crBuilder =
|
|
@@ -875,18 +871,17 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
|
|
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
|
|
.setType(ContainerReportsRequestProto.reportType.fullReport);
|
|
.setType(ContainerReportsRequestProto.reportType.fullReport);
|
|
|
|
|
|
- for (ContainerStatus container: containers) {
|
|
|
|
|
|
+ for (ContainerData container: containers) {
|
|
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
|
|
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
|
|
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
|
|
StorageContainerDatanodeProtocolProtos.ContainerInfo.newBuilder();
|
|
- ciBuilder.setContainerID(container.getContainer().getContainerID())
|
|
|
|
- .setSize(container.getContainer().getMaxSize())
|
|
|
|
- .setUsed(container.getContainer().getBytesUsed())
|
|
|
|
- .setKeyCount(container.getContainer().getKeyCount())
|
|
|
|
|
|
+ ciBuilder.setContainerID(container.getContainerID())
|
|
|
|
+ .setSize(container.getMaxSize())
|
|
|
|
+ .setUsed(container.getBytesUsed())
|
|
|
|
+ .setKeyCount(container.getKeyCount())
|
|
.setReadCount(container.getReadCount())
|
|
.setReadCount(container.getReadCount())
|
|
.setWriteCount(container.getWriteCount())
|
|
.setWriteCount(container.getWriteCount())
|
|
.setReadBytes(container.getReadBytes())
|
|
.setReadBytes(container.getReadBytes())
|
|
- .setWriteBytes(container.getWriteBytes())
|
|
|
|
- .setContainerID(container.getContainer().getContainerID());
|
|
|
|
|
|
+ .setWriteBytes(container.getWriteBytes());
|
|
|
|
|
|
crBuilder.addReports(ciBuilder.build());
|
|
crBuilder.addReports(ciBuilder.build());
|
|
}
|
|
}
|
|
@@ -943,8 +938,8 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
public void incrPendingDeletionBlocks(int numBlocks, long containerId) {
|
|
public void incrPendingDeletionBlocks(int numBlocks, long containerId) {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.incrPendingDeletionBlocks(numBlocks);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.incrPendingDeletionBlocks(numBlocks);
|
|
} finally {
|
|
} finally {
|
|
writeUnlock();
|
|
writeUnlock();
|
|
}
|
|
}
|
|
@@ -954,8 +949,8 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
public void decrPendingDeletionBlocks(int numBlocks, long containerId) {
|
|
public void decrPendingDeletionBlocks(int numBlocks, long containerId) {
|
|
writeLock();
|
|
writeLock();
|
|
try {
|
|
try {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.decrPendingDeletionBlocks(numBlocks);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.decrPendingDeletionBlocks(numBlocks);
|
|
} finally {
|
|
} finally {
|
|
writeUnlock();
|
|
writeUnlock();
|
|
}
|
|
}
|
|
@@ -968,32 +963,37 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void incrReadCount(long containerId) {
|
|
public void incrReadCount(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.incrReadCount();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.incrReadCount();
|
|
}
|
|
}
|
|
|
|
|
|
public long getReadCount(long containerId) {
|
|
public long getReadCount(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getReadCount();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getReadCount();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
- * Increse the read counter for bytes read from the container.
|
|
|
|
|
|
+ * Increase the read counter for bytes read from the container.
|
|
*
|
|
*
|
|
* @param containerId - ID of the container.
|
|
* @param containerId - ID of the container.
|
|
* @param readBytes - bytes read from the container.
|
|
* @param readBytes - bytes read from the container.
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void incrReadBytes(long containerId, long readBytes) {
|
|
public void incrReadBytes(long containerId, long readBytes) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.incrReadBytes(readBytes);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.incrReadBytes(readBytes);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Returns number of bytes read from the container.
|
|
|
|
+ * @param containerId
|
|
|
|
+ * @return
|
|
|
|
+ */
|
|
public long getReadBytes(long containerId) {
|
|
public long getReadBytes(long containerId) {
|
|
readLock();
|
|
readLock();
|
|
try {
|
|
try {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getReadBytes();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getReadBytes();
|
|
} finally {
|
|
} finally {
|
|
readUnlock();
|
|
readUnlock();
|
|
}
|
|
}
|
|
@@ -1006,13 +1006,13 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void incrWriteCount(long containerId) {
|
|
public void incrWriteCount(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.incrWriteCount();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.incrWriteCount();
|
|
}
|
|
}
|
|
|
|
|
|
public long getWriteCount(long containerId) {
|
|
public long getWriteCount(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getWriteCount();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getWriteCount();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1023,13 +1023,13 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public void incrWriteBytes(long containerId, long writeBytes) {
|
|
public void incrWriteBytes(long containerId, long writeBytes) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- status.incrWriteBytes(writeBytes);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ cData.incrWriteBytes(writeBytes);
|
|
}
|
|
}
|
|
|
|
|
|
public long getWriteBytes(long containerId) {
|
|
public long getWriteBytes(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getWriteBytes();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getWriteBytes();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1041,8 +1041,8 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public long incrBytesUsed(long containerId, long used) {
|
|
public long incrBytesUsed(long containerId, long used) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.incrBytesUsed(used);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.incrBytesUsed(used);
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1054,13 +1054,13 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public long decrBytesUsed(long containerId, long used) {
|
|
public long decrBytesUsed(long containerId, long used) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.decrBytesUsed(used);
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.decrBytesUsed(used);
|
|
}
|
|
}
|
|
|
|
|
|
public long getBytesUsed(long containerId) {
|
|
public long getBytesUsed(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getBytesUsed();
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getBytesUsed();
|
|
}
|
|
}
|
|
|
|
|
|
/**
|
|
/**
|
|
@@ -1071,8 +1071,8 @@ public class ContainerManagerImpl implements ContainerManager {
|
|
*/
|
|
*/
|
|
@Override
|
|
@Override
|
|
public long getNumKeys(long containerId) {
|
|
public long getNumKeys(long containerId) {
|
|
- ContainerStatus status = containerMap.get(containerId);
|
|
|
|
- return status.getNumKeys(); }
|
|
|
|
|
|
+ ContainerData cData = containerMap.get(containerId);
|
|
|
|
+ return cData.getKeyCount(); }
|
|
|
|
|
|
/**
|
|
/**
|
|
* Get the container report state to send via HB to SCM.
|
|
* Get the container report state to send via HB to SCM.
|