HDDS-82. Merge ContainerData and ContainerStatus classes. Contributed by Bharat Viswanadham.

This commit is contained in:
Xiaoyu Yao 2018-05-21 16:09:24 -07:00
parent 0b4c44bdee
commit 5e88126776
8 changed files with 257 additions and 320 deletions

View File

@ -152,6 +152,7 @@ enum ContainerLifeCycleState {
OPEN = 1;
CLOSING = 2;
CLOSED = 3;
INVALID = 4;
}
message ContainerCommandRequestProto {

View File

@ -52,6 +52,17 @@ public class ContainerData {
private ContainerType containerType;
private String containerDBType;
/**
* Number of pending deletion blocks in container.
*/
private int numPendingDeletionBlocks;
private AtomicLong readBytes;
private AtomicLong writeBytes;
private AtomicLong readCount;
private AtomicLong writeCount;
/**
* Constructs a ContainerData Object.
*
@ -66,6 +77,34 @@ public class ContainerData {
this.bytesUsed = new AtomicLong(0L);
this.containerID = containerID;
this.state = ContainerLifeCycleState.OPEN;
this.numPendingDeletionBlocks = 0;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
}
/**
* Constructs a ContainerData Object.
*
* @param containerID - ID
* @param conf - Configuration
* @param state - ContainerLifeCycleState
* @param
*/
public ContainerData(long containerID, Configuration conf,
ContainerLifeCycleState state) {
this.metadata = new TreeMap<>();
this.maxSize = conf.getLong(ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_KEY,
ScmConfigKeys.SCM_CONTAINER_CLIENT_MAX_SIZE_DEFAULT) * OzoneConsts.GB;
this.bytesUsed = new AtomicLong(0L);
this.containerID = containerID;
this.state = state;
this.numPendingDeletionBlocks = 0;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
}
/**
@ -292,6 +331,14 @@ public class ContainerData {
return ContainerLifeCycleState.OPEN == state;
}
/**
* checks if the container is invalid.
* @return - boolean
*/
public boolean isValid() {
return !(ContainerLifeCycleState.INVALID == state);
}
/**
* Marks this container as closed.
*/
@ -317,11 +364,119 @@ public class ContainerData {
this.bytesUsed.set(used);
}
public long addBytesUsed(long delta) {
return this.bytesUsed.addAndGet(delta);
}
/**
* Get the number of bytes used by the container.
* @return the number of bytes used by the container.
*/
public long getBytesUsed() {
return bytesUsed.get();
}
/**
* Increase the number of bytes used by the container.
* @param used number of bytes used by the container.
* @return the current number of bytes used by the container afert increase.
*/
public long incrBytesUsed(long used) {
return this.bytesUsed.addAndGet(used);
}
/**
* Decrease the number of bytes used by the container.
* @param reclaimed the number of bytes reclaimed from the container.
* @return the current number of bytes used by the container after decrease.
*/
public long decrBytesUsed(long reclaimed) {
return this.bytesUsed.addAndGet(-1L * reclaimed);
}
/**
* Increase the count of pending deletion blocks.
*
* @param numBlocks increment number
*/
public void incrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks += numBlocks;
}
/**
* Decrease the count of pending deletion blocks.
*
* @param numBlocks decrement number
*/
public void decrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks -= numBlocks;
}
/**
* Get the number of pending deletion blocks.
*/
public int getNumPendingDeletionBlocks() {
return this.numPendingDeletionBlocks;
}
/**
* Get the number of bytes read from the container.
* @return the number of bytes read from the container.
*/
public long getReadBytes() {
return readBytes.get();
}
/**
* Increase the number of bytes read from the container.
* @param bytes number of bytes read.
*/
public void incrReadBytes(long bytes) {
this.readBytes.addAndGet(bytes);
}
/**
* Get the number of times the container is read.
* @return the number of times the container is read.
*/
public long getReadCount() {
return readCount.get();
}
/**
* Increase the number of container read count by 1.
*/
public void incrReadCount() {
this.readCount.incrementAndGet();
}
/**
* Get the number of bytes write into the container.
* @return the number of bytes write into the container.
*/
public long getWriteBytes() {
return writeBytes.get();
}
/**
* Increase the number of bytes write into the container.
* @param bytes the number of bytes write into the container.
*/
public void incrWriteBytes(long bytes) {
this.writeBytes.addAndGet(bytes);
}
/**
* Get the number of writes into the container.
* @return the number of writes into the container.
*/
public long getWriteCount() {
return writeCount.get();
}
/**
* Increase the number of writes into the container by 1.
*/
public void incrWriteCount() {
this.writeCount.incrementAndGet();
}
}

