HDDS-662. Introduce ContainerReplicaState in StorageContainerManager. Contributed by Nanda kumar.

This commit is contained in:
Nandakumar 2018-10-17 17:45:35 +05:30
parent a9a63ae4a8
commit 50715c0699
64 changed files with 1506 additions and 1296 deletions

View File

@ -21,7 +21,7 @@ import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdds.scm.client;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerData;

View File

@ -0,0 +1,46 @@
/**
* 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.hdds.scm.container;
import java.io.IOException;
/**
* Signals that ContainerException of some sort has occurred. This is parent
* of all the exceptions thrown by ContainerManager.
*/
public class ContainerException extends IOException {
/**
* Constructs an {@code ContainerException} with {@code null}
* as its error detail message.
*/
public ContainerException() {
super();
}
/**
* Constructs an {@code ContainerException} with the specified detail message.
*
* @param message
* The detail message (which is saved for later retrieval
* by the {@link #getMessage()} method)
*/
public ContainerException(String message) {
super(message);
}
}

View File

@ -19,6 +19,7 @@
package org.apache.hadoop.hdds.scm.container;
import com.google.common.base.Preconditions;
import com.google.common.primitives.Longs;
import org.apache.commons.lang3.builder.CompareToBuilder;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
@ -29,18 +30,17 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
* We are creating a specific type for this to avoid mixing this with
* normal integers in code.
*/
public class ContainerID implements Comparable {
public final class ContainerID implements Comparable<ContainerID> {
private final long id;
// TODO: make this private.
/**
* Constructs ContainerID.
*
* @param id int
*/
public ContainerID(long id) {
Preconditions.checkState(id > 0,
"Container ID should be a positive long. "+ id);
this.id = id;
}
@ -49,7 +49,9 @@ public class ContainerID implements Comparable {
* @param containerID long
* @return ContainerID.
*/
public static ContainerID valueof(long containerID) {
public static ContainerID valueof(final long containerID) {
Preconditions.checkState(containerID > 0,
"Container ID should be a positive long. "+ containerID);
return new ContainerID(containerID);
}
@ -62,8 +64,12 @@ public class ContainerID implements Comparable {
return id;
}
public byte[] getBytes() {
return Longs.toByteArray(id);
}
@Override
public boolean equals(Object o) {
public boolean equals(final Object o) {
if (this == o) {
return true;
}
@ -72,7 +78,7 @@ public class ContainerID implements Comparable {
return false;
}
ContainerID that = (ContainerID) o;
final ContainerID that = (ContainerID) o;
return new EqualsBuilder()
.append(getId(), that.getId())
@ -87,14 +93,8 @@ public class ContainerID implements Comparable {
}
@Override
public int compareTo(Object o) {
Preconditions.checkNotNull(o);
if(getClass() != o.getClass()) {
throw new ClassCastException("ContainerID class expected. found:" +
o.getClass().toString());
}
ContainerID that = (ContainerID) o;
public int compareTo(final ContainerID that) {
Preconditions.checkNotNull(that);
return new CompareToBuilder()
.append(this.getId(), that.getId())
.build();

View File

@ -15,7 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.container.common.helpers;
package org.apache.hadoop.hdds.scm.container;
import static java.lang.Math.max;
@ -36,7 +36,7 @@ import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.util.Time;
/**
@ -62,9 +62,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
private PipelineID pipelineID;
private ReplicationFactor replicationFactor;
private ReplicationType replicationType;
// Bytes allocated by SCM for clients.
private long allocatedBytes;
// Actual container usage, updated through heartbeat.
private long usedBytes;
private long numberOfKeys;
private long lastUsed;
@ -84,7 +81,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
long containerID,
HddsProtos.LifeCycleState state,
PipelineID pipelineID,
long allocatedBytes,
long usedBytes,
long numberOfKeys,
long stateEnterTime,
@ -94,7 +90,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
ReplicationType repType) {
this.containerID = containerID;
this.pipelineID = pipelineID;
this.allocatedBytes = allocatedBytes;
this.usedBytes = usedBytes;
this.numberOfKeys = numberOfKeys;
this.lastUsed = Time.monotonicNow();
@ -108,7 +103,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
public ContainerInfo(ContainerInfo info) {
this(info.getContainerID(), info.getState(), info.getPipelineID(),
info.getAllocatedBytes(), info.getUsedBytes(), info.getNumberOfKeys(),
info.getUsedBytes(), info.getNumberOfKeys(),
info.getStateEnterTime(), info.getOwner(),
info.getDeleteTransactionId(), info.getReplicationFactor(),
info.getReplicationType());
@ -123,7 +118,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
ContainerInfo.Builder builder = new ContainerInfo.Builder();
return builder.setPipelineID(
PipelineID.getFromProtobuf(info.getPipelineID()))
.setAllocatedBytes(info.getAllocatedBytes())
.setUsedBytes(info.getUsedBytes())
.setNumberOfKeys(info.getNumberOfKeys())
.setState(info.getState())
@ -160,20 +154,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
return pipelineID;
}
public long getAllocatedBytes() {
return allocatedBytes;
}
/**
* Set Allocated bytes.
*
* @param size - newly allocated bytes -- negative size is case of deletes
* can be used.
*/
public void updateAllocatedBytes(long size) {
this.allocatedBytes += size;
}
public long getUsedBytes() {
return usedBytes;
}
@ -211,18 +191,11 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
lastUsed = Time.monotonicNow();
}
public void allocate(long size) {
// should we also have total container size in ContainerInfo
// and check before allocating?
allocatedBytes += size;
}
public HddsProtos.SCMContainerInfo getProtobuf() {
HddsProtos.SCMContainerInfo.Builder builder =
HddsProtos.SCMContainerInfo.newBuilder();
Preconditions.checkState(containerID > 0);
return builder.setAllocatedBytes(getAllocatedBytes())
.setContainerID(getContainerID())
return builder.setContainerID(getContainerID())
.setUsedBytes(getUsedBytes())
.setNumberOfKeys(getNumberOfKeys()).setState(getState())
.setStateEnterTime(getStateEnterTime()).setContainerID(getContainerID())
@ -393,7 +366,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
*/
public static class Builder {
private HddsProtos.LifeCycleState state;
private long allocated;
private long used;
private long keys;
private long stateEnterTime;
@ -431,11 +403,6 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
return this;
}
public Builder setAllocatedBytes(long bytesAllocated) {
this.allocated = bytesAllocated;
return this;
}
public Builder setUsedBytes(long bytesUsed) {
this.used = bytesUsed;
return this;
@ -462,7 +429,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
}
public ContainerInfo build() {
return new ContainerInfo(containerID, state, pipelineID, allocated,
return new ContainerInfo(containerID, state, pipelineID,
used, keys, stateEnterTime, owner, deleteTransactionId,
replicationFactor, replicationType);
}
@ -473,7 +440,7 @@ public class ContainerInfo implements Comparator<ContainerInfo>,
* container is either open, allocated, creating or creating.
* Any containers in these states is managed as an open container by SCM.
*/
public boolean isContainerOpen() {
public boolean isOpen() {
return state == HddsProtos.LifeCycleState.ALLOCATED ||
state == HddsProtos.LifeCycleState.CREATING ||
state == HddsProtos.LifeCycleState.OPEN ||

View File

@ -0,0 +1,44 @@
/**
* 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.hdds.scm.container;
/**
* Signals that a container is missing from ContainerManager.
*/
public class ContainerNotFoundException extends ContainerException {
/**
* Constructs an {@code ContainerNotFoundException} with {@code null}
* as its error detail message.
*/
public ContainerNotFoundException() {
super();
}
/**
* Constructs an {@code ContainerNotFoundException} with the specified
* detail message.
*
* @param message
* The detail message (which is saved for later retrieval
* by the {@link #getMessage()} method)
*/
public ContainerNotFoundException(String message) {
super(message);
}
}

View File

@ -0,0 +1,45 @@
/**
* 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.hdds.scm.container;
/**
* Signals that a ContainerReplica is missing from the Container in
* ContainerManager.
*/
public class ContainerReplicaNotFoundException extends ContainerException {
/**
* Constructs an {@code ContainerReplicaNotFoundException} with {@code null}
* as its error detail message.
*/
public ContainerReplicaNotFoundException() {
super();
}
/**
* Constructs an {@code ContainerReplicaNotFoundException} with the
* specified detail message.
*
* @param message
* The detail message (which is saved for later retrieval
* by the {@link #getMessage()} method)
*/
public ContainerReplicaNotFoundException(String message) {
super(message);
}
}

View File

@ -22,6 +22,7 @@ import java.util.Comparator;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
/**
* Class wraps ozone container info.

View File

@ -19,7 +19,7 @@ package org.apache.hadoop.hdds.scm.protocol;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.InChillModeResponseProto;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;

View File

@ -33,7 +33,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolPro
import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.GetContainerWithPipelineResponseProto;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;

View File

@ -135,16 +135,13 @@ message SCMContainerInfo {
required int64 containerID = 1;
required LifeCycleState state = 2;
optional PipelineID pipelineID = 3;
// This is not total size of container, but space allocated by SCM for
// clients to write blocks
required uint64 allocatedBytes = 4;
required uint64 usedBytes = 5;
required uint64 numberOfKeys = 6;
optional int64 stateEnterTime = 7;
required string owner = 8;
optional int64 deleteTransactionId = 9;
required ReplicationFactor replicationFactor = 10;
required ReplicationType replicationType = 11;
required uint64 usedBytes = 4;
required uint64 numberOfKeys = 5;
optional int64 stateEnterTime = 6;
required string owner = 7;
optional int64 deleteTransactionId = 8;
required ReplicationFactor replicationFactor = 9;
required ReplicationType replicationType = 10;
}
message ContainerWithPipeline {

View File

@ -75,10 +75,10 @@ public class CommandStatusReportPublisher extends
iterator.forEachRemaining(key -> {
CommandStatus cmdStatus = map.get(key);
builder.addCmdStatus(cmdStatus.getProtoBufMessage());
// If status is still pending then don't remove it from map as
// CommandHandler will change its status when it works on this command.
if (!cmdStatus.getStatus().equals(Status.PENDING)) {
builder.addCmdStatus(cmdStatus.getProtoBufMessage());
map.remove(key);
}
});

View File

@ -153,19 +153,10 @@ public class TestReportPublisher {
.build();
cmdStatusMap.put(obj1.getCmdId(), obj1);
cmdStatusMap.put(obj2.getCmdId(), obj2);
Assert.assertEquals("Should publish report with 2 status objects", 2,
// We are not sending the commands whose status is PENDING.
Assert.assertEquals("Should publish report with 2 status objects", 1,
((CommandStatusReportPublisher) publisher).getReport()
.getCmdStatusCount());
Assert.assertEquals(
"Next report should have 1 status objects as command status o"
+ "bjects are still in Pending state",
1, ((CommandStatusReportPublisher) publisher).getReport()
.getCmdStatusCount());
Assert.assertTrue(
"Next report should have 1 status objects as command status "
+ "objects are still in Pending state",
((CommandStatusReportPublisher) publisher).getReport()
.getCmdStatusList().get(0).getStatus().equals(Status.PENDING));
executorService.shutdown();
}

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.scm.ScmUtils;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
@ -69,7 +69,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
// Currently only user of the block service is Ozone, CBlock manages blocks
// by itself and does not rely on the Block service offered by SCM.
private final NodeManager nodeManager;
private final ContainerManager containerManager;
private final long containerSize;
@ -95,7 +94,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
final NodeManager nodeManager, final ContainerManager containerManager,
EventPublisher eventPublisher)
throws IOException {
this.nodeManager = nodeManager;
this.containerManager = containerManager;
this.containerSize = (long)conf.getStorageSize(
@ -226,8 +224,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
// USER of the containers. So there might be cases where a different
// USER has few containers in ALLOCATED state, which will result in
// false positive.
if (!containerManager.getStateManager().getContainerStateMap()
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
if (!containerManager.getContainers(HddsProtos.LifeCycleState.ALLOCATED)
.isEmpty()) {
// Since the above check can result in false positive, we have to do
// the actual check and find out if there are containers in ALLOCATED
@ -242,7 +239,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
HddsProtos.LifeCycleState.ALLOCATED);
if (containerWithPipeline != null) {
containerManager.updateContainerState(
containerWithPipeline.getContainerInfo().getContainerID(),
containerWithPipeline.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
return newBlock(containerWithPipeline,
HddsProtos.LifeCycleState.ALLOCATED);
@ -268,8 +265,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
// state, we have to check again as we only hold a read lock.
// Some other thread might have pre-allocated container in meantime.
synchronized (this) {
if (!containerManager.getStateManager().getContainerStateMap()
.getContainerIDsByState(HddsProtos.LifeCycleState.ALLOCATED)
if (!containerManager.getContainers(HddsProtos.LifeCycleState.ALLOCATED)
.isEmpty()) {
containerWithPipeline = containerManager
.getMatchingContainerWithPipeline(size, owner, type, factor,
@ -285,7 +281,7 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
if (containerWithPipeline != null) {
containerManager.updateContainerState(
containerWithPipeline.getContainerInfo().getContainerID(),
containerWithPipeline.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
return newBlock(containerWithPipeline,
HddsProtos.LifeCycleState.ALLOCATED);

View File

@ -17,6 +17,7 @@
package org.apache.hadoop.hdds.scm.block;
import com.google.common.collect.ArrayListMultimap;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
@ -60,8 +61,9 @@ public class DatanodeDeletedBlockTransactions {
Pipeline pipeline = null;
try {
ContainerWithPipeline containerWithPipeline =
containerManager.getContainerWithPipeline(tx.getContainerID());
if (containerWithPipeline.getContainerInfo().isContainerOpen()
containerManager.getContainerWithPipeline(
ContainerID.valueof(tx.getContainerID()));
if (containerWithPipeline.getContainerInfo().isOpen()
|| containerWithPipeline.getPipeline().isEmpty()) {
return false;
}

View File

@ -29,6 +29,7 @@ import org.apache.hadoop.hdds.protocol.proto
.DeleteBlockTransactionResult;
import org.apache.hadoop.hdds.scm.command
.CommandStatusReportHandler.DeleteBlockStatus;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.server.events.EventHandler;
@ -258,8 +259,8 @@ public class DeletedBlockLogImpl
dnsWithCommittedTxn.add(dnID);
Pipeline pipeline =
containerManager.getContainerWithPipeline(containerId)
.getPipeline();
containerManager.getContainerWithPipeline(
ContainerID.valueof(containerId)).getPipeline();
Collection<DatanodeDetails> containerDnsDetails =
pipeline.getDatanodes().values();
// The delete entry can be safely removed from the log if all the

View File

@ -18,7 +18,6 @@ package org.apache.hadoop.hdds.scm.container;
import java.io.IOException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.server.events.EventHandler;
@ -61,7 +60,7 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
ContainerInfo info;
try {
containerWithPipeline =
containerManager.getContainerWithPipeline(containerID.getId());
containerManager.getContainerWithPipeline(containerID);
info = containerWithPipeline.getContainerInfo();
if (info == null) {
LOG.error("Failed to update the container state. Container with id : {}"
@ -81,8 +80,8 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
// We cannot close a container in ALLOCATED state, moving the
// container to CREATING state, this should eventually
// timeout and the container will be moved to DELETING state.
LOG.debug("Closing container {} in {} state", containerID, state);
containerManager.updateContainerState(containerID.getId(),
LOG.debug("Closing container #{} in {} state", containerID, state);
containerManager.updateContainerState(containerID,
HddsProtos.LifeCycleEvent.CREATE);
break;
case CREATING:
@ -91,7 +90,7 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
LOG.debug("Closing container {} in {} state", containerID, state);
break;
case OPEN:
containerManager.updateContainerState(containerID.getId(),
containerManager.updateContainerState(containerID,
HddsProtos.LifeCycleEvent.FINALIZE);
fireCloseContainerEvents(containerWithPipeline, info, publisher);
break;
@ -101,16 +100,15 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
case CLOSED:
case DELETING:
case DELETED:
LOG.info(
"container with id : {} is in {} state and need not be closed.",
containerID.getId(), info.getState());
LOG.info("Cannot close container #{}, it is already in {} state.",
containerID.getId(), state);
break;
default:
throw new IOException(
"Invalid container state for container " + containerID);
throw new IOException("Invalid container state for container #"
+ containerID);
}
} catch (IOException ex) {
LOG.error("Failed to update the container state for" + "container : {}"
LOG.error("Failed to update the container state for container #{}"
+ containerID, ex);
}
}
@ -125,13 +123,14 @@ public class CloseContainerEventHandler implements EventHandler<ContainerID> {
info.getReplicationType(), info.getPipelineID());
Pipeline pipeline = containerWithPipeline.getPipeline();
pipeline.getMachines().stream().map(
datanode -> new CommandForDatanode<>(datanode.getUuid(),
closeContainerCommand)).forEach((command) -> {
publisher.fireEvent(DATANODE_COMMAND, command);
});
pipeline.getMachines().stream()
.map(node ->
new CommandForDatanode<>(node.getUuid(), closeContainerCommand))
.forEach(command -> publisher.fireEvent(DATANODE_COMMAND, command));
publisher.fireEvent(CLOSE_CONTAINER_RETRYABLE_REQ,
new CloseContainerRetryableReq(containerID));
LOG.trace("Issuing {} on Pipeline {} for container", closeContainerCommand,
pipeline, containerID);
}

View File

@ -88,7 +88,8 @@ public class CloseContainerWatcher extends
publisher) {
try {
// Check if container is still open
if (containerManager.getContainer(containerID).isContainerOpen()) {
if (containerManager.getContainer(
ContainerID.valueof(containerID)).isOpen()) {
publisher.fireEvent(SCMEvents.CLOSE_CONTAINER,
ContainerID.valueof(containerID));
}

View File

@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
@ -31,13 +30,31 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.Set;
// TODO: Write extensive java doc.
// This is the main interface of ContainerManager.
/**
* ContainerManager class contains the mapping from a name to a pipeline
* mapping. This is used by SCM when allocating new locations and when
* looking up a key.
*/
public interface ContainerManager extends Closeable {
/**
* Returns all the containers managed by ContainerManager.
*
* @return List of ContainerInfo
*/
List<ContainerInfo> getContainers();
/**
* Returns all the containers which are in the specified state.
*
* @return List of ContainerInfo
*/
List<ContainerInfo> getContainers(HddsProtos.LifeCycleState state);
/**
* Returns the ContainerInfo from the container ID.
*
@ -45,7 +62,8 @@ public interface ContainerManager extends Closeable {
* @return - ContainerInfo such as creation state and the pipeline.
* @throws IOException
*/
ContainerInfo getContainer(long containerID) throws IOException;
ContainerInfo getContainer(ContainerID containerID)
throws ContainerNotFoundException;
/**
* Returns the ContainerInfo from the container ID.
@ -54,8 +72,8 @@ public interface ContainerManager extends Closeable {
* @return - ContainerWithPipeline such as creation state and the pipeline.
* @throws IOException
*/
ContainerWithPipeline getContainerWithPipeline(long containerID)
throws IOException;
ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException;
/**
* Returns containers under certain conditions.
@ -72,8 +90,7 @@ public interface ContainerManager extends Closeable {
* @return a list of container.
* @throws IOException
*/
List<ContainerInfo> listContainer(long startContainerID, int count)
throws IOException;
List<ContainerInfo> listContainer(ContainerID startContainerID, int count);
/**
* Allocates a new container for a given keyName and replication factor.
@ -93,7 +110,7 @@ public interface ContainerManager extends Closeable {
* @param containerID - Container ID
* @throws IOException
*/
void deleteContainer(long containerID) throws IOException;
void deleteContainer(ContainerID containerID) throws IOException;
/**
* Update container state.
@ -102,23 +119,44 @@ public interface ContainerManager extends Closeable {
* @return - new container state
* @throws IOException
*/
HddsProtos.LifeCycleState updateContainerState(long containerID,
HddsProtos.LifeCycleState updateContainerState(ContainerID containerID,
HddsProtos.LifeCycleEvent event) throws IOException;
/**
* Returns the container State Manager.
* @return ContainerStateManager
*/
ContainerStateManager getStateManager();
/**
* Process container report from Datanode.
*
* @param reports Container report
*/
void processContainerReports(DatanodeDetails datanodeDetails,
ContainerReportsProto reports, boolean isRegisterCall)
throws IOException;
ContainerReportsProto reports) throws IOException;
/**
* Returns the latest list of replicas for given containerId.
*
* @param containerID Container ID
* @return Set of ContainerReplica
*/
Set<ContainerReplica> getContainerReplicas(ContainerID containerID)
throws ContainerNotFoundException;
/**
* Adds a container Replica for the given Container.
*
* @param containerID Container ID
* @param replica ContainerReplica
*/
void updateContainerReplica(ContainerID containerID, ContainerReplica replica)
throws ContainerNotFoundException;
/**
* Remove a container Replica form a given Container.
*
* @param containerID Container ID
* @param replica ContainerReplica
* @return True of dataNode is removed successfully else false.
*/
void removeContainerReplica(ContainerID containerID, ContainerReplica replica)
throws ContainerNotFoundException, ContainerReplicaNotFoundException;
/**
* Update deleteTransactionId according to deleteTransactionMap.

View File

@ -0,0 +1,197 @@
/**
* 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.hdds.scm.container;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.builder.CompareToBuilder;
import org.apache.commons.lang3.builder.EqualsBuilder;
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import java.util.Optional;
import java.util.UUID;
/**
* In-memory state of a container replica.
*/
public final class ContainerReplica implements Comparable<ContainerReplica> {
final private ContainerID containerID;
final private DatanodeDetails datanodeDetails;
final private UUID placeOfBirth;
private Long sequenceId;
private ContainerReplica(ContainerID containerID, DatanodeDetails datanode,
UUID originNodeId) {
this.containerID = containerID;
this.datanodeDetails = datanode;
this.placeOfBirth = originNodeId;
}
private void setSequenceId(Long seqId) {
sequenceId = seqId;
}
/**
* Returns the DatanodeDetails to which this replica belongs.
*
* @return DatanodeDetails
*/
public DatanodeDetails getDatanodeDetails() {
return datanodeDetails;
}
/**
* Returns the UUID of Datanode where this replica originated.
*
* @return UUID
*/
public UUID getOriginDatanodeId() {
return placeOfBirth;
}
/**
* Returns the Sequence Id of this replica.
*
* @return Sequence Id
*/
public Long getSequenceId() {
return sequenceId;
}
@Override
public int hashCode() {
return new HashCodeBuilder(61, 71)
.append(containerID)
.append(datanodeDetails)
.toHashCode();
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final ContainerReplica that = (ContainerReplica) o;
return new EqualsBuilder()
.append(containerID, that.containerID)
.append(datanodeDetails, that.datanodeDetails)
.isEquals();
}
@Override
public int compareTo(ContainerReplica that) {
Preconditions.checkNotNull(that);
return new CompareToBuilder()
.append(this.containerID, that.containerID)
.append(this.datanodeDetails, that.datanodeDetails)
.build();
}
/**
* Returns a new Builder to construct ContainerReplica.
*
* @return ContainerReplicaBuilder
*/
public static ContainerReplicaBuilder newBuilder() {
return new ContainerReplicaBuilder();
}
/**
* Used for building ContainerReplica instance.
*/
public static class ContainerReplicaBuilder {
private ContainerID containerID;
private DatanodeDetails datanode;
private UUID placeOfBirth;
private Long sequenceId;
/**
* Set Container Id.
*
* @param containerId ContainerID
* @return ContainerReplicaBuilder
*/
public ContainerReplicaBuilder setContainerID(
final ContainerID containerId) {
containerID = containerId;
return this;
}
/**
* Set DatanodeDetails.
*
* @param datanodeDetails DatanodeDetails
* @return ContainerReplicaBuilder
*/
public ContainerReplicaBuilder setDatanodeDetails(
DatanodeDetails datanodeDetails) {
datanode = datanodeDetails;
return this;
}
/**
* Set replica origin node id.
*
* @param originNodeId origin node UUID
* @return ContainerReplicaBuilder
*/
public ContainerReplicaBuilder setOriginNodeId(UUID originNodeId) {
placeOfBirth = originNodeId;
return this;
}
/**
* Set sequence Id of the replica.
*
* @param seqId container sequence Id
* @return ContainerReplicaBuilder
*/
public ContainerReplicaBuilder setSequenceId(long seqId) {
sequenceId = seqId;
return this;
}
/**
* Constructs new ContainerReplicaBuilder.
*
* @return ContainerReplicaBuilder
*/
public ContainerReplica build() {
Preconditions.checkNotNull(containerID,
"Container Id can't be null");
Preconditions.checkNotNull(datanode,
"DatanodeDetails can't be null");
ContainerReplica replica = new ContainerReplica(containerID, datanode,
Optional.ofNullable(placeOfBirth).orElse(datanode.getUuid()));
Optional.ofNullable(sequenceId).ifPresent(replica::setSequenceId);
return replica;
}
}
}

View File

@ -24,11 +24,9 @@ import java.util.stream.Collectors;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationActivityStatus;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.node.states.ReportResult;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.ContainerReportFromDatanode;
@ -49,11 +47,7 @@ public class ContainerReportHandler implements
LoggerFactory.getLogger(ContainerReportHandler.class);
private final NodeManager nodeManager;
private final ContainerManager containerManager;
private ContainerStateManager containerStateManager;
private ReplicationActivityStatus replicationStatus;
public ContainerReportHandler(ContainerManager containerManager,
@ -62,7 +56,6 @@ public class ContainerReportHandler implements
Preconditions.checkNotNull(containerManager);
Preconditions.checkNotNull(nodeManager);
Preconditions.checkNotNull(replicationActivityStatus);
this.containerStateManager = containerManager.getStateManager();
this.nodeManager = nodeManager;
this.containerManager = containerManager;
this.replicationStatus = replicationActivityStatus;
@ -81,7 +74,7 @@ public class ContainerReportHandler implements
//update state in container db and trigger close container events
containerManager
.processContainerReports(datanodeOrigin, containerReport, false);
.processContainerReports(datanodeOrigin, containerReport);
Set<ContainerID> containerIds = containerReport.getReportsList().stream()
.map(StorageContainerDatanodeProtocolProtos
@ -97,13 +90,21 @@ public class ContainerReportHandler implements
.setContainersForDatanode(datanodeOrigin.getUuid(), containerIds);
for (ContainerID containerID : reportResult.getMissingEntries()) {
containerStateManager
.removeContainerReplica(containerID, datanodeOrigin);
final ContainerReplica replica = ContainerReplica.newBuilder()
.setContainerID(containerID)
.setDatanodeDetails(datanodeOrigin)
.build();
containerManager
.removeContainerReplica(containerID, replica);
checkReplicationState(containerID, publisher);
}
for (ContainerID containerID : reportResult.getNewEntries()) {
containerStateManager.addContainerReplica(containerID, datanodeOrigin);
final ContainerReplica replica = ContainerReplica.newBuilder()
.setContainerID(containerID)
.setDatanodeDetails(datanodeOrigin)
.build();
containerManager.updateContainerReplica(containerID, replica);
checkReplicationState(containerID, publisher);
}
@ -116,35 +117,30 @@ public class ContainerReportHandler implements
}
private void checkReplicationState(ContainerID containerID,
EventPublisher publisher)
throws SCMException {
ContainerInfo container = containerStateManager.getContainer(containerID);
if (container == null) {
//warning unknown container
EventPublisher publisher) {
try {
ContainerInfo container = containerManager.getContainer(containerID);
replicateIfNeeded(container, publisher);
} catch (ContainerNotFoundException ex) {
LOG.warn(
"Container is missing from containerStateManager. Can't request "
+ "replication. {}",
containerID);
return;
}
if (container.isContainerOpen()) {
return;
}
ReplicationRequest replicationState =
containerStateManager.checkReplicationState(containerID);
if (replicationState != null) {
if (replicationStatus.isReplicationEnabled()) {
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
replicationState);
} else {
LOG.warn(
"Over/under replicated container but the replication is not "
+ "(yet) enabled: "
+ replicationState.toString());
}
}
}
private void replicateIfNeeded(ContainerInfo container,
EventPublisher publisher) throws ContainerNotFoundException {
if (!container.isOpen() && replicationStatus.isReplicationEnabled()) {
final int existingReplicas = containerManager
.getContainerReplicas(container.containerID()).size();
final int expectedReplicas = container.getReplicationFactor().getNumber();
if (existingReplicas != expectedReplicas) {
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
new ReplicationRequest(container.getContainerID(),
existingReplicas, expectedReplicas));
}
}
}
}

View File

@ -17,17 +17,12 @@
package org.apache.hadoop.hdds.scm.container;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.container.states.ContainerState;
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@ -45,11 +40,8 @@ import org.apache.hadoop.util.Time;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
@ -116,7 +108,7 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
* TimeOut Delete Container State Machine - if the container creating times out,
* then Container State manager decides to delete the container.
*/
public class ContainerStateManager implements Closeable {
public class ContainerStateManager {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerStateManager.class);
@ -135,11 +127,10 @@ public class ContainerStateManager implements Closeable {
* TODO : Add Container Tags so we know which containers are owned by SCM.
*/
@SuppressWarnings("unchecked")
public ContainerStateManager(Configuration configuration,
ContainerManager containerManager, PipelineSelector pipelineSelector) {
public ContainerStateManager(final Configuration configuration) {
// Initialize the container state machine.
Set<HddsProtos.LifeCycleState> finalStates = new HashSet();
final Set<HddsProtos.LifeCycleState> finalStates = new HashSet();
// These are the steady states of a container.
finalStates.add(LifeCycleState.OPEN);
@ -155,22 +146,9 @@ public class ContainerStateManager implements Closeable {
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
StorageUnit.BYTES);
lastUsedMap = new ConcurrentHashMap<>();
containerCount = new AtomicLong(0);
containers = new ContainerStateMap();
}
/**
* Return the info of all the containers kept by the in-memory mapping.
*
* @return the list of all container info.
*/
public List<ContainerInfo> getAllContainers() {
List<ContainerInfo> list = new ArrayList<>();
//No Locking needed since the return value is an immutable map.
containers.getContainerMap().forEach((key, value) -> list.add(value));
return list;
this.lastUsedMap = new ConcurrentHashMap<>();
this.containerCount = new AtomicLong(0);
this.containers = new ContainerStateMap();
}
/*
@ -244,17 +222,15 @@ public class ContainerStateManager implements Closeable {
LifeCycleEvent.CLEANUP);
}
public void addExistingContainer(ContainerInfo containerInfo)
void loadContainer(final ContainerInfo containerInfo)
throws SCMException {
containers.addContainer(containerInfo);
long containerID = containerInfo.getContainerID();
if (containerCount.get() < containerID) {
containerCount.set(containerID);
}
containerCount.set(Long.max(
containerInfo.getContainerID(), containerCount.get()));
}
/**
* allocates a new container based on the type, replication etc.
* Allocates a new container based on the type, replication etc.
*
* @param selector -- Pipeline selector class.
* @param type -- Replication type.
@ -262,25 +238,22 @@ public class ContainerStateManager implements Closeable {
* @return ContainerWithPipeline
* @throws IOException on Failure.
*/
public ContainerWithPipeline allocateContainer(PipelineSelector selector,
HddsProtos.ReplicationType type,
HddsProtos.ReplicationFactor replicationFactor, String owner)
ContainerInfo allocateContainer(final PipelineSelector selector,
final HddsProtos.ReplicationType type,
final HddsProtos.ReplicationFactor replicationFactor, final String owner)
throws IOException {
Pipeline pipeline = selector.getReplicationPipeline(type,
final Pipeline pipeline = selector.getReplicationPipeline(type,
replicationFactor);
Preconditions.checkNotNull(pipeline, "Pipeline type=%s/"
+ "replication=%s couldn't be found for the new container. "
+ "Do you have enough nodes?", type, replicationFactor);
long containerID = containerCount.incrementAndGet();
ContainerInfo containerInfo = new ContainerInfo.Builder()
final long containerID = containerCount.incrementAndGet();
final ContainerInfo containerInfo = new ContainerInfo.Builder()
.setState(HddsProtos.LifeCycleState.ALLOCATED)
.setPipelineID(pipeline.getId())
// This is bytes allocated for blocks inside container, not the
// container size
.setAllocatedBytes(0)
.setUsedBytes(0)
.setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
@ -294,35 +267,34 @@ public class ContainerStateManager implements Closeable {
Preconditions.checkNotNull(containerInfo);
containers.addContainer(containerInfo);
LOG.trace("New container allocated: {}", containerInfo);
return new ContainerWithPipeline(containerInfo, pipeline);
return containerInfo;
}
/**
* Update the Container State to the next state.
*
* @param info - ContainerInfo
* @param containerID - ContainerID
* @param event - LifeCycle Event
* @return Updated ContainerInfo.
* @throws SCMException on Failure.
*/
public ContainerInfo updateContainerState(ContainerInfo
info, HddsProtos.LifeCycleEvent event) throws SCMException {
LifeCycleState newState;
ContainerInfo updateContainerState(final ContainerID containerID,
final HddsProtos.LifeCycleEvent event)
throws SCMException, ContainerNotFoundException {
final ContainerInfo info = containers.getContainerInfo(containerID);
try {
newState = this.stateMachine.getNextState(info.getState(), event);
final LifeCycleState newState = stateMachine.getNextState(
info.getState(), event);
containers.updateState(containerID, info.getState(), newState);
return containers.getContainerInfo(containerID);
} catch (InvalidStateTransitionException ex) {
String error = String.format("Failed to update container state %s, " +
"reason: invalid state transition from state: %s upon " +
"event: %s.",
info.getContainerID(), info.getState(), event);
containerID, info.getState(), event);
LOG.error(error);
throw new SCMException(error, FAILED_TO_CHANGE_CONTAINER_STATE);
}
// This is a post condition after executing getNextState.
Preconditions.checkNotNull(newState);
containers.updateState(info, info.getState(), newState);
return containers.getContainerInfo(info);
}
/**
@ -331,10 +303,10 @@ public class ContainerStateManager implements Closeable {
* @return ContainerInfo
* @throws SCMException - on Error.
*/
public ContainerInfo updateContainerInfo(ContainerInfo info)
throws SCMException {
ContainerInfo updateContainerInfo(final ContainerInfo info)
throws ContainerNotFoundException {
containers.updateContainerInfo(info);
return containers.getContainerInfo(info);
return containers.getContainerInfo(info.containerID());
}
/**
@ -343,11 +315,16 @@ public class ContainerStateManager implements Closeable {
* @param deleteTransactionMap maps containerId to its new
* deleteTransactionID
*/
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap) {
for (Map.Entry<Long, Long> entry : deleteTransactionMap.entrySet()) {
containers.getContainerMap().get(ContainerID.valueof(entry.getKey()))
.updateDeleteTransactionId(entry.getValue());
}
void updateDeleteTransactionId(
final Map<Long, Long> deleteTransactionMap) {
deleteTransactionMap.forEach((k, v) -> {
try {
containers.getContainerInfo(ContainerID.valueof(k))
.updateDeleteTransactionId(v);
} catch (ContainerNotFoundException e) {
LOG.warn("Exception while updating delete transaction id.", e);
}
});
}
/**
@ -360,12 +337,12 @@ public class ContainerStateManager implements Closeable {
* @param state - State of the Container-- {Open, Allocated etc.}
* @return ContainerInfo, null if there is no match found.
*/
public ContainerInfo getMatchingContainer(final long size,
ContainerInfo getMatchingContainer(final long size,
String owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) {
// Find containers that match the query spec, if no match return null.
NavigableSet<ContainerID> matchingSet =
final NavigableSet<ContainerID> matchingSet =
containers.getMatchingContainerIDs(state, owner, factor, type);
if (matchingSet == null || matchingSet.size() == 0) {
return null;
@ -373,11 +350,9 @@ public class ContainerStateManager implements Closeable {
// Get the last used container and find container above the last used
// container ID.
ContainerState key = new ContainerState(owner, type, factor);
ContainerID lastID = lastUsedMap.get(key);
if (lastID == null) {
lastID = matchingSet.first();
}
final ContainerState key = new ContainerState(owner, type, factor);
final ContainerID lastID = lastUsedMap
.getOrDefault(key, matchingSet.first());
// There is a small issue here. The first time, we will skip the first
// container. But in most cases it will not matter.
@ -401,32 +376,47 @@ public class ContainerStateManager implements Closeable {
resultSet = matchingSet.headSet(lastID, true);
selectedContainer = findContainerWithSpace(size, resultSet, owner);
}
// Update the allocated Bytes on this container.
if (selectedContainer != null) {
selectedContainer.updateAllocatedBytes(size);
}
return selectedContainer;
}
private ContainerInfo findContainerWithSpace(long size,
NavigableSet<ContainerID> searchSet, String owner) {
// Get the container with space to meet our request.
for (ContainerID id : searchSet) {
ContainerInfo containerInfo = containers.getContainerInfo(id);
if (containerInfo.getAllocatedBytes() + size <= this.containerSize) {
containerInfo.updateLastUsedTime();
private ContainerInfo findContainerWithSpace(final long size,
final NavigableSet<ContainerID> searchSet, final String owner) {
try {
// Get the container with space to meet our request.
for (ContainerID id : searchSet) {
final ContainerInfo containerInfo = containers.getContainerInfo(id);
if (containerInfo.getUsedBytes() + size <= this.containerSize) {
containerInfo.updateLastUsedTime();
ContainerState key = new ContainerState(owner,
containerInfo.getReplicationType(),
containerInfo.getReplicationFactor());
lastUsedMap.put(key, containerInfo.containerID());
return containerInfo;
final ContainerState key = new ContainerState(owner,
containerInfo.getReplicationType(),
containerInfo.getReplicationFactor());
lastUsedMap.put(key, containerInfo.containerID());
return containerInfo;
}
}
} catch (ContainerNotFoundException e) {
// This should not happen!
LOG.warn("Exception while finding container with space", e);
}
return null;
}
Set<ContainerID> getAllContainerIDs() {
return containers.getAllContainerIDs();
}
/**
* Returns Containers by State.
*
* @param state - State - Open, Closed etc.
* @return List of containers by state.
*/
Set<ContainerID> getContainerIDsByState(final LifeCycleState state) {
return containers.getContainerIDsByState(state);
}
/**
* Returns a set of ContainerIDs that match the Container.
*
@ -436,39 +426,25 @@ public class ContainerStateManager implements Closeable {
* @param state - Current State, like Open, Close etc.
* @return Set of containers that match the specific query parameters.
*/
public NavigableSet<ContainerID> getMatchingContainerIDs(
String owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) {
NavigableSet<ContainerID> getMatchingContainerIDs(final String owner,
final ReplicationType type, final ReplicationFactor factor,
final LifeCycleState state) {
return containers.getMatchingContainerIDs(state, owner,
factor, type);
}
/**
* Returns the containerInfo with pipeline for the given container id.
* @param selector -- Pipeline selector class.
* @param containerID id of the container
* @return ContainerInfo containerInfo
* @throws IOException
*/
public ContainerWithPipeline getContainer(PipelineSelector selector,
ContainerID containerID) {
ContainerInfo info = containers.getContainerInfo(containerID.getId());
Pipeline pipeline = selector.getPipeline(info.getPipelineID());
return new ContainerWithPipeline(info, pipeline);
}
/**
* Returns the containerInfo for the given container id.
* @param containerID id of the container
* @return ContainerInfo containerInfo
* @throws IOException
*/
public ContainerInfo getContainer(ContainerID containerID) {
ContainerInfo getContainer(final ContainerID containerID)
throws ContainerNotFoundException {
return containers.getContainerInfo(containerID);
}
@Override
public void close() throws IOException {
void close() throws IOException {
}
/**
@ -478,8 +454,8 @@ public class ContainerStateManager implements Closeable {
* @param containerID
* @return Set<DatanodeDetails>
*/
public Set<DatanodeDetails> getContainerReplicas(ContainerID containerID)
throws SCMException {
Set<ContainerReplica> getContainerReplicas(
final ContainerID containerID) throws ContainerNotFoundException {
return containers.getContainerReplicas(containerID);
}
@ -487,53 +463,29 @@ public class ContainerStateManager implements Closeable {
* Add a container Replica for given DataNode.
*
* @param containerID
* @param dn
* @param replica
*/
public void addContainerReplica(ContainerID containerID, DatanodeDetails dn) {
containers.addContainerReplica(containerID, dn);
void updateContainerReplica(final ContainerID containerID,
final ContainerReplica replica) throws ContainerNotFoundException {
containers.updateContainerReplica(containerID, replica);
}
/**
* Remove a container Replica for given DataNode.
*
* @param containerID
* @param dn
* @param replica
* @return True of dataNode is removed successfully else false.
*/
public boolean removeContainerReplica(ContainerID containerID,
DatanodeDetails dn) throws SCMException {
return containers.removeContainerReplica(containerID, dn);
void removeContainerReplica(final ContainerID containerID,
final ContainerReplica replica)
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
containers.removeContainerReplica(containerID, replica);
}
/**
* Compare the existing replication number with the expected one.
*/
public ReplicationRequest checkReplicationState(ContainerID containerID)
throws SCMException {
int existingReplicas = getContainerReplicas(containerID).size();
int expectedReplicas = getContainer(containerID)
.getReplicationFactor().getNumber();
if (existingReplicas != expectedReplicas) {
return new ReplicationRequest(containerID.getId(), existingReplicas,
expectedReplicas);
}
return null;
}
/**
* Checks if the container is open.
*/
public boolean isOpen(ContainerID containerID) {
Preconditions.checkNotNull(containerID);
ContainerInfo container = Preconditions
.checkNotNull(getContainer(containerID),
"Container can't be found " + containerID);
return container.isContainerOpen();
}
@VisibleForTesting
public ContainerStateMap getContainerStateMap() {
return containers;
void removeContainer(final ContainerID containerID)
throws ContainerNotFoundException {
containers.removeContainer(containerID);
}
}

View File

@ -24,12 +24,10 @@ import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.SCMContainerInfo;
import org.apache.hadoop.hdds.scm.block.PendingDeleteStatusList;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
@ -55,19 +53,23 @@ import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.nio.charset.Charset;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
.OZONE_SCM_CONTAINER_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
.OZONE_SCM_CONTAINER_SIZE;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_CHANGE_CONTAINER_STATE;
import static org.apache.hadoop.hdds.server.ServerUtils.getOzoneMetaDirPath;
@ -82,10 +84,7 @@ public class SCMContainerManager implements ContainerManager {
private static final Logger LOG = LoggerFactory.getLogger(SCMContainerManager
.class);
private final NodeManager nodeManager;
private final long cacheSize;
private final Lock lock;
private final Charset encoding = Charset.forName("UTF-8");
private final MetadataStore containerStore;
private final PipelineSelector pipelineSelector;
private final ContainerStateManager containerStateManager;
@ -100,83 +99,97 @@ public class SCMContainerManager implements ContainerManager {
* @param nodeManager - NodeManager so that we can get the nodes that are
* healthy to place new
* containers.
* @param cacheSizeMB - Amount of memory reserved for the LSM tree to cache
* its nodes. This is
* passed to LevelDB and this memory is allocated in Native code space.
* CacheSize is specified
* in MB.
* @throws IOException on Failure.
*/
@SuppressWarnings("unchecked")
public SCMContainerManager(
final Configuration conf, final NodeManager nodeManager, final int
cacheSizeMB, EventPublisher eventPublisher) throws IOException {
this.nodeManager = nodeManager;
this.cacheSize = cacheSizeMB;
public SCMContainerManager(final Configuration conf,
final NodeManager nodeManager, final EventPublisher eventPublisher)
throws IOException {
File metaDir = getOzoneMetaDirPath(conf);
final File metaDir = getOzoneMetaDirPath(conf);
final File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
final int cacheSize = conf.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
// Write the container name to pipeline mapping.
File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
containerStore =
MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(containerDBPath)
.setCacheSize(this.cacheSize * OzoneConsts.MB)
.build();
this.containerStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(containerDBPath)
.setCacheSize(cacheSize * OzoneConsts.MB)
.build();
this.lock = new ReentrantLock();
size = (long)conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
this.size = (long) conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
this.pipelineSelector = new PipelineSelector(nodeManager,
conf, eventPublisher, cacheSizeMB);
this.containerStateManager =
new ContainerStateManager(conf, this, pipelineSelector);
LOG.trace("Container State Manager created.");
conf, eventPublisher, cacheSize);
this.containerStateManager = new ContainerStateManager(conf);
this.eventPublisher = eventPublisher;
long containerCreationLeaseTimeout = conf.getTimeDuration(
final long containerCreationLeaseTimeout = conf.getTimeDuration(
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT,
ScmConfigKeys.OZONE_SCM_CONTAINER_CREATION_LEASE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
containerLeaseManager = new LeaseManager<>("ContainerCreation",
this.containerLeaseManager = new LeaseManager<>("ContainerCreation",
containerCreationLeaseTimeout);
containerLeaseManager.start();
this.containerLeaseManager.start();
loadExistingContainers();
}
private void loadExistingContainers() {
List<ContainerInfo> containerList;
try {
containerList = listContainer(0, Integer.MAX_VALUE);
// if there are no container to load, let us return.
if (containerList == null || containerList.size() == 0) {
LOG.info("No containers to load for this cluster.");
return;
}
} catch (IOException e) {
if (!e.getMessage().equals("No container exists in current db")) {
LOG.error("Could not list the containers", e);
}
return;
private void loadExistingContainers() throws IOException {
List<Map.Entry<byte[], byte[]>> range = containerStore
.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
for (Map.Entry<byte[], byte[]> entry : range) {
ContainerInfo container = ContainerInfo.fromProtobuf(
HddsProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
Preconditions.checkNotNull(container);
containerStateManager.loadContainer(container);
pipelineSelector.addContainerToPipeline(
container.getPipelineID(), container.getContainerID());
}
}
@VisibleForTesting
// TODO: remove this later.
public ContainerStateManager getContainerStateManager() {
return containerStateManager;
}
@Override
public List<ContainerInfo> getContainers() {
lock.lock();
try {
for (ContainerInfo container : containerList) {
containerStateManager.addExistingContainer(container);
pipelineSelector.addContainerToPipeline(
container.getPipelineID(), container.getContainerID());
}
} catch (SCMException ex) {
LOG.error("Unable to create a container information. ", ex);
// Fix me, what is the proper shutdown procedure for SCM ??
// System.exit(1) // Should we exit here?
return containerStateManager.getAllContainerIDs().stream().map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException e) {
// How can this happen?
return null;
}
}).filter(Objects::nonNull).collect(Collectors.toList());
} finally {
lock.unlock();
}
}
@Override
public List<ContainerInfo> getContainers(LifeCycleState state) {
lock.lock();
try {
return containerStateManager.getContainerIDsByState(state).stream()
.map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException e) {
// How can this happen?
return null;
}
}).filter(Objects::nonNull).collect(Collectors.toList());
} finally {
lock.unlock();
}
}
@ -184,26 +197,9 @@ public class SCMContainerManager implements ContainerManager {
* {@inheritDoc}
*/
@Override
public ContainerInfo getContainer(final long containerID) throws
IOException {
ContainerInfo containerInfo;
lock.lock();
try {
byte[] containerBytes = containerStore.get(
Longs.toByteArray(containerID));
if (containerBytes == null) {
throw new SCMException(
"Specified key does not exist. key : " + containerID,
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER
.parseFrom(containerBytes);
containerInfo = ContainerInfo.fromProtobuf(temp);
return containerInfo;
} finally {
lock.unlock();
}
public ContainerInfo getContainer(final ContainerID containerID)
throws ContainerNotFoundException {
return containerStateManager.getContainer(containerID);
}
/**
@ -217,38 +213,29 @@ public class SCMContainerManager implements ContainerManager {
* @throws IOException
*/
@Override
public ContainerWithPipeline getContainerWithPipeline(long containerID)
throws IOException {
ContainerInfo contInfo;
public ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException {
lock.lock();
try {
byte[] containerBytes = containerStore.get(
Longs.toByteArray(containerID));
if (containerBytes == null) {
throw new SCMException(
"Specified key does not exist. key : " + containerID,
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
}
HddsProtos.SCMContainerInfo temp = HddsProtos.SCMContainerInfo.PARSER
.parseFrom(containerBytes);
contInfo = ContainerInfo.fromProtobuf(temp);
final ContainerInfo contInfo = getContainer(containerID);
Pipeline pipeline;
String leaderId = "";
if (contInfo.isContainerOpen()) {
if (contInfo.isOpen()) {
// If pipeline with given pipeline Id already exist return it
pipeline = pipelineSelector.getPipeline(contInfo.getPipelineID());
} else {
// For close containers create pipeline from datanodes with replicas
Set<DatanodeDetails> dnWithReplicas = containerStateManager
Set<ContainerReplica> dnWithReplicas = containerStateManager
.getContainerReplicas(contInfo.containerID());
if (!dnWithReplicas.isEmpty()) {
leaderId = dnWithReplicas.iterator().next().getUuidString();
leaderId = dnWithReplicas.iterator().next()
.getDatanodeDetails().getUuidString();
}
pipeline = new Pipeline(leaderId, contInfo.getState(),
ReplicationType.STAND_ALONE, contInfo.getReplicationFactor(),
PipelineID.randomId());
dnWithReplicas.forEach(pipeline::addMember);
dnWithReplicas.stream().map(ContainerReplica::getDatanodeDetails).
forEach(pipeline::addMember);
}
return new ContainerWithPipeline(contInfo, pipeline);
} finally {
@ -260,33 +247,32 @@ public class SCMContainerManager implements ContainerManager {
* {@inheritDoc}
*/
@Override
public List<ContainerInfo> listContainer(long startContainerID,
int count) throws IOException {
List<ContainerInfo> containerList = new ArrayList<>();
public List<ContainerInfo> listContainer(ContainerID startContainerID,
int count) {
lock.lock();
try {
if (containerStore.isEmpty()) {
throw new IOException("No container exists in current db");
}
byte[] startKey = startContainerID <= 0 ? null :
Longs.toByteArray(startContainerID);
List<Map.Entry<byte[], byte[]>> range =
containerStore.getSequentialRangeKVs(startKey, count, null);
final long startId = startContainerID == null ?
0 : startContainerID.getId();
final List<ContainerID> containersIds =
new ArrayList<>(containerStateManager.getAllContainerIDs());
Collections.sort(containersIds);
// Transform the values into the pipelines.
// TODO: filter by container state
for (Map.Entry<byte[], byte[]> entry : range) {
ContainerInfo containerInfo =
ContainerInfo.fromProtobuf(
HddsProtos.SCMContainerInfo.PARSER.parseFrom(
entry.getValue()));
Preconditions.checkNotNull(containerInfo);
containerList.add(containerInfo);
}
return containersIds.stream()
.filter(id -> id.getId() > startId)
.limit(count)
.map(id -> {
try {
return containerStateManager.getContainer(id);
} catch (ContainerNotFoundException ex) {
// This can never happen, as we hold lock no one else can remove
// the container after we got the container ids.
LOG.warn("Container Missing.", ex);
return null;
}
}).collect(Collectors.toList());
} finally {
lock.unlock();
}
return containerList;
}
/**
@ -298,29 +284,35 @@ public class SCMContainerManager implements ContainerManager {
* @throws IOException - Exception
*/
@Override
public ContainerWithPipeline allocateContainer(
ReplicationType type,
ReplicationFactor replicationFactor,
String owner)
public ContainerWithPipeline allocateContainer(final ReplicationType type,
final ReplicationFactor replicationFactor, final String owner)
throws IOException {
ContainerInfo containerInfo;
ContainerWithPipeline containerWithPipeline;
lock.lock();
try {
containerWithPipeline = containerStateManager.allocateContainer(
pipelineSelector, type, replicationFactor, owner);
containerInfo = containerWithPipeline.getContainerInfo();
final ContainerInfo containerInfo; containerInfo = containerStateManager
.allocateContainer(pipelineSelector, type, replicationFactor, owner);
final Pipeline pipeline = pipelineSelector.getPipeline(
containerInfo.getPipelineID());
byte[] containerIDBytes = Longs.toByteArray(
containerInfo.getContainerID());
containerStore.put(containerIDBytes, containerInfo.getProtobuf()
.toByteArray());
try {
final byte[] containerIDBytes = Longs.toByteArray(
containerInfo.getContainerID());
containerStore.put(containerIDBytes,
containerInfo.getProtobuf().toByteArray());
} catch (IOException ex) {
// If adding to containerStore fails, we should remove the container
// from in-memory map.
try {
containerStateManager.removeContainer(containerInfo.containerID());
} catch (ContainerNotFoundException cnfe) {
// No need to worry much, everything is going as planned.
}
throw ex;
}
return new ContainerWithPipeline(containerInfo, pipeline);
} finally {
lock.unlock();
}
return containerWithPipeline;
}
/**
@ -332,18 +324,24 @@ public class SCMContainerManager implements ContainerManager {
* specified key.
*/
@Override
public void deleteContainer(long containerID) throws IOException {
public void deleteContainer(ContainerID containerID) throws IOException {
lock.lock();
try {
byte[] dbKey = Longs.toByteArray(containerID);
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException(
"Failed to delete container " + containerID + ", reason : " +
"container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
containerStateManager.removeContainer(containerID);
final byte[] dbKey = Longs.toByteArray(containerID.getId());
final byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes != null) {
containerStore.delete(dbKey);
} else {
// Where did the container go? o_O
LOG.warn("Unable to remove the container {} from container store," +
" it's missing!", containerID);
}
containerStore.delete(dbKey);
} catch (ContainerNotFoundException cnfe) {
throw new SCMException(
"Failed to delete container " + containerID + ", reason : " +
"container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} finally {
lock.unlock();
}
@ -354,40 +352,51 @@ public class SCMContainerManager implements ContainerManager {
*/
@Override
public HddsProtos.LifeCycleState updateContainerState(
long containerID, HddsProtos.LifeCycleEvent event) throws
IOException {
ContainerInfo containerInfo;
ContainerID containerID, HddsProtos.LifeCycleEvent event)
throws IOException {
// Should we return the updated ContainerInfo instead of LifeCycleState?
lock.lock();
try {
byte[] dbKey = Longs.toByteArray(containerID);
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes == null) {
throw new SCMException(
"Failed to update container state"
+ containerID
+ ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
ContainerInfo updatedContainer =
updateContainerStateInternal(containerID, event);
if (!updatedContainer.isOpen()) {
pipelineSelector.removeContainerFromPipeline(
updatedContainer.getPipelineID(), containerID.getId());
}
containerInfo =
ContainerInfo.fromProtobuf(HddsProtos.SCMContainerInfo.PARSER
.parseFrom(containerBytes));
final byte[] dbKey = Longs.toByteArray(containerID.getId());
containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
return updatedContainer.getState();
} catch (ContainerNotFoundException cnfe) {
throw new SCMException(
"Failed to update container state"
+ containerID
+ ", reason : container doesn't exist.",
SCMException.ResultCodes.FAILED_TO_FIND_CONTAINER);
} finally {
lock.unlock();
}
}
Preconditions.checkNotNull(containerInfo);
private ContainerInfo updateContainerStateInternal(ContainerID containerID,
HddsProtos.LifeCycleEvent event) throws IOException {
// Refactor the below code for better clarity.
try {
final ContainerInfo info =
containerStateManager.getContainer(containerID);
switch (event) {
case CREATE:
// Acquire lease on container
Lease<ContainerInfo> containerLease =
containerLeaseManager.acquire(containerInfo);
containerLeaseManager.acquire(info);
// Register callback to be executed in case of timeout
containerLease.registerCallBack(() -> {
updateContainerState(containerID,
HddsProtos.LifeCycleEvent.TIMEOUT);
return null;
});
return null; });
break;
case CREATED:
// Release the lease on container
containerLeaseManager.release(containerInfo);
containerLeaseManager.release(info);
break;
case FINALIZE:
// TODO: we don't need a lease manager here for closing as the
@ -412,28 +421,20 @@ public class SCMContainerManager implements ContainerManager {
// If the below updateContainerState call fails, we should revert the
// changes made in switch case.
// Like releasing the lease in case of BEGIN_CREATE.
ContainerInfo updatedContainer = containerStateManager
.updateContainerState(containerInfo, event);
if (!updatedContainer.isContainerOpen()) {
pipelineSelector.removeContainerFromPipeline(
containerInfo.getPipelineID(), containerID);
}
containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
return updatedContainer.getState();
return containerStateManager.updateContainerState(containerID, event);
} catch (LeaseException e) {
throw new IOException("Lease Exception.", e);
} finally {
lock.unlock();
}
}
/**
* Update deleteTransactionId according to deleteTransactionMap.
*
* @param deleteTransactionMap Maps the containerId to latest delete
* transaction id for the container.
* @throws IOException
*/
/**
* Update deleteTransactionId according to deleteTransactionMap.
*
* @param deleteTransactionMap Maps the containerId to latest delete
* transaction id for the container.
* @throws IOException
*/
public void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
throws IOException {
if (deleteTransactionMap == null) {
@ -466,16 +467,6 @@ public class SCMContainerManager implements ContainerManager {
}
}
/**
* Returns the container State Manager.
*
* @return ContainerStateManager
*/
@Override
public ContainerStateManager getStateManager() {
return containerStateManager;
}
/**
* Return a container matching the attributes specified.
*
@ -489,7 +480,7 @@ public class SCMContainerManager implements ContainerManager {
public ContainerWithPipeline getMatchingContainerWithPipeline(
final long sizeRequired, String owner, ReplicationType type,
ReplicationFactor factor, LifeCycleState state) throws IOException {
ContainerInfo containerInfo = getStateManager()
ContainerInfo containerInfo = containerStateManager
.getMatchingContainer(sizeRequired, owner, type, factor, state);
if (containerInfo == null) {
return null;
@ -518,70 +509,45 @@ public class SCMContainerManager implements ContainerManager {
*/
@Override
public void processContainerReports(DatanodeDetails datanodeDetails,
ContainerReportsProto reports, boolean isRegisterCall)
throws IOException {
ContainerReportsProto reports) throws IOException {
List<StorageContainerDatanodeProtocolProtos.ContainerInfo>
containerInfos = reports.getReportsList();
PendingDeleteStatusList pendingDeleteStatusList =
new PendingDeleteStatusList(datanodeDetails);
for (StorageContainerDatanodeProtocolProtos.ContainerInfo contInfo :
for (StorageContainerDatanodeProtocolProtos.ContainerInfo newInfo :
containerInfos) {
// Update replica info during registration process.
if (isRegisterCall) {
try {
getStateManager().addContainerReplica(ContainerID.
valueof(contInfo.getContainerID()), datanodeDetails);
} catch (Exception ex) {
// Continue to next one after logging the error.
LOG.error("Error while adding replica for containerId {}.",
contInfo.getContainerID(), ex);
}
}
byte[] dbKey = Longs.toByteArray(contInfo.getContainerID());
ContainerID id = ContainerID.valueof(newInfo.getContainerID());
ContainerReplica replica = ContainerReplica.newBuilder()
.setContainerID(id)
.setDatanodeDetails(datanodeDetails)
.setOriginNodeId(datanodeDetails.getUuid())
.build();
lock.lock();
try {
byte[] containerBytes = containerStore.get(dbKey);
if (containerBytes != null) {
HddsProtos.SCMContainerInfo knownState =
HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
if (knownState.getState() == LifeCycleState.CLOSING
&& contInfo.getState() == LifeCycleState.CLOSED) {
updateContainerState(contInfo.getContainerID(),
LifeCycleEvent.CLOSE);
//reread the container
knownState =
HddsProtos.SCMContainerInfo.PARSER
.parseFrom(containerStore.get(dbKey));
}
HddsProtos.SCMContainerInfo newState =
reconcileState(contInfo, knownState, datanodeDetails);
if (knownState.getDeleteTransactionId() > contInfo
.getDeleteTransactionId()) {
pendingDeleteStatusList
.addPendingDeleteStatus(contInfo.getDeleteTransactionId(),
knownState.getDeleteTransactionId(),
knownState.getContainerID());
}
// FIX ME: This can be optimized, we write twice to memory, where a
// single write would work well.
//
// We need to write this to DB again since the closed only write
// the updated State.
containerStore.put(dbKey, newState.toByteArray());
} else {
// Container not found in our container db.
LOG.error("Error while processing container report from datanode :" +
" {}, for container: {}, reason: container doesn't exist in" +
"container database.", datanodeDetails,
contInfo.getContainerID());
containerStateManager.updateContainerReplica(id, replica);
ContainerInfo currentInfo = containerStateManager.getContainer(id);
if (newInfo.getState() == LifeCycleState.CLOSING
&& currentInfo.getState() == LifeCycleState.CLOSED) {
currentInfo = updateContainerStateInternal(id, LifeCycleEvent.CLOSE);
}
HddsProtos.SCMContainerInfo newState =
reconcileState(newInfo, currentInfo);
if (currentInfo.getDeleteTransactionId() >
newInfo.getDeleteTransactionId()) {
pendingDeleteStatusList
.addPendingDeleteStatus(newInfo.getDeleteTransactionId(),
currentInfo.getDeleteTransactionId(),
currentInfo.getContainerID());
}
containerStateManager.updateContainerInfo(
ContainerInfo.fromProtobuf(newState));
containerStore.put(id.getBytes(), newState.toByteArray());
} catch (ContainerNotFoundException e) {
LOG.error("Error while processing container report from datanode :" +
" {}, for container: {}, reason: container doesn't exist in" +
"container database.", datanodeDetails, id);
} finally {
lock.unlock();
}
@ -598,36 +564,21 @@ public class SCMContainerManager implements ContainerManager {
*
* @param datanodeState - State from the Datanode.
* @param knownState - State inside SCM.
* @param dnDetails
* @return new SCM State for this container.
*/
private HddsProtos.SCMContainerInfo reconcileState(
StorageContainerDatanodeProtocolProtos.ContainerInfo datanodeState,
SCMContainerInfo knownState, DatanodeDetails dnDetails) {
ContainerInfo knownState) {
HddsProtos.SCMContainerInfo.Builder builder =
HddsProtos.SCMContainerInfo.newBuilder();
builder.setContainerID(knownState.getContainerID())
.setPipelineID(knownState.getPipelineID())
.setReplicationType(knownState.getReplicationType())
.setReplicationFactor(knownState.getReplicationFactor());
// TODO: If current state doesn't have this DN in list of DataNodes with
// replica then add it in list of replicas.
// If used size is greater than allocated size, we will be updating
// allocated size with used size. This update is done as a fallback
// mechanism in case SCM crashes without properly updating allocated
// size. Correct allocated value will be updated by
// ContainerStateManager during SCM shutdown.
long usedSize = datanodeState.getUsed();
long allocated = knownState.getAllocatedBytes() > usedSize ?
knownState.getAllocatedBytes() : usedSize;
builder.setAllocatedBytes(allocated)
.setUsedBytes(usedSize)
.setNumberOfKeys(datanodeState.getKeyCount())
.setPipelineID(knownState.getPipelineID().getProtobuf())
.setState(knownState.getState())
.setReplicationType(knownState.getReplicationType())
.setReplicationFactor(knownState.getReplicationFactor())
.setUsedBytes(datanodeState.getUsed())
.setNumberOfKeys(datanodeState.getKeyCount())
.setStateEnterTime(knownState.getStateEnterTime())
.setContainerID(knownState.getContainerID())
.setDeleteTransactionId(knownState.getDeleteTransactionId());
if (knownState.getOwner() != null) {
builder.setOwner(knownState.getOwner());
@ -635,20 +586,40 @@ public class SCMContainerManager implements ContainerManager {
return builder.build();
}
/**
* In Container is in closed state, if it is in closed, Deleting or Deleted
* State.
* Returns the latest list of DataNodes where replica for given containerId
* exist. Throws an SCMException if no entry is found for given containerId.
*
* @param info - ContainerInfo.
* @return true if is in open state, false otherwise
* @param containerID
* @return Set<DatanodeDetails>
*/
private boolean shouldClose(ContainerInfo info) {
return info.getState() == HddsProtos.LifeCycleState.OPEN;
public Set<ContainerReplica> getContainerReplicas(
final ContainerID containerID) throws ContainerNotFoundException {
return containerStateManager.getContainerReplicas(containerID);
}
private boolean isClosed(ContainerInfo info) {
return info.getState() == HddsProtos.LifeCycleState.CLOSED;
/**
* Add a container Replica for given DataNode.
*
* @param containerID
* @param replica
*/
public void updateContainerReplica(final ContainerID containerID,
final ContainerReplica replica) throws ContainerNotFoundException {
containerStateManager.updateContainerReplica(containerID, replica);
}
/**
* Remove a container Replica for given DataNode.
*
* @param containerID
* @param replica
* @return True of dataNode is removed successfully else false.
*/
public void removeContainerReplica(final ContainerID containerID,
final ContainerReplica replica)
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
containerStateManager.removeContainerReplica(containerID, replica);
}
/**
@ -671,7 +642,6 @@ public class SCMContainerManager implements ContainerManager {
containerLeaseManager.shutdown();
}
if (containerStateManager != null) {
flushContainerInfo();
containerStateManager.close();
}
if (containerStore != null) {
@ -683,48 +653,6 @@ public class SCMContainerManager implements ContainerManager {
}
}
/**
* Since allocatedBytes of a container is only in memory, stored in
* containerStateManager, when closing SCMContainerManager, we need to update
* this in the container store.
*
* @throws IOException on failure.
*/
@VisibleForTesting
public void flushContainerInfo() throws IOException {
List<ContainerInfo> containers = containerStateManager.getAllContainers();
List<Long> failedContainers = new ArrayList<>();
for (ContainerInfo info : containers) {
// even if some container updated failed, others can still proceed
try {
byte[] dbKey = Longs.toByteArray(info.getContainerID());
byte[] containerBytes = containerStore.get(dbKey);
// TODO : looks like when a container is deleted, the container is
// removed from containerStore but not containerStateManager, so it can
// return info of a deleted container. may revisit this in the future,
// for now, just skip a not-found container
if (containerBytes != null) {
containerStore.put(dbKey, info.getProtobuf().toByteArray());
} else {
LOG.debug("Container state manager has container {} but not found " +
"in container store, a deleted container?",
info.getContainerID());
}
} catch (IOException ioe) {
failedContainers.add(info.getContainerID());
}
}
if (!failedContainers.isEmpty()) {
throw new IOException("Error in flushing container info from container " +
"state manager: " + failedContainers);
}
}
@VisibleForTesting
public MetadataStore getContainerStore() {
return containerStore;
}
public PipelineSelector getPipelineSelector() {
return pipelineSelector;
}

View File

@ -22,12 +22,14 @@ import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.ThreadFactory;
import java.util.stream.Collectors;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.placement.algorithms
.ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
@ -64,14 +66,14 @@ public class ReplicationManager implements Runnable {
private boolean running = true;
private ContainerStateManager containerStateManager;
private ContainerManager containerManager;
public ReplicationManager(ContainerPlacementPolicy containerPlacement,
ContainerStateManager containerStateManager, EventQueue eventQueue,
ContainerManager containerManager, EventQueue eventQueue,
LeaseManager<Long> commandWatcherLeaseManager) {
this.containerPlacement = containerPlacement;
this.containerStateManager = containerStateManager;
this.containerManager = containerManager;
this.eventPublisher = eventQueue;
this.replicationCommandWatcher =
@ -106,7 +108,7 @@ public class ReplicationManager implements Runnable {
ContainerID containerID = new ContainerID(request.getContainerId());
ContainerInfo containerInfo =
containerStateManager.getContainer(containerID);
containerManager.getContainer(containerID);
Preconditions.checkNotNull(containerInfo,
"No information about the container " + request.getContainerId());
@ -116,10 +118,10 @@ public class ReplicationManager implements Runnable {
"Container should be in closed state");
//check the current replication
List<DatanodeDetails> datanodesWithReplicas =
List<ContainerReplica> containerReplicas =
new ArrayList<>(getCurrentReplicas(request));
if (datanodesWithReplicas.size() == 0) {
if (containerReplicas.size() == 0) {
LOG.warn(
"Container {} should be replicated but can't find any existing "
+ "replicas",
@ -134,21 +136,23 @@ public class ReplicationManager implements Runnable {
.size();
int deficit =
request.getExpecReplicationCount() - datanodesWithReplicas.size()
request.getExpecReplicationCount() - containerReplicas.size()
- inFlightReplications;
if (deficit > 0) {
List<DatanodeDetails> datanodes = containerReplicas.stream()
.map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList());
List<DatanodeDetails> selectedDatanodes = containerPlacement
.chooseDatanodes(datanodesWithReplicas, deficit,
.chooseDatanodes(datanodes, deficit,
containerInfo.getUsedBytes());
//send the command
for (DatanodeDetails datanode : selectedDatanodes) {
ReplicateContainerCommand replicateCommand =
new ReplicateContainerCommand(containerID.getId(),
datanodesWithReplicas);
new ReplicateContainerCommand(containerID.getId(), datanodes);
eventPublisher.fireEvent(SCMEvents.DATANODE_COMMAND,
new CommandForDatanode<>(
@ -174,9 +178,9 @@ public class ReplicationManager implements Runnable {
}
@VisibleForTesting
protected Set<DatanodeDetails> getCurrentReplicas(ReplicationRequest request)
protected Set<ContainerReplica> getCurrentReplicas(ReplicationRequest request)
throws IOException {
return containerStateManager
return containerManager
.getContainerReplicas(new ContainerID(request.getContainerId()));
}
@ -234,7 +238,11 @@ public class ReplicationManager implements Runnable {
}
}
public static class ReplicationCompleted implements IdentifiableEventPayload {
/**
* Add javadoc.
*/
public static class ReplicationCompleted
implements IdentifiableEventPayload {
private final long uuid;

View File

@ -20,19 +20,21 @@ package org.apache.hadoop.hdds.scm.container.states;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.util.HashSet;
import java.util.Set;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerReplicaNotFoundException;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.util.HashSet;
import java.util.Set;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
@ -46,8 +48,6 @@ import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.CONTAINER_EXISTS;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_CHANGE_CONTAINER_STATE;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_FIND_CONTAINER;
/**
* Container State Map acts like a unified map for various attributes that are
@ -84,16 +84,15 @@ public class ContainerStateMap {
private static final Logger LOG =
LoggerFactory.getLogger(ContainerStateMap.class);
private final static NavigableSet<ContainerID> EMPTY_SET =
Collections.unmodifiableNavigableSet(new TreeSet<>());
private final ContainerAttribute<LifeCycleState> lifeCycleStateMap;
private final ContainerAttribute<String> ownerMap;
private final ContainerAttribute<ReplicationFactor> factorMap;
private final ContainerAttribute<ReplicationType> typeMap;
private final Map<ContainerID, ContainerInfo> containerMap;
// Map to hold replicas of given container.
private final Map<ContainerID, Set<DatanodeDetails>> contReplicaMap;
private final static NavigableSet<ContainerID> EMPTY_SET =
Collections.unmodifiableNavigableSet(new TreeSet<>());
private final Map<ContainerID, Set<ContainerReplica>> replicaMap;
private final Map<ContainerQueryKey, NavigableSet<ContainerID>> resultCache;
// Container State Map lock should be held before calling into
@ -105,18 +104,14 @@ public class ContainerStateMap {
* Create a ContainerStateMap.
*/
public ContainerStateMap() {
lifeCycleStateMap = new ContainerAttribute<>();
ownerMap = new ContainerAttribute<>();
factorMap = new ContainerAttribute<>();
typeMap = new ContainerAttribute<>();
containerMap = new HashMap<>();
lock = new ReentrantReadWriteLock();
contReplicaMap = new HashMap<>();
// new InstrumentedLock(getClass().getName(), LOG,
// new ReentrantLock(),
// 1000,
// 300));
resultCache = new ConcurrentHashMap<>();
this.lifeCycleStateMap = new ContainerAttribute<>();
this.ownerMap = new ContainerAttribute<>();
this.factorMap = new ContainerAttribute<>();
this.typeMap = new ContainerAttribute<>();
this.containerMap = new HashMap<>();
this.lock = new ReentrantReadWriteLock();
this.replicaMap = new HashMap<>();
this.resultCache = new ConcurrentHashMap<>();
}
/**
@ -125,7 +120,7 @@ public class ContainerStateMap {
* @param info - container info
* @throws SCMException - throws if create failed.
*/
public void addContainer(ContainerInfo info)
public void addContainer(final ContainerInfo info)
throws SCMException {
Preconditions.checkNotNull(info, "Container Info cannot be null");
Preconditions.checkArgument(info.getReplicationFactor().getNumber() > 0,
@ -133,7 +128,7 @@ public class ContainerStateMap {
lock.writeLock().lock();
try {
ContainerID id = ContainerID.valueof(info.getContainerID());
final ContainerID id = info.containerID();
if (containerMap.putIfAbsent(id, info) != null) {
LOG.debug("Duplicate container ID detected. {}", id);
throw new
@ -145,6 +140,7 @@ public class ContainerStateMap {
ownerMap.insert(info.getOwner(), id);
factorMap.insert(info.getReplicationFactor(), id);
typeMap.insert(info.getReplicationType(), id);
replicaMap.put(id, new HashSet<>());
// Flush the cache of this container type, will be added later when
// get container queries are executed.
@ -156,23 +152,30 @@ public class ContainerStateMap {
}
/**
* Returns the latest state of Container from SCM's Container State Map.
* Removes a Container Entry from ContainerStateMap.
*
* @param info - ContainerInfo
* @return ContainerInfo
* @param containerID - ContainerID
* @throws SCMException - throws if create failed.
*/
public ContainerInfo getContainerInfo(ContainerInfo info) {
return getContainerInfo(info.getContainerID());
}
/**
* Returns the latest state of Container from SCM's Container State Map.
*
* @param containerID - int
* @return container info, if found.
*/
public ContainerInfo getContainerInfo(long containerID) {
return getContainerInfo(ContainerID.valueof(containerID));
public void removeContainer(final ContainerID containerID)
throws ContainerNotFoundException {
Preconditions.checkNotNull(containerID, "ContainerID cannot be null");
lock.writeLock().lock();
try {
checkIfContainerExist(containerID);
// Should we revert back to the original state if any of the below
// remove operation fails?
final ContainerInfo info = containerMap.remove(containerID);
lifeCycleStateMap.remove(info.getState(), containerID);
ownerMap.remove(info.getOwner(), containerID);
factorMap.remove(info.getReplicationFactor(), containerID);
typeMap.remove(info.getReplicationType(), containerID);
// Flush the cache of this container type.
flushCache(info);
LOG.trace("Removed container with {} successfully.", containerID);
} finally {
lock.writeLock().unlock();
}
}
/**
@ -181,9 +184,11 @@ public class ContainerStateMap {
* @param containerID - ContainerID
* @return container info, if found.
*/
public ContainerInfo getContainerInfo(ContainerID containerID) {
public ContainerInfo getContainerInfo(final ContainerID containerID)
throws ContainerNotFoundException {
lock.readLock().lock();
try {
checkIfContainerExist(containerID);
return containerMap.get(containerID);
} finally {
lock.readLock().unlock();
@ -197,21 +202,17 @@ public class ContainerStateMap {
* @param containerID
* @return Set<DatanodeDetails>
*/
public Set<DatanodeDetails> getContainerReplicas(ContainerID containerID)
throws SCMException {
public Set<ContainerReplica> getContainerReplicas(
final ContainerID containerID) throws ContainerNotFoundException {
Preconditions.checkNotNull(containerID);
lock.readLock().lock();
try {
if (contReplicaMap.containsKey(containerID)) {
return Collections
.unmodifiableSet(contReplicaMap.get(containerID));
}
checkIfContainerExist(containerID);
return Collections
.unmodifiableSet(new HashSet<>(replicaMap.get(containerID)));
} finally {
lock.readLock().unlock();
}
throw new SCMException(
"No entry exist for containerId: " + containerID + " in replica map.",
ResultCodes.NO_REPLICA_FOUND);
}
/**
@ -220,26 +221,17 @@ public class ContainerStateMap {
* ContainerId.
*
* @param containerID
* @param dnList
* @param replica
*/
public void addContainerReplica(ContainerID containerID,
DatanodeDetails... dnList) {
public void updateContainerReplica(final ContainerID containerID,
final ContainerReplica replica) throws ContainerNotFoundException {
Preconditions.checkNotNull(containerID);
lock.writeLock().lock();
try {
for (DatanodeDetails dn : dnList) {
Preconditions.checkNotNull(dn);
if (contReplicaMap.containsKey(containerID)) {
if(!contReplicaMap.get(containerID).add(dn)) {
LOG.debug("ReplicaMap already contains entry for container Id: "
+ "{},DataNode: {}", containerID, dn);
}
} else {
Set<DatanodeDetails> dnSet = new HashSet<>();
dnSet.add(dn);
contReplicaMap.put(containerID, dnSet);
}
}
checkIfContainerExist(containerID);
Set<ContainerReplica> replicas = replicaMap.get(containerID);
replicas.remove(replica);
replicas.add(replica);
} finally {
lock.writeLock().unlock();
}
@ -249,61 +241,45 @@ public class ContainerStateMap {
* Remove a container Replica for given DataNode.
*
* @param containerID
* @param dn
* @param replica
* @return True of dataNode is removed successfully else false.
*/
public boolean removeContainerReplica(ContainerID containerID,
DatanodeDetails dn) throws SCMException {
public void removeContainerReplica(final ContainerID containerID,
final ContainerReplica replica)
throws ContainerNotFoundException, ContainerReplicaNotFoundException {
Preconditions.checkNotNull(containerID);
Preconditions.checkNotNull(dn);
Preconditions.checkNotNull(replica);
lock.writeLock().lock();
try {
if (contReplicaMap.containsKey(containerID)) {
return contReplicaMap.get(containerID).remove(dn);
checkIfContainerExist(containerID);
if(!replicaMap.get(containerID).remove(replica)) {
throw new ContainerReplicaNotFoundException(
"Container #"
+ containerID.getId() + ", replica: " + replica);
}
} finally {
lock.writeLock().unlock();
}
throw new SCMException(
"No entry exist for containerId: " + containerID + " in replica map.",
ResultCodes.FAILED_TO_FIND_CONTAINER);
}
@VisibleForTesting
// TODO: fix the test case and remove this method!
public static Logger getLOG() {
return LOG;
}
/**
* Returns the full container Map.
*
* @return - Map
*/
public Map<ContainerID, ContainerInfo> getContainerMap() {
lock.readLock().lock();
try {
return Collections.unmodifiableMap(containerMap);
} finally {
lock.readLock().unlock();
}
}
/**
* Just update the container State.
* @param info ContainerInfo.
*/
public void updateContainerInfo(ContainerInfo info) throws SCMException {
Preconditions.checkNotNull(info);
ContainerInfo currentInfo = null;
public void updateContainerInfo(final ContainerInfo info)
throws ContainerNotFoundException {
lock.writeLock().lock();
try {
currentInfo = containerMap.get(
ContainerID.valueof(info.getContainerID()));
if (currentInfo == null) {
throw new SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
}
Preconditions.checkNotNull(info);
checkIfContainerExist(info.containerID());
final ContainerInfo currentInfo = containerMap.get(info.containerID());
flushCache(info, currentInfo);
containerMap.put(info.containerID(), info);
} finally {
@ -314,33 +290,23 @@ public class ContainerStateMap {
/**
* Update the State of a container.
*
* @param info - ContainerInfo
* @param containerID - ContainerID
* @param currentState - CurrentState
* @param newState - NewState.
* @throws SCMException - in case of failure.
*/
public void updateState(ContainerInfo info, LifeCycleState currentState,
LifeCycleState newState) throws SCMException {
public void updateState(ContainerID containerID, LifeCycleState currentState,
LifeCycleState newState) throws SCMException, ContainerNotFoundException {
Preconditions.checkNotNull(currentState);
Preconditions.checkNotNull(newState);
ContainerID id = new ContainerID(info.getContainerID());
ContainerInfo currentInfo = null;
lock.writeLock().lock();
try {
checkIfContainerExist(containerID);
final ContainerInfo currentInfo = containerMap.get(containerID);
try {
// Just flush both old and new data sets from the result cache.
ContainerInfo newInfo = new ContainerInfo(info);
final ContainerInfo newInfo = new ContainerInfo(currentInfo);
newInfo.setState(newState);
flushCache(newInfo, info);
currentInfo = containerMap.get(id);
if (currentInfo == null) {
throw new
SCMException("No such container.", FAILED_TO_FIND_CONTAINER);
}
// We are updating two places before this update is done, these can
// fail independently, since the code needs to handle it.
@ -351,11 +317,13 @@ public class ContainerStateMap {
// roll back the earlier change we did. If the rollback fails, we can
// be in an inconsistent state,
info.setState(newState);
containerMap.put(id, info);
lifeCycleStateMap.update(currentState, newState, id);
containerMap.put(containerID, newInfo);
lifeCycleStateMap.update(currentState, newState, containerID);
LOG.trace("Updated the container {} to new state. Old = {}, new = " +
"{}", id, currentState, newState);
"{}", containerID, currentState, newState);
// Just flush both old and new data sets from the result cache.
flushCache(currentInfo, newInfo);
} catch (SCMException ex) {
LOG.error("Unable to update the container state. {}", ex);
// we need to revert the change in this attribute since we are not
@ -364,13 +332,13 @@ public class ContainerStateMap {
"old state. Old = {}, Attempted state = {}", currentState,
newState);
containerMap.put(id, currentInfo);
containerMap.put(containerID, currentInfo);
// if this line throws, the state map can be in an inconsistent
// state, since we will have modified the attribute by the
// container state will not in sync since we were not able to put
// that into the hash table.
lifeCycleStateMap.update(newState, currentState, id);
lifeCycleStateMap.update(newState, currentState, containerID);
throw new SCMException("Updating the container map failed.", ex,
FAILED_TO_CHANGE_CONTAINER_STATE);
@ -380,13 +348,17 @@ public class ContainerStateMap {
}
}
public Set<ContainerID> getAllContainerIDs() {
return containerMap.keySet();
}
/**
* Returns A list of containers owned by a name service.
*
* @param ownerName - Name of the NameService.
* @return - NavigableSet of ContainerIDs.
*/
NavigableSet<ContainerID> getContainerIDsByOwner(String ownerName) {
NavigableSet<ContainerID> getContainerIDsByOwner(final String ownerName) {
Preconditions.checkNotNull(ownerName);
lock.readLock().lock();
try {
@ -402,7 +374,7 @@ public class ContainerStateMap {
* @param type - Replication type -- StandAlone, Ratis etc.
* @return NavigableSet
*/
NavigableSet<ContainerID> getContainerIDsByType(ReplicationType type) {
NavigableSet<ContainerID> getContainerIDsByType(final ReplicationType type) {
Preconditions.checkNotNull(type);
lock.readLock().lock();
try {
@ -418,7 +390,8 @@ public class ContainerStateMap {
* @param factor - Replication Factor.
* @return NavigableSet.
*/
NavigableSet<ContainerID> getContainerIDsByFactor(ReplicationFactor factor) {
NavigableSet<ContainerID> getContainerIDsByFactor(
final ReplicationFactor factor) {
Preconditions.checkNotNull(factor);
lock.readLock().lock();
try {
@ -435,7 +408,7 @@ public class ContainerStateMap {
* @return List of containers by state.
*/
public NavigableSet<ContainerID> getContainerIDsByState(
LifeCycleState state) {
final LifeCycleState state) {
Preconditions.checkNotNull(state);
lock.readLock().lock();
try {
@ -455,8 +428,8 @@ public class ContainerStateMap {
* @return ContainerInfo or Null if not container satisfies the criteria.
*/
public NavigableSet<ContainerID> getMatchingContainerIDs(
LifeCycleState state, String owner,
ReplicationFactor factor, ReplicationType type) {
final LifeCycleState state, final String owner,
final ReplicationFactor factor, final ReplicationType type) {
Preconditions.checkNotNull(state, "State cannot be null");
Preconditions.checkNotNull(owner, "Owner cannot be null");
@ -465,7 +438,7 @@ public class ContainerStateMap {
lock.readLock().lock();
try {
ContainerQueryKey queryKey =
final ContainerQueryKey queryKey =
new ContainerQueryKey(state, owner, factor, type);
if(resultCache.containsKey(queryKey)){
return resultCache.get(queryKey);
@ -474,30 +447,33 @@ public class ContainerStateMap {
// If we cannot meet any one condition we return EMPTY_SET immediately.
// Since when we intersect these sets, the result will be empty if any
// one is empty.
NavigableSet<ContainerID> stateSet =
final NavigableSet<ContainerID> stateSet =
lifeCycleStateMap.getCollection(state);
if (stateSet.size() == 0) {
return EMPTY_SET;
}
NavigableSet<ContainerID> ownerSet = ownerMap.getCollection(owner);
final NavigableSet<ContainerID> ownerSet =
ownerMap.getCollection(owner);
if (ownerSet.size() == 0) {
return EMPTY_SET;
}
NavigableSet<ContainerID> factorSet = factorMap.getCollection(factor);
final NavigableSet<ContainerID> factorSet =
factorMap.getCollection(factor);
if (factorSet.size() == 0) {
return EMPTY_SET;
}
NavigableSet<ContainerID> typeSet = typeMap.getCollection(type);
final NavigableSet<ContainerID> typeSet =
typeMap.getCollection(type);
if (typeSet.size() == 0) {
return EMPTY_SET;
}
// if we add more constraints we will just add those sets here..
NavigableSet<ContainerID>[] sets = sortBySize(stateSet,
final NavigableSet<ContainerID>[] sets = sortBySize(stateSet,
ownerSet, factorSet, typeSet);
NavigableSet<ContainerID> currentSet = sets[0];
@ -521,12 +497,12 @@ public class ContainerStateMap {
* @return resultSet which is the intersection of these two sets.
*/
private NavigableSet<ContainerID> intersectSets(
NavigableSet<ContainerID> smaller,
NavigableSet<ContainerID> bigger) {
final NavigableSet<ContainerID> smaller,
final NavigableSet<ContainerID> bigger) {
Preconditions.checkState(smaller.size() <= bigger.size(),
"This function assumes the first set is lesser or equal to second " +
"set");
NavigableSet<ContainerID> resultSet = new TreeSet<>();
final NavigableSet<ContainerID> resultSet = new TreeSet<>();
for (ContainerID id : smaller) {
if (bigger.contains(id)) {
resultSet.add(id);
@ -544,11 +520,11 @@ public class ContainerStateMap {
*/
@SuppressWarnings("unchecked")
private NavigableSet<ContainerID>[] sortBySize(
NavigableSet<ContainerID>... sets) {
final NavigableSet<ContainerID>... sets) {
for (int x = 0; x < sets.length - 1; x++) {
for (int y = 0; y < sets.length - x - 1; y++) {
if (sets[y].size() > sets[y + 1].size()) {
NavigableSet temp = sets[y];
final NavigableSet temp = sets[y];
sets[y] = sets[y + 1];
sets[y + 1] = temp;
}
@ -557,13 +533,22 @@ public class ContainerStateMap {
return sets;
}
private void flushCache(ContainerInfo... containerInfos) {
private void flushCache(final ContainerInfo... containerInfos) {
for (ContainerInfo containerInfo : containerInfos) {
ContainerQueryKey key = new ContainerQueryKey(containerInfo.getState(),
containerInfo.getOwner(), containerInfo.getReplicationFactor(),
final ContainerQueryKey key = new ContainerQueryKey(
containerInfo.getState(),
containerInfo.getOwner(),
containerInfo.getReplicationFactor(),
containerInfo.getReplicationType());
resultCache.remove(key);
}
}
private void checkIfContainerExist(ContainerID containerID)
throws ContainerNotFoundException {
if (!containerMap.containsKey(containerID)) {
throw new ContainerNotFoundException("#" + containerID.getId());
}
}
}

View File

@ -21,11 +21,14 @@ package org.apache.hadoop.hdds.scm.node;
import java.util.Set;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerException;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
@ -37,7 +40,7 @@ import org.slf4j.LoggerFactory;
*/
public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
private final ContainerStateManager containerStateManager;
private final ContainerManager containerManager;
private final NodeManager nodeManager;
@ -45,8 +48,8 @@ public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
LoggerFactory.getLogger(DeadNodeHandler.class);
public DeadNodeHandler(NodeManager nodeManager,
ContainerStateManager containerStateManager) {
this.containerStateManager = containerStateManager;
ContainerManager containerManager) {
this.containerManager = containerManager;
this.nodeManager = nodeManager;
}
@ -55,45 +58,58 @@ public class DeadNodeHandler implements EventHandler<DatanodeDetails> {
EventPublisher publisher) {
nodeManager.processDeadNode(datanodeDetails.getUuid());
Set<ContainerID> containers =
// TODO: check if there are any pipeline on this node and fire close
// pipeline event
Set<ContainerID> ids =
nodeManager.getContainers(datanodeDetails.getUuid());
if (containers == null) {
if (ids == null) {
LOG.info("There's no containers in dead datanode {}, no replica will be"
+ " removed from the in-memory state.", datanodeDetails.getUuid());
return;
}
LOG.info(
"Datanode {} is dead. Removing replications from the in-memory state.",
datanodeDetails.getUuid());
for (ContainerID container : containers) {
LOG.info("Datanode {} is dead. Removing replications from the in-memory" +
" state.", datanodeDetails.getUuid());
for (ContainerID id : ids) {
try {
try {
containerStateManager.removeContainerReplica(container,
datanodeDetails);
} catch (SCMException ex) {
LOG.info("DataNode {} doesn't have replica for container {}.",
datanodeDetails.getUuid(), container.getId());
}
if (!containerStateManager.isOpen(container)) {
ReplicationRequest replicationRequest =
containerStateManager.checkReplicationState(container);
if (replicationRequest != null) {
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
replicationRequest);
final ContainerInfo container = containerManager.getContainer(id);
if (!container.isOpen()) {
final ContainerReplica replica = ContainerReplica.newBuilder()
.setContainerID(id)
.setDatanodeDetails(datanodeDetails)
.build();
try {
containerManager.removeContainerReplica(id, replica);
replicateIfNeeded(container, publisher);
} catch (ContainerException ex) {
LOG.warn("Exception while removing container replica #{} for " +
"container #{}.", replica, container, ex);
}
}
} catch (SCMException e) {
LOG.error("Can't remove container from containerStateMap {}", container
.getId(), e);
} catch (ContainerNotFoundException cnfe) {
LOG.warn("Container Not found!", cnfe);
}
}
}
/**
* Compare the existing replication number with the expected one.
*/
private void replicateIfNeeded(ContainerInfo container,
EventPublisher publisher) throws ContainerNotFoundException {
final int existingReplicas = containerManager
.getContainerReplicas(container.containerID()).size();
final int expectedReplicas = container.getReplicationFactor().getNumber();
if (existingReplicas != expectedReplicas) {
publisher.fireEvent(SCMEvents.REPLICATE_CONTAINER,
new ReplicationRequest(
container.getContainerID(), existingReplicas, expectedReplicas));
}
}
/**
* Returns logger.
* */
// TODO: remove this.
public static Logger getLogger() {
return LOG;
}

View File

@ -31,7 +31,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
.NodeRegistrationContainerReport;

View File

@ -33,8 +33,10 @@ import org.apache.hadoop.hdds.protocol.proto
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.ScmUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
@ -169,7 +171,7 @@ public class SCMClientProtocolServer implements
String remoteUser = getRpcRemoteUsername();
getScm().checkAdminAccess(remoteUser);
return scm.getContainerManager()
.getContainer(containerID);
.getContainer(ContainerID.valueof(containerID));
}
@Override
@ -177,8 +179,8 @@ public class SCMClientProtocolServer implements
throws IOException {
if (chillModePrecheck.isInChillMode()) {
ContainerInfo contInfo = scm.getContainerManager()
.getContainer(containerID);
if (contInfo.isContainerOpen()) {
.getContainer(ContainerID.valueof(containerID));
if (contInfo.isOpen()) {
if (!hasRequiredReplicas(contInfo)) {
throw new SCMException("Open container " + containerID + " doesn't"
+ " have enough replicas to service this operation in "
@ -189,7 +191,7 @@ public class SCMClientProtocolServer implements
String remoteUser = getRpcRemoteUsername();
getScm().checkAdminAccess(null);
return scm.getContainerManager()
.getContainerWithPipeline(containerID);
.getContainerWithPipeline(ContainerID.valueof(containerID));
}
/**
@ -198,10 +200,10 @@ public class SCMClientProtocolServer implements
*/
private boolean hasRequiredReplicas(ContainerInfo contInfo) {
try{
return getScm().getContainerManager().getStateManager()
return getScm().getContainerManager()
.getContainerReplicas(contInfo.containerID())
.size() >= contInfo.getReplicationFactor().getNumber();
} catch (SCMException ex) {
} catch (ContainerNotFoundException ex) {
// getContainerReplicas throws exception if no replica's exist for given
// container.
return false;
@ -212,14 +214,14 @@ public class SCMClientProtocolServer implements
public List<ContainerInfo> listContainer(long startContainerID,
int count) throws IOException {
return scm.getContainerManager().
listContainer(startContainerID, count);
listContainer(ContainerID.valueof(startContainerID), count);
}
@Override
public void deleteContainer(long containerID) throws IOException {
String remoteUser = getRpcRemoteUsername();
getScm().checkAdminAccess(remoteUser);
scm.getContainerManager().deleteContainer(containerID);
scm.getContainerManager().deleteContainer(ContainerID.valueof(containerID));
}
@ -257,10 +259,12 @@ public class SCMClientProtocolServer implements
.ObjectStageChangeRequestProto.Op.create) {
if (stage == StorageContainerLocationProtocolProtos
.ObjectStageChangeRequestProto.Stage.begin) {
scm.getContainerManager().updateContainerState(id, HddsProtos
scm.getContainerManager().updateContainerState(
ContainerID.valueof(id), HddsProtos
.LifeCycleEvent.CREATE);
} else {
scm.getContainerManager().updateContainerState(id, HddsProtos
scm.getContainerManager().updateContainerState(
ContainerID.valueof(id), HddsProtos
.LifeCycleEvent.CREATED);
}
} else {
@ -268,10 +272,12 @@ public class SCMClientProtocolServer implements
.ObjectStageChangeRequestProto.Op.close) {
if (stage == StorageContainerLocationProtocolProtos
.ObjectStageChangeRequestProto.Stage.begin) {
scm.getContainerManager().updateContainerState(id, HddsProtos
scm.getContainerManager().updateContainerState(
ContainerID.valueof(id), HddsProtos
.LifeCycleEvent.FINALIZE);
} else {
scm.getContainerManager().updateContainerState(id, HddsProtos
scm.getContainerManager().updateContainerState(
ContainerID.valueof(id), HddsProtos
.LifeCycleEvent.CLOSE);
}
}

View File

@ -197,7 +197,7 @@ public class SCMDatanodeProtocolServer implements
if (registeredCommand.getError()
== SCMRegisteredResponseProto.ErrorCode.success) {
scm.getContainerManager().processContainerReports(datanodeDetails,
containerReportsProto, true);
containerReportsProto);
eventPublisher.fireEvent(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
new NodeRegistrationContainerReport(datanodeDetails,
containerReportsProto));

View File

@ -40,13 +40,14 @@ import org.apache.hadoop.hdds.scm.command.CommandStatusReportHandler;
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler;
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
import org.apache.hadoop.hdds.scm.container.ContainerActionsHandler;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerReportHandler;
import org.apache.hadoop.hdds.scm.container.replication
.ReplicationActivityStatus;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.placement.algorithms
.ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.placement.algorithms
@ -97,9 +98,6 @@ import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_DEFAULT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DB_CACHE_SIZE_MB;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
import static org.apache.hadoop.util.ExitUtil.terminate;
@ -190,9 +188,6 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
*/
private StorageContainerManager(OzoneConfiguration conf) throws IOException {
final int cacheSize = conf.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
StorageContainerManager.initMetrics();
initContainerReportCache(conf);
@ -207,9 +202,9 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
scmNodeManager = new SCMNodeManager(
conf, scmStorage.getClusterID(), this, eventQueue);
containerManager = new SCMContainerManager(
conf, getScmNodeManager(), cacheSize, eventQueue);
conf, scmNodeManager, eventQueue);
scmBlockManager = new BlockManagerImpl(
conf, getScmNodeManager(), containerManager, eventQueue);
conf, scmNodeManager, containerManager, eventQueue);
replicationStatus = new ReplicationActivityStatus();
@ -227,7 +222,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
StaleNodeHandler staleNodeHandler =
new StaleNodeHandler(containerManager.getPipelineSelector());
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager,
getContainerManager().getStateManager());
containerManager);
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
PendingDeleteHandler pendingDeleteHandler =
new PendingDeleteHandler(scmBlockManager.getSCMBlockDeletingService());
@ -236,7 +231,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
new ContainerReportHandler(containerManager, scmNodeManager,
replicationStatus);
scmChillModeManager = new SCMChillModeManager(conf,
getContainerManager().getStateManager().getAllContainers(),
containerManager.getContainers(),
eventQueue);
PipelineActionEventHandler pipelineActionEventHandler =
new PipelineActionEventHandler();
@ -263,8 +258,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
new SCMContainerPlacementCapacity(scmNodeManager, conf);
replicationManager = new ReplicationManager(containerPlacementPolicy,
containerManager.getStateManager(), eventQueue,
commandWatcherLeaseManager);
containerManager, eventQueue, commandWatcherLeaseManager);
// setup CloseContainer watcher
CloseContainerWatcher closeContainerWatcher =
@ -632,7 +626,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
@VisibleForTesting
public ContainerInfo getContainerInfo(long containerID) throws
IOException {
return containerManager.getContainer(containerID);
return containerManager.getContainer(ContainerID.valueof(containerID));
}
/**

View File

@ -22,7 +22,7 @@ import java.util.List;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
.NodeRegistrationContainerReport;

View File

@ -23,11 +23,10 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.server
.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import org.apache.hadoop.hdds.protocol.proto
@ -42,13 +41,8 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.StorageTypeProto;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.protocol.commands.RegisteredCommand;
@ -412,39 +406,21 @@ public final class TestUtils {
return report.build();
}
public static
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
allocateContainer(ContainerStateManager containerStateManager)
public static org.apache.hadoop.hdds.scm.container.ContainerInfo
allocateContainer(ContainerManager containerManager)
throws IOException {
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
Pipeline pipeline = new Pipeline("leader", HddsProtos.LifeCycleState.CLOSED,
HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE,
PipelineID.randomId());
when(pipelineSelector
.getReplicationPipeline(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
return containerStateManager
.allocateContainer(pipelineSelector,
HddsProtos.ReplicationType.STAND_ALONE,
return containerManager
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE, "root").getContainerInfo();
}
public static void closeContainer(ContainerStateManager containerStateManager,
org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
container)
throws SCMException {
containerStateManager.getContainerStateMap()
.updateState(container, container.getState(), LifeCycleState.CLOSING);
containerStateManager.getContainerStateMap()
.updateState(container, container.getState(), LifeCycleState.CLOSED);
public static void closeContainer(ContainerManager containerManager,
ContainerID id) throws IOException {
containerManager.updateContainerState(
id, HddsProtos.LifeCycleEvent.FINALIZE);
containerManager.updateContainerState(
id, HddsProtos.LifeCycleEvent.CLOSE);
}
}

View File

@ -83,7 +83,7 @@ public class TestBlockManager implements EventHandler<Boolean> {
throw new IOException("Unable to create test directory path");
}
nodeManager = new MockNodeManager(true, 10);
mapping = new SCMContainerManager(conf, nodeManager, 128, eventQueue);
mapping = new SCMContainerManager(conf, nodeManager, eventQueue);
blockManager = new BlockManagerImpl(conf,
nodeManager, mapping, eventQueue);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);

View File

@ -19,9 +19,10 @@ package org.apache.hadoop.hdds.scm.block;
import org.apache.commons.io.FileUtils;
import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
@ -61,7 +62,7 @@ import java.util.stream.Collectors;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys
.OZONE_SCM_BLOCK_DELETION_MAX_RETRY;
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_METADATA_DIRS;
import static org.mockito.Matchers.anyLong;
import static org.mockito.Matchers.anyObject;
import static org.mockito.Mockito.when;
/**
@ -109,9 +110,10 @@ public class TestDeletedBlockLog {
pipeline.addMember(dnList.get(2));
ContainerWithPipeline containerWithPipeline =
new ContainerWithPipeline(containerInfo, pipeline);
when(containerManager.getContainerWithPipeline(anyLong()))
when(containerManager.getContainerWithPipeline(anyObject()))
.thenReturn(containerWithPipeline);
when(containerManager.getContainer(anyLong())).thenReturn(containerInfo);
when(containerManager.getContainer(anyObject()))
.thenReturn(containerInfo);
}
@After
@ -396,8 +398,8 @@ public class TestDeletedBlockLog {
ContainerWithPipeline containerWithPipeline = new ContainerWithPipeline(
containerInfo, pipeline);
Mockito.doReturn(containerInfo).when(containerManager)
.getContainer(containerID);
.getContainer(ContainerID.valueof(containerID));
Mockito.doReturn(containerWithPipeline).when(containerManager)
.getContainerWithPipeline(containerID);
.getContainerWithPipeline(ContainerID.valueof(containerID));
}
}

View File

@ -50,7 +50,7 @@ public class TestCloseContainerEventHandler {
private static Configuration configuration;
private static MockNodeManager nodeManager;
private static SCMContainerManager mapping;
private static SCMContainerManager containerManager;
private static long size;
private static File testDir;
private static EventQueue eventQueue;
@ -65,18 +65,18 @@ public class TestCloseContainerEventHandler {
configuration
.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
nodeManager = new MockNodeManager(true, 10);
mapping = new SCMContainerManager(configuration, nodeManager, 128,
containerManager = new SCMContainerManager(configuration, nodeManager,
new EventQueue());
eventQueue = new EventQueue();
eventQueue.addHandler(CLOSE_CONTAINER,
new CloseContainerEventHandler(mapping));
new CloseContainerEventHandler(containerManager));
eventQueue.addHandler(DATANODE_COMMAND, nodeManager);
}
@AfterClass
public static void tearDown() throws Exception {
if (mapping != null) {
mapping.close();
if (containerManager != null) {
containerManager.close();
}
FileUtil.fullyDelete(testDir);
}
@ -109,7 +109,7 @@ public class TestCloseContainerEventHandler {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(CloseContainerEventHandler.LOG);
ContainerWithPipeline containerWithPipeline = mapping
ContainerWithPipeline containerWithPipeline = containerManager
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.ONE, "ozone");
ContainerID id = new ContainerID(
@ -123,7 +123,7 @@ public class TestCloseContainerEventHandler {
// command in the Datanode
Assert.assertEquals(0, nodeManager.getCommandCount(datanode));
//Execute these state transitions so that we can close the container.
mapping.updateContainerState(id.getId(), CREATED);
containerManager.updateContainerState(id, CREATED);
eventQueue.fireEvent(CLOSE_CONTAINER,
new ContainerID(
containerWithPipeline.getContainerInfo().getContainerID()));
@ -131,7 +131,7 @@ public class TestCloseContainerEventHandler {
Assert.assertEquals(closeCount + 1,
nodeManager.getCommandCount(datanode));
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
mapping.getStateManager().getContainer(id).getState());
containerManager.getContainer(id).getState());
}
@Test
@ -139,7 +139,7 @@ public class TestCloseContainerEventHandler {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(CloseContainerEventHandler.LOG);
ContainerWithPipeline containerWithPipeline = mapping
ContainerWithPipeline containerWithPipeline = containerManager
.allocateContainer(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE, "ozone");
ContainerID id = new ContainerID(
@ -160,7 +160,7 @@ public class TestCloseContainerEventHandler {
i++;
}
//Execute these state transitions so that we can close the container.
mapping.updateContainerState(id.getId(), CREATED);
containerManager.updateContainerState(id, CREATED);
eventQueue.fireEvent(CLOSE_CONTAINER, id);
eventQueue.processAll(1000);
i = 0;
@ -170,7 +170,7 @@ public class TestCloseContainerEventHandler {
Assert.assertEquals(closeCount[i] + 1,
nodeManager.getCommandCount(details));
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,
mapping.getStateManager().getContainer(id).getState());
containerManager.getContainer(id).getState());
i++;
}
}

View File

@ -23,34 +23,28 @@ import java.util.List;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo
.Builder;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.container.replication
.ReplicationActivityStatus;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.ContainerReportFromDatanode;
import org.apache.hadoop.hdds.server.events.Event;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import static org.mockito.Matchers.anyLong;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
import org.mockito.stubbing.Answer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -60,7 +54,7 @@ import org.slf4j.LoggerFactory;
public class TestContainerReportHandler implements EventPublisher {
private List<Object> publishedEvents = new ArrayList<>();
private final NodeManager nodeManager = new MockNodeManager(true, 1);
private final NodeManager nodeManager = new MockNodeManager(true, 15);
private static final Logger LOG =
LoggerFactory.getLogger(TestContainerReportHandler.class);
@ -70,27 +64,17 @@ public class TestContainerReportHandler implements EventPublisher {
publishedEvents.clear();
}
//TODO: Rewrite it
@Ignore
@Test
public void test() throws IOException {
String testDir = GenericTestUtils.getTempPath(
this.getClass().getSimpleName());
//GIVEN
OzoneConfiguration conf = new OzoneConfiguration();
ContainerManager containerManager = Mockito.mock(ContainerManager.class);
PipelineSelector selector = Mockito.mock(PipelineSelector.class);
when(containerManager.getContainer(anyLong()))
.thenAnswer(
(Answer<ContainerInfo>) invocation ->
new Builder()
.setReplicationFactor(ReplicationFactor.THREE)
.setContainerID((Long) invocation.getArguments()[0])
.setState(LifeCycleState.CLOSED)
.build()
);
ContainerStateManager containerStateManager =
new ContainerStateManager(conf, containerManager, selector);
when(containerManager.getStateManager()).thenReturn(containerStateManager);
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir);
SCMContainerManager containerManager = new SCMContainerManager(
conf, nodeManager, new EventQueue());
ReplicationActivityStatus replicationActivityStatus =
new ReplicationActivityStatus();
@ -107,24 +91,16 @@ public class TestContainerReportHandler implements EventPublisher {
nodeManager.addDatanodeInContainerMap(dn2.getUuid(), new HashSet<>());
nodeManager.addDatanodeInContainerMap(dn3.getUuid(), new HashSet<>());
nodeManager.addDatanodeInContainerMap(dn4.getUuid(), new HashSet<>());
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
Pipeline pipeline = new Pipeline("leader", LifeCycleState.CLOSED,
ReplicationType.STAND_ALONE, ReplicationFactor.THREE,
PipelineID.randomId());
when(pipelineSelector.getReplicationPipeline(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE)).thenReturn(pipeline);
ContainerInfo cont1 = containerStateManager
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
ContainerInfo cont1 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo();
ContainerInfo cont2 = containerStateManager
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
ContainerInfo cont2 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo();
// Open Container
ContainerInfo cont3 = containerStateManager
.allocateContainer(pipelineSelector, ReplicationType.STAND_ALONE,
ContainerInfo cont3 = containerManager
.allocateContainer(ReplicationType.STAND_ALONE,
ReplicationFactor.THREE, "root").getContainerInfo();
long c1 = cont1.getContainerID();
@ -132,8 +108,8 @@ public class TestContainerReportHandler implements EventPublisher {
long c3 = cont3.getContainerID();
// Close remaining containers
TestUtils.closeContainer(containerStateManager, cont1);
TestUtils.closeContainer(containerStateManager, cont2);
TestUtils.closeContainer(containerManager, cont1.containerID());
TestUtils.closeContainer(containerManager, cont2.containerID());
//when

View File

@ -18,19 +18,23 @@
package org.apache.hadoop.hdds.scm.container;
import java.io.IOException;
import java.util.Set;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.Mockito;
import static org.mockito.Mockito.when;
/**
* Testing ContainerStatemanager.
*/
@ -41,16 +45,14 @@ public class TestContainerStateManager {
@Before
public void init() throws IOException {
OzoneConfiguration conf = new OzoneConfiguration();
ContainerManager mapping = Mockito.mock(ContainerManager.class);
PipelineSelector selector = Mockito.mock(PipelineSelector.class);
containerStateManager = new ContainerStateManager(conf, mapping, selector);
containerStateManager = new ContainerStateManager(conf);
}
@Test
public void checkReplicationStateOK() throws IOException {
//GIVEN
ContainerInfo c1 = TestUtils.allocateContainer(containerStateManager);
ContainerInfo c1 = allocateContainer();
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
@ -61,18 +63,18 @@ public class TestContainerStateManager {
addReplica(c1, d3);
//WHEN
ReplicationRequest replicationRequest = containerStateManager
.checkReplicationState(new ContainerID(c1.getContainerID()));
Set<ContainerReplica> replicas = containerStateManager
.getContainerReplicas(c1.containerID());
//THEN
Assert.assertNull(replicationRequest);
Assert.assertEquals(3, replicas.size());
}
@Test
public void checkReplicationStateMissingReplica() throws IOException {
//GIVEN
ContainerInfo c1 = TestUtils.allocateContainer(containerStateManager);
ContainerInfo c1 = allocateContainer();
DatanodeDetails d1 = TestUtils.randomDatanodeDetails();
DatanodeDetails d2 = TestUtils.randomDatanodeDetails();
@ -81,18 +83,40 @@ public class TestContainerStateManager {
addReplica(c1, d2);
//WHEN
ReplicationRequest replicationRequest = containerStateManager
.checkReplicationState(new ContainerID(c1.getContainerID()));
Set<ContainerReplica> replicas = containerStateManager
.getContainerReplicas(c1.containerID());
Assert
.assertEquals(c1.getContainerID(), replicationRequest.getContainerId());
Assert.assertEquals(2, replicationRequest.getReplicationCount());
Assert.assertEquals(3, replicationRequest.getExpecReplicationCount());
Assert.assertEquals(2, replicas.size());
Assert.assertEquals(3, c1.getReplicationFactor().getNumber());
}
private void addReplica(ContainerInfo c1, DatanodeDetails d1) {
private void addReplica(ContainerInfo cont, DatanodeDetails node)
throws ContainerNotFoundException {
ContainerReplica replica = ContainerReplica.newBuilder()
.setContainerID(cont.containerID())
.setDatanodeDetails(node)
.build();
containerStateManager
.addContainerReplica(new ContainerID(c1.getContainerID()), d1);
.updateContainerReplica(cont.containerID(), replica);
}
private ContainerInfo allocateContainer() throws IOException {
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
Pipeline pipeline = new Pipeline("leader", HddsProtos.LifeCycleState.CLOSED,
HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE,
PipelineID.randomId());
when(pipelineSelector
.getReplicationPipeline(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
return containerStateManager.allocateContainer(
pipelineSelector, HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE, "root");
}
}

View File

@ -23,7 +23,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
@ -32,12 +31,10 @@ import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
@ -50,7 +47,6 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import java.util.Random;
import java.util.Set;
import java.util.TreeSet;
@ -89,7 +85,7 @@ public class TestSCMContainerManager {
throw new IOException("Unable to create test directory path");
}
nodeManager = new MockNodeManager(true, 10);
containerManager = new SCMContainerManager(conf, nodeManager, 128,
containerManager = new SCMContainerManager(conf, nodeManager,
new EventQueue());
xceiverClientManager = new XceiverClientManager(conf);
random = new Random();
@ -169,28 +165,36 @@ public class TestSCMContainerManager {
.setIpAddress("2.2.2.2")
.setUuid(UUID.randomUUID().toString()).build();
containerManager
.updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CREATE);
containerManager.updateContainerState(contInfo.getContainerID(),
.updateContainerState(contInfo.containerID(), LifeCycleEvent.CREATE);
containerManager.updateContainerState(contInfo.containerID(),
LifeCycleEvent.CREATED);
containerManager.updateContainerState(contInfo.getContainerID(),
containerManager.updateContainerState(contInfo.containerID(),
LifeCycleEvent.FINALIZE);
containerManager
.updateContainerState(contInfo.getContainerID(), LifeCycleEvent.CLOSE);
.updateContainerState(contInfo.containerID(), LifeCycleEvent.CLOSE);
ContainerInfo finalContInfo = contInfo;
LambdaTestUtils.intercept(SCMException.class, "No entry exist for "
+ "containerId:", () -> containerManager.getContainerWithPipeline(
finalContInfo.getContainerID()));
Assert.assertEquals(0,
containerManager.getContainerReplicas(
finalContInfo.containerID()).size());
containerManager.getStateManager().getContainerStateMap()
.addContainerReplica(contInfo.containerID(), dn1, dn2);
containerManager.updateContainerReplica(contInfo.containerID(),
ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
.setDatanodeDetails(dn1).build());
containerManager.updateContainerReplica(contInfo.containerID(),
ContainerReplica.newBuilder().setContainerID(contInfo.containerID())
.setDatanodeDetails(dn2).build());
contInfo = containerManager.getContainer(contInfo.getContainerID());
Assert.assertEquals(2,
containerManager.getContainerReplicas(
finalContInfo.containerID()).size());
contInfo = containerManager.getContainer(contInfo.containerID());
Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED);
Pipeline pipeline = containerWithPipeline.getPipeline();
containerManager.getPipelineSelector().finalizePipeline(pipeline);
ContainerWithPipeline containerWithPipeline2 = containerManager
.getContainerWithPipeline(contInfo.getContainerID());
.getContainerWithPipeline(contInfo.containerID());
pipeline = containerWithPipeline2.getPipeline();
Assert.assertNotEquals(containerWithPipeline, containerWithPipeline2);
Assert.assertNotNull("Pipeline should not be null", pipeline);
@ -199,9 +203,14 @@ public class TestSCMContainerManager {
}
@Test
public void testgetNoneExistentContainer() throws IOException {
thrown.expectMessage("Specified key does not exist.");
containerManager.getContainer(random.nextLong());
public void testgetNoneExistentContainer() {
try {
containerManager.getContainer(ContainerID.valueof(
random.nextInt() & Integer.MAX_VALUE));
Assert.fail();
} catch (ContainerNotFoundException ex) {
// Success!
}
}
@Test
@ -213,21 +222,13 @@ public class TestSCMContainerManager {
xceiverClientManager.getFactor(),
containerOwner);
containerManager.updateContainerState(containerInfo.getContainerInfo()
.getContainerID(), HddsProtos.LifeCycleEvent.CREATE);
.containerID(), HddsProtos.LifeCycleEvent.CREATE);
Thread.sleep(TIMEOUT + 1000);
NavigableSet<ContainerID> deleteContainers = containerManager
.getStateManager().getMatchingContainerIDs("OZONE",
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.DELETING);
Assert.assertTrue(deleteContainers
.contains(containerInfo.getContainerInfo().containerID()));
thrown.expect(IOException.class);
thrown.expectMessage("Lease Exception");
containerManager
.updateContainerState(containerInfo.getContainerInfo().getContainerID(),
.updateContainerState(containerInfo.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATED);
}
@ -257,26 +258,24 @@ public class TestSCMContainerManager {
crBuilder.addAllReports(reports);
containerManager.processContainerReports(
datanodeDetails, crBuilder.build(), false);
datanodeDetails, crBuilder.build());
ContainerInfo updatedContainer =
containerManager.getContainer(info.getContainerID());
containerManager.getContainer(info.containerID());
Assert.assertEquals(100000000L,
updatedContainer.getNumberOfKeys());
Assert.assertEquals(2000000000L, updatedContainer.getUsedBytes());
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
LambdaTestUtils.intercept(SCMException.class, "No entry "
+ "exist for containerId:", () -> containerManager.getStateManager()
.getContainerReplicas(ContainerID.valueof(c.getContainerID())));
Assert.assertEquals(containerManager.getContainerReplicas(
ContainerID.valueof(c.getContainerID())).size(), 1);
}
containerManager.processContainerReports(TestUtils.randomDatanodeDetails(),
crBuilder.build(), true);
crBuilder.build());
for (StorageContainerDatanodeProtocolProtos.ContainerInfo c : reports) {
Assert.assertTrue(containerManager.getStateManager()
.getContainerReplicas(
ContainerID.valueof(c.getContainerID())).size() > 0);
Assert.assertEquals(containerManager.getContainerReplicas(
ContainerID.valueof(c.getContainerID())).size(), 2);
}
}
@ -314,9 +313,10 @@ public class TestSCMContainerManager {
crBuilder.addAllReports(reports);
containerManager.processContainerReports(
datanodeDetails, crBuilder.build(), false);
datanodeDetails, crBuilder.build());
List<ContainerInfo> list = containerManager.listContainer(0, 50);
List<ContainerInfo> list = containerManager.listContainer(
ContainerID.valueof(1), 50);
Assert.assertEquals(2, list.stream().filter(
x -> x.getContainerID() == cID1 || x.getContainerID() == cID2).count());
Assert.assertEquals(300000000L, list.stream().filter(
@ -329,23 +329,13 @@ public class TestSCMContainerManager {
@Test
public void testCloseContainer() throws IOException {
ContainerInfo info = createContainer();
containerManager.updateContainerState(info.getContainerID(),
ContainerID id = createContainer().containerID();
containerManager.updateContainerState(id,
HddsProtos.LifeCycleEvent.FINALIZE);
NavigableSet<ContainerID> pendingCloseContainers = containerManager
.getStateManager().getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.CLOSING);
Assert.assertTrue(pendingCloseContainers.contains(info.containerID()));
containerManager.updateContainerState(info.getContainerID(),
containerManager.updateContainerState(id,
HddsProtos.LifeCycleEvent.CLOSE);
NavigableSet<ContainerID> closeContainers = containerManager
.getStateManager().getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(),
xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.CLOSED);
Assert.assertTrue(closeContainers.contains(info.containerID()));
ContainerInfo closedContainer = containerManager.getContainer(id);
Assert.assertEquals(LifeCycleState.CLOSED, closedContainer.getState());
}
/**
@ -359,20 +349,11 @@ public class TestSCMContainerManager {
.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
containerManager.updateContainerState(containerInfo.getContainerID(),
containerManager.updateContainerState(containerInfo.containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(containerInfo.getContainerID(),
containerManager.updateContainerState(containerInfo.containerID(),
HddsProtos.LifeCycleEvent.CREATED);
return containerInfo;
}
@Test
public void testFlushAllContainers() throws IOException {
ContainerInfo info = createContainer();
List<ContainerInfo> containers = containerManager.getStateManager()
.getAllContainers();
Assert.assertTrue(containers.size() > 0);
containerManager.flushContainerInfo();
}
}

View File

@ -23,6 +23,7 @@ import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.stream.IntStream;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@ -31,8 +32,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ReplicateContainerCommandProto;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
@ -62,10 +64,11 @@ public class TestReplicationManager {
private List<CommandForDatanode<ReplicateContainerCommandProto>> copyEvents;
private ContainerStateManager containerStateManager;
private ContainerManager containerManager;
private ContainerPlacementPolicy containerPlacementPolicy;
private List<DatanodeDetails> listOfDatanodeDetails;
private List<ContainerReplica> listOfContainerReplica;
private LeaseManager<Long> leaseManager;
private ReplicationManager replicationManager;
@ -73,33 +76,36 @@ public class TestReplicationManager {
public void initReplicationManager() throws IOException {
listOfDatanodeDetails = new ArrayList<>();
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
listOfDatanodeDetails.add(TestUtils.randomDatanodeDetails());
listOfContainerReplica = new ArrayList<>();
IntStream.range(1, 6).forEach(i -> {
DatanodeDetails dd = TestUtils.randomDatanodeDetails();
listOfDatanodeDetails.add(dd);
listOfContainerReplica.add(ContainerReplica.newBuilder()
.setContainerID(ContainerID.valueof(i))
.setDatanodeDetails(dd).build());
});
containerPlacementPolicy =
(excludedNodes, nodesRequired, sizeRequired) -> listOfDatanodeDetails
.subList(2, 2 + nodesRequired);
containerStateManager = Mockito.mock(ContainerStateManager.class);
containerManager = Mockito.mock(ContainerManager.class);
ContainerInfo containerInfo = new ContainerInfo.Builder()
.setState(LifeCycleState.CLOSED)
.build();
when(containerStateManager.getContainer(anyObject()))
when(containerManager.getContainer(anyObject()))
.thenReturn(containerInfo);
when(containerStateManager.getContainerReplicas(new ContainerID(1L)))
when(containerManager.getContainerReplicas(new ContainerID(1L)))
.thenReturn(new HashSet<>(Arrays.asList(
listOfDatanodeDetails.get(0),
listOfDatanodeDetails.get(1)
listOfContainerReplica.get(0),
listOfContainerReplica.get(1)
)));
when(containerStateManager.getContainerReplicas(new ContainerID(3L)))
when(containerManager.getContainerReplicas(new ContainerID(3L)))
.thenReturn(new HashSet<>());
queue = new EventQueue();
@ -115,7 +121,7 @@ public class TestReplicationManager {
leaseManager = new LeaseManager<>("Test", 100000L);
replicationManager = new ReplicationManager(containerPlacementPolicy,
containerStateManager, queue, leaseManager);
containerManager, queue, leaseManager);
@ -182,7 +188,7 @@ public class TestReplicationManager {
new LeaseManager<>("Test", 1000L);
replicationManager = new ReplicationManager(containerPlacementPolicy,
containerStateManager, queue, rapidLeaseManager);
containerManager, queue, rapidLeaseManager);
try {
rapidLeaseManager.start();

View File

@ -102,7 +102,7 @@ public class TestContainerPlacement {
EventQueue eventQueue = new EventQueue();
final int cacheSize = config.getInt(OZONE_SCM_DB_CACHE_SIZE_MB,
OZONE_SCM_DB_CACHE_SIZE_DEFAULT);
return new SCMContainerManager(config, scmNodeManager, cacheSize,
return new SCMContainerManager(config, scmNodeManager,
eventQueue);
}

View File

@ -18,38 +18,40 @@
package org.apache.hadoop.hdds.scm.node;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.ContainerStateManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.ContainerReplica;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import org.mockito.ArgumentCaptor;
import static org.mockito.Matchers.eq;
import org.mockito.Mockito;
/**
@ -57,86 +59,122 @@ import org.mockito.Mockito;
*/
public class TestDeadNodeHandler {
private List<ReplicationRequest> sentEvents = new ArrayList<>();
private SCMNodeManager nodeManager;
private ContainerStateManager containerStateManager;
private ContainerManager containerManager;
private NodeReportHandler nodeReportHandler;
private DeadNodeHandler deadNodeHandler;
private EventPublisher publisher;
private EventQueue eventQueue;
private String storageDir;
@Before
public void setup() throws IOException {
OzoneConfiguration conf = new OzoneConfiguration();
containerStateManager = new ContainerStateManager(conf,
Mockito.mock(ContainerManager.class),
Mockito.mock(PipelineSelector.class));
storageDir = GenericTestUtils.getTempPath(
TestDeadNodeHandler.class.getSimpleName() + UUID.randomUUID());
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, storageDir);
eventQueue = new EventQueue();
nodeManager = new SCMNodeManager(conf, "cluster1", null, eventQueue);
deadNodeHandler = new DeadNodeHandler(nodeManager,
containerStateManager);
containerManager = new SCMContainerManager(conf, nodeManager, eventQueue);
deadNodeHandler = new DeadNodeHandler(nodeManager, containerManager);
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
publisher = Mockito.mock(EventPublisher.class);
nodeReportHandler = new NodeReportHandler(nodeManager);
}
@After
public void teardown() {
FileUtil.fullyDelete(new File(storageDir));
}
@Test
public void testOnMessage() throws IOException {
//GIVEN
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
DatanodeDetails datanode3 = TestUtils.randomDatanodeDetails();
String storagePath = GenericTestUtils.getRandomizedTempPath()
.concat("/" + datanode1.getUuidString());
StorageReportProto storageOne = TestUtils.createStorageReport(
datanode1.getUuid(), storagePath, 100, 10, 90, null);
// Standalone pipeline now excludes the nodes which are already used,
// is the a proper behavior. Adding 9 datanodes for now to make the
// test case happy.
nodeManager.register(datanode1,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(datanode2,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(datanode3,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(TestUtils.randomDatanodeDetails(),
TestUtils.createNodeReport(storageOne), null);
ContainerInfo container1 =
TestUtils.allocateContainer(containerStateManager);
TestUtils.allocateContainer(containerManager);
ContainerInfo container2 =
TestUtils.allocateContainer(containerStateManager);
TestUtils.allocateContainer(containerManager);
ContainerInfo container3 =
TestUtils.allocateContainer(containerStateManager);
TestUtils.allocateContainer(containerManager);
containerManager.updateContainerState(
container1.containerID(), HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(
container1.containerID(), HddsProtos.LifeCycleEvent.CREATED);
containerManager.updateContainerState(
container2.containerID(), HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(
container2.containerID(), HddsProtos.LifeCycleEvent.CREATED);
containerManager.updateContainerState(
container3.containerID(), HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(
container3.containerID(), HddsProtos.LifeCycleEvent.CREATED);
registerReplicas(datanode1, container1, container2);
registerReplicas(datanode2, container1, container3);
registerReplicas(containerStateManager, container1, datanode1, datanode2);
registerReplicas(containerStateManager, container2, datanode1);
registerReplicas(containerStateManager, container3, datanode2);
registerReplicas(containerManager, container1, datanode1, datanode2);
registerReplicas(containerManager, container2, datanode1);
registerReplicas(containerManager, container3, datanode2);
TestUtils.closeContainer(containerStateManager, container1);
TestUtils.closeContainer(containerManager, container1.containerID());
TestUtils.closeContainer(containerManager, container2.containerID());
TestUtils.closeContainer(containerManager, container3.containerID());
deadNodeHandler.onMessage(datanode1, publisher);
Set<DatanodeDetails> container1Replicas =
containerStateManager.getContainerStateMap()
.getContainerReplicas(new ContainerID(container1.getContainerID()));
Set<ContainerReplica> container1Replicas = containerManager
.getContainerReplicas(new ContainerID(container1.getContainerID()));
Assert.assertEquals(1, container1Replicas.size());
Assert.assertEquals(datanode2, container1Replicas.iterator().next());
Assert.assertEquals(datanode2,
container1Replicas.iterator().next().getDatanodeDetails());
Set<DatanodeDetails> container2Replicas =
containerStateManager.getContainerStateMap()
.getContainerReplicas(new ContainerID(container2.getContainerID()));
Set<ContainerReplica> container2Replicas = containerManager
.getContainerReplicas(new ContainerID(container2.getContainerID()));
Assert.assertEquals(0, container2Replicas.size());
Set<DatanodeDetails> container3Replicas =
containerStateManager.getContainerStateMap()
Set<ContainerReplica> container3Replicas = containerManager
.getContainerReplicas(new ContainerID(container3.getContainerID()));
Assert.assertEquals(1, container3Replicas.size());
Assert.assertEquals(datanode2, container3Replicas.iterator().next());
ArgumentCaptor<ReplicationRequest> replicationRequestParameter =
ArgumentCaptor.forClass(ReplicationRequest.class);
Mockito.verify(publisher)
.fireEvent(eq(SCMEvents.REPLICATE_CONTAINER),
replicationRequestParameter.capture());
Assert
.assertEquals(container1.getContainerID(),
replicationRequestParameter.getValue().getContainerId());
Assert
.assertEquals(1,
replicationRequestParameter.getValue().getReplicationCount());
Assert
.assertEquals(3,
replicationRequestParameter.getValue().getExpecReplicationCount());
Assert.assertEquals(datanode2,
container3Replicas.iterator().next().getDatanodeDetails());
}
@Test
@ -144,6 +182,7 @@ public class TestDeadNodeHandler {
//GIVEN
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
String storagePath1 = GenericTestUtils.getRandomizedTempPath()
.concat("/" + datanode1.getUuidString());
String storagePath2 = GenericTestUtils.getRandomizedTempPath()
@ -153,15 +192,17 @@ public class TestDeadNodeHandler {
datanode1.getUuid(), storagePath1, 100, 10, 90, null);
StorageReportProto storageTwo = TestUtils.createStorageReport(
datanode2.getUuid(), storagePath2, 200, 20, 180, null);
nodeManager.register(datanode1,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(datanode2,
TestUtils.createNodeReport(storageTwo), null);
nodeReportHandler.onMessage(getNodeReport(datanode1, storageOne),
Mockito.mock(EventPublisher.class));
nodeReportHandler.onMessage(getNodeReport(datanode2, storageTwo),
Mockito.mock(EventPublisher.class));
ContainerInfo container1 =
TestUtils.allocateContainer(containerStateManager);
registerReplicas(datanode1, container1);
SCMNodeStat stat = nodeManager.getStats();
Assert.assertTrue(stat.getCapacity().get() == 300);
Assert.assertTrue(stat.getRemaining().get() == 270);
@ -190,32 +231,56 @@ public class TestDeadNodeHandler {
@Test
public void testOnMessageReplicaFailure() throws Exception {
DatanodeDetails datanode1 = TestUtils.randomDatanodeDetails();
DatanodeDetails datanode2 = TestUtils.randomDatanodeDetails();
DatanodeDetails datanode3 = TestUtils.randomDatanodeDetails();
String storagePath = GenericTestUtils.getRandomizedTempPath()
.concat("/" + datanode1.getUuidString());
StorageReportProto storageOne = TestUtils.createStorageReport(
datanode1.getUuid(), storagePath, 100, 10, 90, null);
nodeManager.register(datanode1,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(datanode2,
TestUtils.createNodeReport(storageOne), null);
nodeManager.register(datanode3,
TestUtils.createNodeReport(storageOne), null);
DatanodeDetails dn1 = TestUtils.randomDatanodeDetails();
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(DeadNodeHandler.getLogger());
String storagePath1 = GenericTestUtils.getRandomizedTempPath()
.concat("/" + dn1.getUuidString());
StorageReportProto storageOne = TestUtils.createStorageReport(
dn1.getUuid(), storagePath1, 100, 10, 90, null);
nodeReportHandler.onMessage(getNodeReport(dn1, storageOne),
Mockito.mock(EventPublisher.class));
ContainerInfo container1 =
TestUtils.allocateContainer(containerStateManager);
TestUtils.allocateContainer(containerManager);
containerManager.updateContainerState(
container1.containerID(), HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(
container1.containerID(), HddsProtos.LifeCycleEvent.CREATED);
TestUtils.closeContainer(containerManager, container1.containerID());
registerReplicas(dn1, container1);
deadNodeHandler.onMessage(dn1, eventQueue);
Assert.assertTrue(logCapturer.getOutput().contains(
"DataNode " + dn1.getUuid() + " doesn't have replica for container "
+ container1.getContainerID()));
"Exception while removing container replica "));
}
private void registerReplicas(ContainerStateManager csm,
ContainerInfo container, DatanodeDetails... datanodes) {
csm.getContainerStateMap()
.addContainerReplica(new ContainerID(container.getContainerID()),
datanodes);
private void registerReplicas(ContainerManager containerManager,
ContainerInfo container, DatanodeDetails... datanodes)
throws ContainerNotFoundException {
for (DatanodeDetails datanode : datanodes) {
containerManager.updateContainerReplica(
new ContainerID(container.getContainerID()),
ContainerReplica.newBuilder()
.setContainerID(container.containerID())
.setDatanodeDetails(datanode).build());
}
}
private void registerReplicas(DatanodeDetails datanode,

View File

@ -24,7 +24,7 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.HddsTestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.test.GenericTestUtils;

View File

@ -28,7 +28,7 @@ import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler
import org.apache.hadoop.hdds.scm.container.CloseContainerWatcher;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
@ -274,10 +274,12 @@ public class TestCloseContainerWatcher implements EventHandler<ContainerID> {
throws IOException {
ContainerInfo containerInfo = Mockito.mock(ContainerInfo.class);
ContainerInfo containerInfo2 = Mockito.mock(ContainerInfo.class);
when(containerManager.getContainer(id1)).thenReturn(containerInfo);
when(containerManager.getContainer(id2)).thenReturn(containerInfo2);
when(containerInfo.isContainerOpen()).thenReturn(true);
when(containerInfo2.isContainerOpen()).thenReturn(isOpen);
when(containerManager.getContainer(ContainerID.valueof(id1)))
.thenReturn(containerInfo);
when(containerManager.getContainer(ContainerID.valueof(id2)))
.thenReturn(containerInfo2);
when(containerInfo.isOpen()).thenReturn(true);
when(containerInfo2.isOpen()).thenReturn(isOpen);
}
@Override

View File

@ -34,7 +34,7 @@ import org.apache.hadoop.hdds.scm.cli.container.InfoSubcommand;
import org.apache.hadoop.hdds.scm.cli.container.ListSubcommand;
import org.apache.hadoop.hdds.scm.client.ContainerOperationClient;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolPB;

View File

@ -24,7 +24,7 @@ import java.util.concurrent.Callable;
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
import org.apache.hadoop.hdds.scm.cli.SCMCLI;
import org.apache.hadoop.hdds.scm.client.ScmClient;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.ozone.web.utils.JsonUtils;
import org.slf4j.Logger;
@ -52,7 +52,7 @@ public class ListSubcommand implements Callable<Void> {
@Option(names = {"-s", "--start"},
description = "Container id to start the iteration", required = true)
private long startId;
private long startId = 1;
@Option(names = {"-c", "--count"},
description = "Maximum number of containers to list",

View File

@ -21,7 +21,7 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.fs.FSExceptionMessages;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.io.retry.RetryPolicy;

View File

@ -16,7 +16,6 @@
*/
package org.apache.hadoop.hdds.scm.container;
import com.google.common.primitives.Longs;
import java.util.Set;
import java.util.UUID;
import org.apache.commons.lang3.RandomUtils;
@ -24,26 +23,22 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.test.LambdaTestUtils;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.NavigableSet;
import java.util.Random;
import java.util.concurrent.TimeoutException;
import org.slf4j.event.Level;
/**
@ -57,7 +52,6 @@ public class TestContainerStateManagerIntegration {
private StorageContainerManager scm;
private ContainerManager containerManager;
private ContainerStateManager containerStateManager;
private PipelineSelector selector;
private String containerOwner = "OZONE";
@ -70,8 +64,8 @@ public class TestContainerStateManagerIntegration {
xceiverClientManager = new XceiverClientManager(conf);
scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
containerStateManager = containerManager.getStateManager();
selector = containerManager.getPipelineSelector();
containerStateManager = ((SCMContainerManager)containerManager)
.getContainerStateManager();
}
@After
@ -88,13 +82,13 @@ public class TestContainerStateManagerIntegration {
.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
ContainerStateManager stateManager = new ContainerStateManager(conf);
ContainerInfo info = containerStateManager
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.ALLOCATED);
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
info.getContainerID());
Assert.assertEquals(OzoneConsts.GB * 3, info.getAllocatedBytes());
Assert.assertEquals(containerOwner, info.getOwner());
Assert.assertEquals(xceiverClientManager.getType(),
info.getReplicationType());
@ -117,35 +111,49 @@ public class TestContainerStateManagerIntegration {
}
@Test
public void testContainerStateManagerRestart() throws IOException {
public void testContainerStateManagerRestart()
throws IOException, TimeoutException, InterruptedException {
// Allocate 5 containers in ALLOCATED state and 5 in CREATING state
List<ContainerInfo> containers = new ArrayList<>();
for (int i = 0; i < 10; i++) {
ContainerWithPipeline container = scm.getClientProtocolServer()
.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
containers.add(container.getContainerInfo());
if (i >= 5) {
scm.getContainerManager().updateContainerState(container
.getContainerInfo().getContainerID(),
.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
}
}
// New instance of ContainerStateManager should load all the containers in
// container store.
ContainerStateManager stateManager =
new ContainerStateManager(conf, containerManager, selector);
int matchCount = stateManager
.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.ALLOCATED).size();
cluster.restartStorageContainerManager();
List<ContainerInfo> result = cluster.getStorageContainerManager()
.getContainerManager().listContainer(null, 100);
long matchCount = result.stream()
.filter(info ->
info.getOwner().equals(containerOwner))
.filter(info ->
info.getReplicationType() == xceiverClientManager.getType())
.filter(info ->
info.getReplicationFactor() == xceiverClientManager.getFactor())
.filter(info ->
info.getState() == HddsProtos.LifeCycleState.ALLOCATED)
.count();
Assert.assertEquals(5, matchCount);
matchCount = stateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.CREATING).size();
matchCount = result.stream()
.filter(info ->
info.getOwner().equals(containerOwner))
.filter(info ->
info.getReplicationType() == xceiverClientManager.getType())
.filter(info ->
info.getReplicationFactor() == xceiverClientManager.getFactor())
.filter(info ->
info.getState() == HddsProtos.LifeCycleState.CREATING)
.count();
Assert.assertEquals(5, matchCount);
}
@ -155,10 +163,10 @@ public class TestContainerStateManagerIntegration {
allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATED);
ContainerWithPipeline container2 = scm.getClientProtocolServer().
@ -176,23 +184,24 @@ public class TestContainerStateManagerIntegration {
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.ALLOCATED);
// space has already been allocated in container1, now container 2 should
// be chosen.
Assert.assertEquals(container2.getContainerInfo().getContainerID(),
info.getContainerID());
containerManager
.updateContainerState(container2.getContainerInfo().getContainerID(),
.updateContainerState(container2.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager
.updateContainerState(container2.getContainerInfo().getContainerID(),
.updateContainerState(container2.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATED);
// space has already been allocated in container1, now container 2 should
// be chosen.
// now we have to get container1
info = containerStateManager
.getMatchingContainer(OzoneConsts.GB * 3, containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.OPEN);
Assert.assertEquals(container2.getContainerInfo().getContainerID(),
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
info.getContainerID());
}
@ -217,7 +226,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -225,7 +234,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATED);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -233,7 +242,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.FINALIZE);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -241,7 +250,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CLOSE);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -249,7 +258,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.DELETE);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -257,7 +266,7 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
containerManager
.updateContainerState(container1.getContainerInfo().getContainerID(),
.updateContainerState(container1.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CLEANUP);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -271,10 +280,10 @@ public class TestContainerStateManagerIntegration {
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
containerManager
.updateContainerState(container2.getContainerInfo().getContainerID(),
.updateContainerState(container2.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager
.updateContainerState(container2.getContainerInfo().getContainerID(),
.updateContainerState(container2.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.TIMEOUT);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -288,16 +297,16 @@ public class TestContainerStateManagerIntegration {
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
containerManager
.updateContainerState(container3.getContainerInfo().getContainerID(),
.updateContainerState(container3.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager
.updateContainerState(container3.getContainerInfo().getContainerID(),
.updateContainerState(container3.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CREATED);
containerManager
.updateContainerState(container3.getContainerInfo().getContainerID(),
.updateContainerState(container3.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.FINALIZE);
containerManager
.updateContainerState(container3.getContainerInfo().getContainerID(),
.updateContainerState(container3.getContainerInfo().containerID(),
HddsProtos.LifeCycleEvent.CLOSE);
containers = containerStateManager.getMatchingContainerIDs(containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
@ -305,46 +314,6 @@ public class TestContainerStateManagerIntegration {
Assert.assertEquals(1, containers);
}
@Test
public void testUpdatingAllocatedBytes() throws Exception {
ContainerWithPipeline container1 = scm.getClientProtocolServer()
.allocateContainer(xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
containerManager.updateContainerState(container1
.getContainerInfo().getContainerID(),
HddsProtos.LifeCycleEvent.CREATE);
containerManager.updateContainerState(container1
.getContainerInfo().getContainerID(),
HddsProtos.LifeCycleEvent.CREATED);
Random ran = new Random();
long allocatedSize = 0;
for (int i = 0; i<5; i++) {
long size = Math.abs(ran.nextLong() % OzoneConsts.GB);
allocatedSize += size;
// trigger allocating bytes by calling getMatchingContainer
ContainerInfo info = containerStateManager
.getMatchingContainer(size, containerOwner,
xceiverClientManager.getType(), xceiverClientManager.getFactor(),
HddsProtos.LifeCycleState.OPEN);
Assert.assertEquals(container1.getContainerInfo().getContainerID(),
info.getContainerID());
SCMContainerManager containerMapping =
(SCMContainerManager) containerManager;
// manually trigger a flush, this will persist the allocated bytes value
// to disk
containerMapping.flushContainerInfo();
// the persisted value should always be equal to allocated size.
byte[] containerBytes = containerMapping.getContainerStore().get(
Longs.toByteArray(container1.getContainerInfo().getContainerID()));
HddsProtos.SCMContainerInfo infoProto =
HddsProtos.SCMContainerInfo.PARSER.parseFrom(containerBytes);
ContainerInfo currentInfo = ContainerInfo.fromProtobuf(infoProto);
Assert.assertEquals(allocatedSize, currentInfo.getAllocatedBytes());
}
}
@Test
public void testReplicaMap() throws Exception {
@ -360,59 +329,71 @@ public class TestContainerStateManagerIntegration {
// Test 1: no replica's exist
ContainerID containerID = ContainerID.valueof(RandomUtils.nextLong());
Set<DatanodeDetails> replicaSet;
LambdaTestUtils.intercept(SCMException.class, "", () -> {
Set<ContainerReplica> replicaSet;
try {
containerStateManager.getContainerReplicas(containerID);
});
Assert.fail();
} catch (ContainerNotFoundException ex) {
// expected.
}
ContainerWithPipeline container = scm.getClientProtocolServer()
.allocateContainer(
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), containerOwner);
ContainerID id = container.getContainerInfo().containerID();
// Test 2: Add replica nodes and then test
containerStateManager.addContainerReplica(containerID, dn1);
containerStateManager.addContainerReplica(containerID, dn2);
replicaSet = containerStateManager.getContainerReplicas(containerID);
ContainerReplica replicaOne = ContainerReplica.newBuilder()
.setContainerID(id)
.setDatanodeDetails(dn1)
.build();
ContainerReplica replicaTwo = ContainerReplica.newBuilder()
.setContainerID(id)
.setDatanodeDetails(dn2)
.build();
containerStateManager.updateContainerReplica(id, replicaOne);
containerStateManager.updateContainerReplica(id, replicaTwo);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(2, replicaSet.size());
Assert.assertTrue(replicaSet.contains(dn1));
Assert.assertTrue(replicaSet.contains(dn2));
Assert.assertTrue(replicaSet.contains(replicaOne));
Assert.assertTrue(replicaSet.contains(replicaTwo));
// Test 3: Remove one replica node and then test
containerStateManager.removeContainerReplica(containerID, dn1);
replicaSet = containerStateManager.getContainerReplicas(containerID);
containerStateManager.removeContainerReplica(id, replicaOne);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(1, replicaSet.size());
Assert.assertFalse(replicaSet.contains(dn1));
Assert.assertTrue(replicaSet.contains(dn2));
Assert.assertFalse(replicaSet.contains(replicaOne));
Assert.assertTrue(replicaSet.contains(replicaTwo));
// Test 3: Remove second replica node and then test
containerStateManager.removeContainerReplica(containerID, dn2);
replicaSet = containerStateManager.getContainerReplicas(containerID);
containerStateManager.removeContainerReplica(id, replicaTwo);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(0, replicaSet.size());
Assert.assertFalse(replicaSet.contains(dn1));
Assert.assertFalse(replicaSet.contains(dn2));
Assert.assertFalse(replicaSet.contains(replicaOne));
Assert.assertFalse(replicaSet.contains(replicaTwo));
// Test 4: Re-insert dn1
containerStateManager.addContainerReplica(containerID, dn1);
replicaSet = containerStateManager.getContainerReplicas(containerID);
containerStateManager.updateContainerReplica(id, replicaOne);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(1, replicaSet.size());
Assert.assertTrue(replicaSet.contains(dn1));
Assert.assertFalse(replicaSet.contains(dn2));
Assert.assertTrue(replicaSet.contains(replicaOne));
Assert.assertFalse(replicaSet.contains(replicaTwo));
// Re-insert dn2
containerStateManager.addContainerReplica(containerID, dn2);
replicaSet = containerStateManager.getContainerReplicas(containerID);
containerStateManager.updateContainerReplica(id, replicaTwo);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(2, replicaSet.size());
Assert.assertTrue(replicaSet.contains(dn1));
Assert.assertTrue(replicaSet.contains(dn2));
Assert.assertTrue(replicaSet.contains(replicaOne));
Assert.assertTrue(replicaSet.contains(replicaTwo));
Assert.assertFalse(logCapturer.getOutput().contains(
"ReplicaMap already contains entry for container Id: " + containerID
.toString() + ",DataNode: " + dn1.toString()));
// Re-insert dn1
containerStateManager.addContainerReplica(containerID, dn1);
replicaSet = containerStateManager.getContainerReplicas(containerID);
containerStateManager.updateContainerReplica(id, replicaOne);
replicaSet = containerStateManager.getContainerReplicas(id);
Assert.assertEquals(2, replicaSet.size());
Assert.assertTrue(replicaSet.contains(dn1));
Assert.assertTrue(replicaSet.contains(dn2));
Assert.assertTrue(logCapturer.getOutput().contains(
"ReplicaMap already contains entry for container Id: " + containerID
.toString() + ",DataNode: " + dn1.toString()));
Assert.assertTrue(replicaSet.contains(replicaOne));
Assert.assertTrue(replicaSet.contains(replicaTwo));
}
}

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
@ -50,7 +49,6 @@ public class TestNode2PipelineMap {
private static OzoneConfiguration conf;
private static StorageContainerManager scm;
private static ContainerWithPipeline ratisContainer;
private static ContainerStateMap stateMap;
private static ContainerManager containerManager;
private static PipelineSelector pipelineSelector;
@ -66,7 +64,6 @@ public class TestNode2PipelineMap {
cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
stateMap = containerManager.getStateManager().getContainerStateMap();
ratisContainer = containerManager.allocateContainer(
RATIS, THREE, "testOwner");
pipelineSelector = containerManager.getPipelineSelector();
@ -89,10 +86,10 @@ public class TestNode2PipelineMap {
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
ratisContainer.getPipeline().getId());
long cId = ratisContainer.getContainerInfo().getContainerID();
ContainerID cId = ratisContainer.getContainerInfo().containerID();
Assert.assertEquals(1, set.size());
set.forEach(containerID ->
Assert.assertEquals(containerID, ContainerID.valueof(cId)));
Assert.assertEquals(containerID, cId));
List<DatanodeDetails> dns = ratisContainer.getPipeline().getMachines();
Assert.assertEquals(3, dns.size());

View File

@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -50,7 +49,6 @@ public class TestPipelineClose {
private static StorageContainerManager scm;
private static ContainerWithPipeline ratisContainer1;
private static ContainerWithPipeline ratisContainer2;
private static ContainerStateMap stateMap;
private static ContainerManager containerManager;
private static PipelineSelector pipelineSelector;
@ -66,7 +64,6 @@ public class TestPipelineClose {
cluster.waitForClusterToBeReady();
scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
stateMap = containerManager.getStateManager().getContainerStateMap();
ratisContainer1 = containerManager
.allocateContainer(RATIS, THREE, "testOwner");
ratisContainer2 = containerManager
@ -93,10 +90,9 @@ public class TestPipelineClose {
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
ratisContainer1.getPipeline().getId());
long cId = ratisContainer1.getContainerInfo().getContainerID();
ContainerID cId = ratisContainer1.getContainerInfo().containerID();
Assert.assertEquals(1, set.size());
set.forEach(containerID ->
Assert.assertEquals(containerID, ContainerID.valueof(cId)));
set.forEach(containerID -> Assert.assertEquals(containerID, cId));
// Now close the container and it should not show up while fetching
// containers by pipeline
@ -133,7 +129,7 @@ public class TestPipelineClose {
ratisContainer2.getPipeline().getId());
Assert.assertEquals(1, setOpen.size());
long cId2 = ratisContainer2.getContainerInfo().getContainerID();
ContainerID cId2 = ratisContainer2.getContainerInfo().containerID();
containerManager
.updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
containerManager

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.ozone;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
@ -45,14 +46,15 @@ public class OzoneTestUtils {
return performOperationOnKeyContainers((blockID) -> {
try {
scm.getContainerManager()
.updateContainerState(blockID.getContainerID(),
.updateContainerState(ContainerID.valueof(blockID.getContainerID()),
HddsProtos.LifeCycleEvent.FINALIZE);
scm.getContainerManager()
.updateContainerState(blockID.getContainerID(),
.updateContainerState(ContainerID.valueof(blockID.getContainerID()),
HddsProtos.LifeCycleEvent.CLOSE);
Assert.assertFalse(scm.getContainerManager()
.getContainerWithPipeline(blockID.getContainerID())
.getContainerInfo().isContainerOpen());
.getContainerWithPipeline(ContainerID.valueof(
blockID.getContainerID()))
.getContainerInfo().isOpen());
} catch (IOException e) {
e.printStackTrace();
}

View File

@ -45,6 +45,7 @@ import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.block.DeletedBlockLog;
import org.apache.hadoop.hdds.scm.block.SCMBlockDeletingService;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.NodeManager;
@ -168,8 +169,7 @@ public class TestStorageContainerManager {
} else {
// If passes permission check, it should fail with
// key not exist exception.
Assert.assertTrue(e.getMessage()
.contains("Specified key does not exist"));
Assert.assertTrue(e instanceof ContainerNotFoundException);
}
}
} finally {

View File

@ -23,6 +23,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.ozone.client.*;
@ -445,7 +446,8 @@ public class TestOzoneRestClient {
// Sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the actually value size.
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);

View File

@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.hdds.scm.container.common.helpers.
StorageContainerException;
@ -302,7 +303,8 @@ public class TestCloseContainerHandlingByClient {
for (long containerID : containerIdList) {
Pipeline pipeline =
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(containerID).getPipeline();
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline();
pipelineList.add(pipeline);
List<DatanodeDetails> datanodes = pipeline.getMachines();
for (DatanodeDetails details : datanodes) {
@ -349,7 +351,8 @@ public class TestCloseContainerHandlingByClient {
long containerID = locationInfos.get(0).getContainerID();
List<DatanodeDetails> datanodes =
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(containerID).getPipeline().getMachines();
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline().getMachines();
Assert.assertEquals(1, datanodes.size());
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
dataString = fixedLengthString(keyString, (1 * blockSize));
@ -451,7 +454,8 @@ public class TestCloseContainerHandlingByClient {
long containerID = locationInfos.get(0).getContainerID();
List<DatanodeDetails> datanodes =
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(containerID).getPipeline().getMachines();
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline().getMachines();
Assert.assertEquals(1, datanodes.size());
// move the container on the datanode to Closing state, this will ensure
// closing the key will hit BLOCK_NOT_COMMITTED_EXCEPTION while trying

View File

@ -23,7 +23,8 @@ import org.apache.commons.lang3.RandomUtils;
import org.apache.hadoop.hdds.protocol.StorageType;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -642,7 +643,8 @@ public class TestOzoneRpcClient {
// Second, sum the data size from chunks in Container via containerID
// and localID, make sure the size equals to the size from keyDetails.
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);

View File

@ -242,8 +242,7 @@ public class TestBlockDeletion {
logCapturer.clearOutput();
scm.getContainerManager().processContainerReports(
cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport,
false);
cluster.getHddsDatanodes().get(0).getDatanodeDetails(), dummyReport);
// wait for event to be handled by event handler
Thread.sleep(1000);
String output = logCapturer.getOutput();

View File

@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
@ -102,7 +103,8 @@ public class TestCloseContainerByPipeline {
long containerID = omKeyLocationInfo.getContainerID();
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);
@ -157,7 +159,8 @@ public class TestCloseContainerByPipeline {
long containerID = omKeyLocationInfo.getContainerID();
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(datanodes.size(), 1);
@ -214,7 +217,8 @@ public class TestCloseContainerByPipeline {
long containerID = omKeyLocationInfo.getContainerID();
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
Assert.assertEquals(3, datanodes.size());

View File

@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.common.statemachine.commandhandler;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.ObjectStore;
@ -81,7 +82,8 @@ public class TestCloseContainerHandler {
long containerID = omKeyLocationInfo.getContainerID();
Pipeline pipeline = cluster.getStorageContainerManager()
.getContainerManager().getContainerWithPipeline(containerID)
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
Assert.assertFalse(isContainerClosed(cluster, containerID));

View File

@ -22,7 +22,7 @@ import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;

View File

@ -30,7 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.container.SCMContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB;
@ -127,8 +127,7 @@ public class TestScmChillMode {
new TestStorageContainerManagerHelper(cluster, conf);
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100, 4096);
final List<ContainerInfo> containers = cluster
.getStorageContainerManager()
.getContainerManager().getStateManager().getAllContainers();
.getStorageContainerManager().getContainerManager().getContainers();
GenericTestUtils.waitFor(() -> {
return containers.size() > 10;
}, 100, 1000);
@ -251,8 +250,7 @@ public class TestScmChillMode {
new TestStorageContainerManagerHelper(miniCluster, conf);
Map<String, OmKeyInfo> keyLocations = helper.createKeys(100 * 2, 4096);
final List<ContainerInfo> containers = miniCluster
.getStorageContainerManager().getContainerManager()
.getStateManager().getAllContainers();
.getStorageContainerManager().getContainerManager().getContainers();
GenericTestUtils.waitFor(() -> {
return containers.size() > 10;
}, 100, 1000 * 2);
@ -268,9 +266,9 @@ public class TestScmChillMode {
.getStorageContainerManager().getContainerManager();
containers.forEach(c -> {
try {
mapping.updateContainerState(c.getContainerID(),
mapping.updateContainerState(c.containerID(),
HddsProtos.LifeCycleEvent.FINALIZE);
mapping.updateContainerState(c.getContainerID(),
mapping.updateContainerState(c.containerID(),
LifeCycleEvent.CLOSE);
} catch (IOException e) {
LOG.info("Failed to change state of open containers.", e);
@ -348,7 +346,7 @@ public class TestScmChillMode {
.getStorageContainerManager().getClientProtocolServer();
assertFalse((scm.getClientProtocolServer()).getChillModeStatus());
final List<ContainerInfo> containers = scm.getContainerManager()
.getStateManager().getAllContainers();
.getContainers();
scm.getEventQueue().fireEvent(SCMEvents.CHILL_MODE_STATUS, true);
GenericTestUtils.waitFor(() -> {
return clientProtocolServer.getChillModeStatus();

View File

@ -411,10 +411,6 @@ public class KeyManagerImpl implements KeyManager {
// A rename is a no-op if the target and source name is same.
// TODO: Discuss if we need to throw?.
// TODO: Define the semantics of rename more clearly. Today this code
// will allow rename of a Key across volumes. This should *not* be
// allowed. The documentation of Ozone says that rename is permitted only
// within a volume.
if (fromKeyName.equals(toKeyName)) {
return;
}

View File

@ -22,7 +22,7 @@ import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.container.states.ContainerStateMap;
@ -70,9 +70,6 @@ public class BenchMarkContainerStateMap {
.setPipelineID(pipeline.getId())
.setReplicationType(pipeline.getType())
.setReplicationFactor(pipeline.getFactor())
// This is bytes allocated for blocks inside container, not the
// container size
.setAllocatedBytes(0)
.setUsedBytes(0)
.setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
@ -93,9 +90,6 @@ public class BenchMarkContainerStateMap {
.setPipelineID(pipeline.getId())
.setReplicationType(pipeline.getType())
.setReplicationFactor(pipeline.getFactor())
// This is bytes allocated for blocks inside container, not the
// container size
.setAllocatedBytes(0)
.setUsedBytes(0)
.setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
@ -115,9 +109,6 @@ public class BenchMarkContainerStateMap {
.setPipelineID(pipeline.getId())
.setReplicationType(pipeline.getType())
.setReplicationFactor(pipeline.getFactor())
// This is bytes allocated for blocks inside container, not the
// container size
.setAllocatedBytes(0)
.setUsedBytes(0)
.setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())
@ -188,9 +179,6 @@ public class BenchMarkContainerStateMap {
.setPipelineID(pipeline.getId())
.setReplicationType(pipeline.getType())
.setReplicationFactor(pipeline.getFactor())
// This is bytes allocated for blocks inside container, not the
// container size
.setAllocatedBytes(0)
.setUsedBytes(0)
.setNumberOfKeys(0)
.setStateEnterTime(Time.monotonicNow())

View File

@ -36,7 +36,7 @@ import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.KeyInfo
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeInfo;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.VolumeList;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.util.Tool;
import org.apache.hadoop.util.ToolRunner;
import org.apache.hadoop.utils.MetadataStore;
@ -83,7 +83,6 @@ public class SQLCLI extends Configured implements Tool {
"replicationType TEXT NOT NULL," +
"replicationFactor TEXT NOT NULL," +
"usedBytes LONG NOT NULL," +
"allocatedBytes LONG NOT NULL," +
"owner TEXT," +
"numberOfKeys LONG)";
private static final String CREATE_DATANODE_INFO =
@ -94,8 +93,8 @@ public class SQLCLI extends Configured implements Tool {
"containerPort INTEGER NOT NULL);";
private static final String INSERT_CONTAINER_INFO =
"INSERT INTO containerInfo (containerID, replicationType, "
+ "replicationFactor, usedBytes, allocatedBytes, owner, "
+ "numberOfKeys) VALUES (\"%d\", \"%s\", \"%s\", \"%d\", \"%d\", "
+ "replicationFactor, usedBytes, owner, "
+ "numberOfKeys) VALUES (\"%d\", \"%s\", \"%s\", \"%d\", "
+ "\"%s\", \"%d\")";
private static final String INSERT_DATANODE_INFO =
"INSERT INTO datanodeInfo (hostname, datanodeUUid, ipAddress, " +
@ -498,7 +497,6 @@ public class SQLCLI extends Configured implements Tool {
containerInfo.getReplicationType(),
containerInfo.getReplicationFactor(),
containerInfo.getUsedBytes(),
containerInfo.getAllocatedBytes(),
containerInfo.getOwner(),
containerInfo.getNumberOfKeys());

View File

@ -120,8 +120,7 @@ public class TestContainerSQLCli {
cluster.getStorageContainerManager().stop();
eventQueue = new EventQueue();
nodeManager = cluster.getStorageContainerManager().getScmNodeManager();
containerManager = new SCMContainerManager(conf, nodeManager, 128,
eventQueue);
containerManager = new SCMContainerManager(conf, nodeManager, eventQueue);
blockManager = new BlockManagerImpl(
conf, nodeManager, containerManager, eventQueue);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);