View File

@ -22,13 +22,14 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.commons.io.FileUtils;
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.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.hdfs.server.datanode.StorageLocation;
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
.ContainerLifeCycleState;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
@ -39,8 +40,6 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMNodeReport;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMStorageReport;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageTypeProto;
import org.apache.hadoop.io.IOUtils;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -116,7 +115,7 @@ public class ContainerManagerImpl implements ContainerManager {
// TODO: consider primitive collection like eclipse-collections
// to avoid autoboxing overhead
private final ConcurrentSkipListMap<Long, ContainerStatus>
private final ConcurrentSkipListMap<Long, ContainerData>
containerMap = new ConcurrentSkipListMap<>();
// 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
// SCM was ending up at some middle phase causing that the metadata
// 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;
}
containerData = ContainerData.getFromProtBuf(containerDataProto, conf);
ContainerStatus containerStatus = new ContainerStatus(containerData);
// Initialize pending deletion blocks count in in-memory
// container status.
MetadataStore metadata = KeyUtils.getDB(containerData, conf);
List<Map.Entry<byte[], byte[]>> underDeletionBlocks = metadata
.getSequentialRangeKVs(null, Integer.MAX_VALUE,
MetadataKeyFilters.getDeletingKeyFilter());
containerStatus.incrPendingDeletionBlocks(underDeletionBlocks.size());
containerData.incrPendingDeletionBlocks(underDeletionBlocks.size());
List<Map.Entry<byte[], byte[]>> liveKeys = metadata
.getRangeKVs(null, Integer.MAX_VALUE,
@ -277,9 +277,9 @@ public class ContainerManagerImpl implements ContainerManager {
return 0L;
}
}).sum();
containerStatus.setBytesUsed(bytesUsed);
containerData.setBytesUsed(bytesUsed);
containerMap.put(containerID, containerStatus);
containerMap.put(containerID, containerData);
} catch (IOException ex) {
LOG.error("read failed for file: {} ex: {}", containerName,
ex.getMessage());
@ -287,7 +287,9 @@ public class ContainerManagerImpl implements ContainerManager {
// TODO : Add this file to a recovery Queue.
// 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",
UNABLE_TO_READ_METADATA_DB);
} finally {
@ -456,18 +458,19 @@ public class ContainerManagerImpl implements ContainerManager {
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);
throw new StorageContainerException("No such container. ID : " +
containerID, CONTAINER_NOT_FOUND);
}
if (status.getContainer() == null) {
if(!containerData.isValid()) {
LOG.debug("Invalid container data. ID: {}", containerID);
throw new StorageContainerException("Invalid container data. Name : " +
containerID, CONTAINER_NOT_FOUND);
}
ContainerUtils.removeContainer(status.getContainer(), conf, forceDelete);
ContainerUtils.removeContainer(containerData, conf, forceDelete);
containerMap.remove(containerID);
} catch (StorageContainerException e) {
throw e;
@ -509,7 +512,7 @@ public class ContainerManagerImpl implements ContainerManager {
readLock();
try {
ConcurrentNavigableMap<Long, ContainerStatus> map;
ConcurrentNavigableMap<Long, ContainerData> map;
if (startContainerID == 0) {
map = containerMap.tailMap(containerMap.firstKey(), true);
} else {
@ -517,9 +520,9 @@ public class ContainerManagerImpl implements ContainerManager {
}
int currentCount = 0;
for (ContainerStatus entry : map.values()) {
for (ContainerData entry : map.values()) {
if (currentCount < count) {
data.add(entry.getContainer());
data.add(entry);
currentCount++;
} else {
return;
@ -546,7 +549,7 @@ public class ContainerManagerImpl implements ContainerManager {
throw new StorageContainerException("Unable to find the container. ID: "
+ containerID, CONTAINER_NOT_FOUND);
}
ContainerData cData = containerMap.get(containerID).getContainer();
ContainerData cData = containerMap.get(containerID);
if (cData == null) {
throw new StorageContainerException("Invalid container data. ID: "
+ 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
// issues.
ContainerStatus status = new ContainerStatus(containerData);
containerMap.put(containerID, status);
containerMap.put(containerID, containerData);
}
@Override
@ -614,7 +616,7 @@ public class ContainerManagerImpl implements ContainerManager {
try {
Path location = locationManager.getContainerPath();
ContainerData orgData = containerMap.get(containerID).getContainer();
ContainerData orgData = containerMap.get(containerID);
if (orgData == null) {
// updating a invalid container
throw new StorageContainerException("Update a container with invalid" +
@ -652,8 +654,7 @@ public class ContainerManagerImpl implements ContainerManager {
}
// Update the in-memory map
ContainerStatus newStatus = new ContainerStatus(data);
containerMap.replace(containerID, newStatus);
containerMap.replace(containerID, data);
} catch (IOException e) {
// Restore the container file from backup
if(containerFileBK != null && containerFileBK.exists() && deleted) {
@ -699,17 +700,12 @@ public class ContainerManagerImpl implements ContainerManager {
*/
@Override
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(
"Container not found: " + containerID, CONTAINER_NOT_FOUND);
}
return cData.isOpen();
return containerData.isOpen();
}
/**
@ -727,7 +723,7 @@ public class ContainerManagerImpl implements ContainerManager {
@VisibleForTesting
public ConcurrentSkipListMap<Long, ContainerStatus> getContainerMap() {
public ConcurrentSkipListMap<Long, ContainerData> getContainerMap() {
return containerMap;
}
@ -847,9 +843,9 @@ public class ContainerManagerImpl implements ContainerManager {
// And we can never get the exact state since close might happen
// after we iterate a point.
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());
}
@ -865,7 +861,7 @@ public class ContainerManagerImpl implements ContainerManager {
// No need for locking since containerMap is a ConcurrentSkipListMap
// And we can never get the exact state since close might happen
// after we iterate a point.
List<ContainerStatus> containers = containerMap.values().stream()
List<ContainerData> containers = containerMap.values().stream()
.collect(Collectors.toList());
ContainerReportsRequestProto.Builder crBuilder =
@ -875,18 +871,17 @@ public class ContainerManagerImpl implements ContainerManager {
crBuilder.setDatanodeDetails(datanodeDetails.getProtoBufMessage())
.setType(ContainerReportsRequestProto.reportType.fullReport);
for (ContainerStatus container: containers) {
for (ContainerData container: containers) {
StorageContainerDatanodeProtocolProtos.ContainerInfo.Builder ciBuilder =
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())
.setWriteCount(container.getWriteCount())
.setReadBytes(container.getReadBytes())
.setWriteBytes(container.getWriteBytes())
.setContainerID(container.getContainer().getContainerID());
.setWriteBytes(container.getWriteBytes());
crBuilder.addReports(ciBuilder.build());
}
@ -943,8 +938,8 @@ public class ContainerManagerImpl implements ContainerManager {
public void incrPendingDeletionBlocks(int numBlocks, long containerId) {
writeLock();
try {
ContainerStatus status = containerMap.get(containerId);
status.incrPendingDeletionBlocks(numBlocks);
ContainerData cData = containerMap.get(containerId);
cData.incrPendingDeletionBlocks(numBlocks);
} finally {
writeUnlock();
}
@ -954,8 +949,8 @@ public class ContainerManagerImpl implements ContainerManager {
public void decrPendingDeletionBlocks(int numBlocks, long containerId) {
writeLock();
try {
ContainerStatus status = containerMap.get(containerId);
status.decrPendingDeletionBlocks(numBlocks);
ContainerData cData = containerMap.get(containerId);
cData.decrPendingDeletionBlocks(numBlocks);
} finally {
writeUnlock();
}
@ -968,32 +963,37 @@ public class ContainerManagerImpl implements ContainerManager {
*/
@Override
public void incrReadCount(long containerId) {
ContainerStatus status = containerMap.get(containerId);
status.incrReadCount();
ContainerData cData = containerMap.get(containerId);
cData.incrReadCount();
}
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 readBytes - bytes read from the container.
*/
@Override
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) {
readLock();
try {
ContainerStatus status = containerMap.get(containerId);
return status.getReadBytes();
ContainerData cData = containerMap.get(containerId);
return cData.getReadBytes();
} finally {
readUnlock();
}
@ -1006,13 +1006,13 @@ public class ContainerManagerImpl implements ContainerManager {
*/
@Override
public void incrWriteCount(long containerId) {
ContainerStatus status = containerMap.get(containerId);
status.incrWriteCount();
ContainerData cData = containerMap.get(containerId);
cData.incrWriteCount();
}
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
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) {
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
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
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) {
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
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.

View File

@ -1,217 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.impl;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import java.util.concurrent.atomic.AtomicLong;
/**
* This class represents the state of a container. if the
* container reading encountered an error when we boot up we will post that
* info to a recovery queue and keep the info in the containerMap.
* <p/>
* if and when the issue is fixed, the expectation is that this entry will be
* deleted by the recovery thread from the containerMap and will insert entry
* instead of modifying this class.
*/
public class ContainerStatus {
private final ContainerData containerData;
/**
* Number of pending deletion blocks in container.
*/
private int numPendingDeletionBlocks;
private AtomicLong readBytes;
private AtomicLong writeBytes;
private AtomicLong readCount;
private AtomicLong writeCount;
/**
* Creates a Container Status class.
*
* @param containerData - ContainerData.
*/
ContainerStatus(ContainerData containerData) {
this.numPendingDeletionBlocks = 0;
this.containerData = containerData;
this.readCount = new AtomicLong(0L);
this.readBytes = new AtomicLong(0L);
this.writeCount = new AtomicLong(0L);
this.writeBytes = new AtomicLong(0L);
}
/**
* Returns container if it is active. It is not active if we have had an
* error and we are waiting for the background threads to fix the issue.
*
* @return ContainerData.
*/
public ContainerData getContainer() {
return containerData;
}
/**
* Increase the count of pending deletion blocks.
*
* @param numBlocks increment number
*/
public void incrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks += numBlocks;
}
/**
* Decrease the count of pending deletion blocks.
*
* @param numBlocks decrement number
*/
public void decrPendingDeletionBlocks(int numBlocks) {
this.numPendingDeletionBlocks -= numBlocks;
}
/**
* Get the number of pending deletion blocks.
*/
public int getNumPendingDeletionBlocks() {
return this.numPendingDeletionBlocks;
}
/**
* Get the number of bytes read from the container.
* @return the number of bytes read from the container.
*/
public long getReadBytes() {
return readBytes.get();
}
/**
* Increase the number of bytes read from the container.
* @param bytes number of bytes read.
*/
public void incrReadBytes(long bytes) {
this.readBytes.addAndGet(bytes);
}
/**
* Get the number of times the container is read.
* @return the number of times the container is read.
*/
public long getReadCount() {
return readCount.get();
}
/**
* Increase the number of container read count by 1.
*/
public void incrReadCount() {
this.readCount.incrementAndGet();
}
/**
* Get the number of bytes write into the container.
* @return the number of bytes write into the container.
*/
public long getWriteBytes() {
return writeBytes.get();
}
/**
* Increase the number of bytes write into the container.
* @param bytes the number of bytes write into the container.
*/
public void incrWriteBytes(long bytes) {
this.writeBytes.addAndGet(bytes);
}
/**
* Get the number of writes into the container.
* @return the number of writes into the container.
*/
public long getWriteCount() {
return writeCount.get();
}
/**
* Increase the number of writes into the container by 1.
*/
public void incrWriteCount() {
this.writeCount.incrementAndGet();
}
/**
* Get the number of bytes used by the container.
* @return the number of bytes used by the container.
*/
public long getBytesUsed() {
return containerData.getBytesUsed();
}
/**
* Increase the number of bytes used by the container.
* @param used number of bytes used by the container.
* @return the current number of bytes used by the container afert increase.
*/
public long incrBytesUsed(long used) {
return containerData.addBytesUsed(used);
}
/**
* Set the number of bytes used by the container.
* @param used the number of bytes used by the container.
*/
public void setBytesUsed(long used) {
containerData.setBytesUsed(used);
}
/**
* Decrease the number of bytes used by the container.
* @param reclaimed the number of bytes reclaimed from the container.
* @return the current number of bytes used by the container after decrease.
*/
public long decrBytesUsed(long reclaimed) {
return this.containerData.addBytesUsed(-1L * reclaimed);
}
/**
* Get the maximum container size.
* @return the maximum container size.
*/
public long getMaxSize() {
return containerData.getMaxSize();
}
/**
* Set the maximum container size.
* @param size the maximum container size.
*/
public void setMaxSize(long size) {
this.containerData.setMaxSize(size);
}
/**
* Get the number of keys in the container.
* @return the number of keys in the container.
*/
public long getNumKeys() {
return containerData.getKeyCount();
}
}

View File

@ -41,24 +41,24 @@ public class RandomContainerDeletionChoosingPolicy
@Override
public List<ContainerData> chooseContainerForBlockDeletion(int count,
Map<Long, ContainerStatus> candidateContainers)
Map<Long, ContainerData> candidateContainers)
throws StorageContainerException {
Preconditions.checkNotNull(candidateContainers,
"Internal assertion: candidate containers cannot be null");
int currentCount = 0;
List<ContainerData> result = new LinkedList<>();
ContainerStatus[] values = new ContainerStatus[candidateContainers.size()];
ContainerData[] values = new ContainerData[candidateContainers.size()];
// to get a shuffle list
for (ContainerStatus entry : DFSUtil.shuffle(
for (ContainerData entry : DFSUtil.shuffle(
candidateContainers.values().toArray(values))) {
if (currentCount < count) {
result.add(entry.getContainer());
result.add(entry);
currentCount++;
LOG.debug("Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainer().getContainerID(),
entry.getContainerID(),
entry.getNumPendingDeletionBlocks());
} else {
break;

View File

@ -41,11 +41,11 @@ public class TopNOrderedContainerDeletionChoosingPolicy
private static final Logger LOG =
LoggerFactory.getLogger(TopNOrderedContainerDeletionChoosingPolicy.class);
/** customized comparator used to compare differentiate container status. **/
private static final Comparator<ContainerStatus> CONTAINER_STATUS_COMPARATOR
= new Comparator<ContainerStatus>() {
/** customized comparator used to compare differentiate container data. **/
private static final Comparator<ContainerData> CONTAINER_DATA_COMPARATOR
= new Comparator<ContainerData>() {
@Override
public int compare(ContainerStatus c1, ContainerStatus c2) {
public int compare(ContainerData c1, ContainerData c2) {
return Integer.compare(c2.getNumPendingDeletionBlocks(),
c1.getNumPendingDeletionBlocks());
}
@ -53,28 +53,28 @@ public class TopNOrderedContainerDeletionChoosingPolicy
@Override
public List<ContainerData> chooseContainerForBlockDeletion(int count,
Map<Long, ContainerStatus> candidateContainers)
Map<Long, ContainerData> candidateContainers)
throws StorageContainerException {
Preconditions.checkNotNull(candidateContainers,
"Internal assertion: candidate containers cannot be null");
List<ContainerData> result = new LinkedList<>();
List<ContainerStatus> orderedList = new LinkedList<>();
List<ContainerData> orderedList = new LinkedList<>();
orderedList.addAll(candidateContainers.values());
Collections.sort(orderedList, CONTAINER_STATUS_COMPARATOR);
Collections.sort(orderedList, CONTAINER_DATA_COMPARATOR);
// get top N list ordered by pending deletion blocks' number
int currentCount = 0;
for (ContainerStatus entry : orderedList) {
for (ContainerData entry : orderedList) {
if (currentCount < count) {
if (entry.getNumPendingDeletionBlocks() > 0) {
result.add(entry.getContainer());
result.add(entry);
currentCount++;
LOG.debug(
"Select container {} for block deletion, "
+ "pending deletion blocks num: {}.",
entry.getContainer().getContainerID(),
entry.getContainerID(),
entry.getNumPendingDeletionBlocks());
} else {
LOG.debug("Stop looking for next container, there is no"

View File

@ -20,7 +20,6 @@ package org.apache.hadoop.ozone.container.common.interfaces;
import org.apache.hadoop.hdds.scm.container.common.helpers
.StorageContainerException;
import org.apache.hadoop.ozone.container.common.helpers.ContainerData;
import org.apache.hadoop.ozone.container.common.impl.ContainerStatus;
import java.util.List;
import java.util.Map;
@ -41,6 +40,6 @@ public interface ContainerDeletionChoosingPolicy {
* @throws StorageContainerException
*/
List<ContainerData> chooseContainerForBlockDeletion(int count,
Map<Long, ContainerStatus> candidateContainers)
Map<Long, ContainerData> candidateContainers)
throws StorageContainerException;
}

View File

@ -65,7 +65,6 @@ import java.util.LinkedList;
import java.util.List;
import java.util.Map;
import java.util.ArrayList;
import java.util.UUID;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
import static org.apache.hadoop.ozone.OzoneConsts.CONTAINER_ROOT_PREFIX;
@ -191,23 +190,23 @@ public class TestContainerPersistence {
containerManager.createContainer(data);
Assert.assertTrue(containerManager.getContainerMap()
.containsKey(testContainerID));
ContainerStatus status = containerManager
ContainerData cData = containerManager
.getContainerMap().get(testContainerID);
Assert.assertNotNull(status.getContainer());
Assert.assertNotNull(status.getContainer().getContainerPath());
Assert.assertNotNull(status.getContainer().getDBPath());
Assert.assertNotNull(cData);
Assert.assertNotNull(cData.getContainerPath());
Assert.assertNotNull(cData.getDBPath());
Assert.assertTrue(new File(status.getContainer().getContainerPath())
Assert.assertTrue(new File(cData.getContainerPath())
.exists());
Path meta = Paths.get(status.getContainer().getDBPath()).getParent();
Path meta = Paths.get(cData.getDBPath()).getParent();
Assert.assertTrue(meta != null && Files.exists(meta));
MetadataStore store = null;
try {
store = KeyUtils.getDB(status.getContainer(), conf);
store = KeyUtils.getDB(cData, conf);
Assert.assertNotNull(store);
} finally {
if (store != null) {
@ -762,7 +761,7 @@ public class TestContainerPersistence {
// Verify in-memory map
ContainerData actualNewData = containerManager.getContainerMap()
.get(testContainerID).getContainer();
.get(testContainerID);
Assert.assertEquals("shire_new",
actualNewData.getAllMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new",
@ -805,7 +804,7 @@ public class TestContainerPersistence {
// Verify in-memory map
actualNewData = containerManager.getContainerMap()
.get(testContainerID).getContainer();
.get(testContainerID);
Assert.assertEquals("shire_new_1",
actualNewData.getAllMetadata().get("VOLUME"));
Assert.assertEquals("bilbo_new_1",