HDDS-694. Plugin new Pipeline management code in SCM.

Contributed by Lokesh Jain.
This commit is contained in:
Nanda kumar 2018-10-26 17:53:47 +05:30
parent e28c00c290
commit dce4ebe814
102 changed files with 792 additions and 2364 deletions

View File

@ -29,7 +29,7 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.XceiverClientProtocolServiceGrpc.XceiverClientProtocolServiceStub;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.util.Time;
import org.apache.ratis.thirdparty.io.grpc.ManagedChannel;
@ -39,6 +39,7 @@
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ExecutionException;
import java.util.UUID;
@ -84,9 +85,9 @@ public XceiverClientGrpc(Pipeline pipeline, Configuration config) {
public void connect() throws Exception {
// leader by default is the 1st datanode in the datanode list of pipleline
DatanodeDetails leader = this.pipeline.getLeader();
DatanodeDetails dn = this.pipeline.getFirstNode();
// just make a connection to the 1st datanode at the beginning
connectToDatanode(leader);
connectToDatanode(dn);
}
private void connectToDatanode(DatanodeDetails dn) {
@ -148,18 +149,16 @@ public ContainerCommandResponseProto sendCommand(
public ContainerCommandResponseProto sendCommandWithRetry(
ContainerCommandRequestProto request) throws IOException {
int size = pipeline.getMachines().size();
ContainerCommandResponseProto responseProto = null;
DatanodeDetails dn = null;
// In case of an exception or an error, we will try to read from the
// datanodes in the pipeline in a round robin fashion.
// TODO: cache the correct leader info in here, so that any subsequent calls
// should first go to leader
for (int dnIndex = 0; dnIndex < size; dnIndex++) {
List<DatanodeDetails> dns = pipeline.getNodes();
for (DatanodeDetails dn : dns) {
try {
dn = pipeline.getMachines().get(dnIndex);
LOG.debug("Executing command " + request + " on datanode " + dn);
// In case the command gets retried on a 2nd datanode,
// sendCommandAsyncCall will create a new channel and async stub
@ -201,7 +200,7 @@ public ContainerCommandResponseProto sendCommandWithRetry(
public CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(
ContainerCommandRequestProto request)
throws IOException, ExecutionException, InterruptedException {
return sendCommandAsync(request, pipeline.getLeader());
return sendCommandAsync(request, pipeline.getFirstNode());
}
private CompletableFuture<ContainerCommandResponseProto> sendCommandAsync(

View File

@ -25,7 +25,7 @@
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.io.Closeable;
@ -115,8 +115,8 @@ public Cache<String, XceiverClientSpi> getClientCache() {
public XceiverClientSpi acquireClient(Pipeline pipeline)
throws IOException {
Preconditions.checkNotNull(pipeline);
Preconditions.checkArgument(pipeline.getMachines() != null);
Preconditions.checkArgument(!pipeline.getMachines().isEmpty());
Preconditions.checkArgument(pipeline.getNodes() != null);
Preconditions.checkArgument(!pipeline.getNodes().isEmpty());
synchronized (clientCache) {
XceiverClientSpi info = getClient(pipeline);

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.hdds.scm;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.io.MultipleIOException;
import org.apache.ratis.proto.RaftProtos;
import org.apache.ratis.retry.RetryPolicy;
@ -27,7 +26,7 @@
.InvalidProtocolBufferException;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto;
@ -63,19 +62,6 @@
public final class XceiverClientRatis extends XceiverClientSpi {
static final Logger LOG = LoggerFactory.getLogger(XceiverClientRatis.class);
public static XceiverClientRatis newXceiverClientRatis(
Pipeline pipeline, Configuration ozoneConf) {
final String rpcType = ozoneConf.get(
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
final int maxOutstandingRequests =
HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
return new XceiverClientRatis(pipeline,
SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests,
retryPolicy);
}
public static XceiverClientRatis newXceiverClientRatis(
org.apache.hadoop.hdds.scm.pipeline.Pipeline pipeline,
Configuration ozoneConf) {
@ -85,11 +71,7 @@ public static XceiverClientRatis newXceiverClientRatis(
final int maxOutstandingRequests =
HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
Pipeline pipeline1 =
new Pipeline(pipeline.getNodes().get(0).getUuidString(),
HddsProtos.LifeCycleState.OPEN, pipeline.getType(),
pipeline.getFactor(), PipelineID.valueOf(pipeline.getID().getId()));
return new XceiverClientRatis(pipeline1,
return new XceiverClientRatis(pipeline,
SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests,
retryPolicy);
}
@ -118,7 +100,7 @@ private XceiverClientRatis(Pipeline pipeline, RpcType rpcType,
public void createPipeline() throws IOException {
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
callRatisRpc(pipeline.getMachines(),
callRatisRpc(pipeline.getNodes(),
(raftClient, peer) -> raftClient.groupAdd(group, peer.getId()));
}
@ -128,7 +110,7 @@ public void createPipeline() throws IOException {
public void destroyPipeline() throws IOException {
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
LOG.debug("destroying pipeline:{} with {}", pipeline.getId(), group);
callRatisRpc(pipeline.getMachines(), (raftClient, peer) -> raftClient
callRatisRpc(pipeline.getNodes(), (raftClient, peer) -> raftClient
.groupRemove(group.getGroupId(), true, peer.getId()));
}
@ -174,9 +156,8 @@ public Pipeline getPipeline() {
@Override
public void connect() throws Exception {
LOG.debug("Connecting to pipeline:{} leader:{}",
getPipeline().getId(),
RatisHelper.toRaftPeerId(pipeline.getLeader()));
LOG.debug("Connecting to pipeline:{} datanode:{}", getPipeline().getId(),
RatisHelper.toRaftPeerId(pipeline.getFirstNode()));
// TODO : XceiverClient ratis should pass the config value of
// maxOutstandingRequests so as to set the upper bound on max no of async
// requests to be handled by raft client

View File

@ -22,7 +22,7 @@
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocolPB
.StorageContainerLocationProtocolClientSideTranslatorPB;
import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls;
@ -40,11 +40,6 @@
import java.util.List;
import java.util.UUID;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState
.ALLOCATED;
import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState
.OPEN;
/**
* This class provides the client-facing APIs of container operations.
*/
@ -98,14 +93,10 @@ public ContainerWithPipeline createContainer(String owner)
Pipeline pipeline = containerWithPipeline.getPipeline();
client = xceiverClientManager.acquireClient(pipeline);
// Allocated State means that SCM has allocated this pipeline in its
// namespace. The client needs to create the pipeline on the machines
// which was choosen by the SCM.
Preconditions.checkState(pipeline.getLifeCycleState() == ALLOCATED ||
pipeline.getLifeCycleState() == OPEN, "Unexpected pipeline state");
if (pipeline.getLifeCycleState() == ALLOCATED) {
createPipeline(client, pipeline);
}
Preconditions.checkState(pipeline.isOpen(), String
.format("Unexpected state=%s for pipeline=%s, expected state=%s",
pipeline.getPipelineState(), pipeline.getId(),
Pipeline.PipelineState.OPEN));
createContainer(client,
containerWithPipeline.getContainerInfo().getContainerID());
return containerWithPipeline;
@ -142,8 +133,7 @@ public void createContainer(XceiverClientSpi client,
// creation state.
if (LOG.isDebugEnabled()) {
LOG.debug("Created container " + containerId
+ " leader:" + client.getPipeline().getLeader()
+ " machines:" + client.getPipeline().getMachines());
+ " machines:" + client.getPipeline().getNodes());
}
}
@ -208,12 +198,6 @@ public ContainerWithPipeline createContainer(HddsProtos.ReplicationType type,
Pipeline pipeline = containerWithPipeline.getPipeline();
client = xceiverClientManager.acquireClient(pipeline);
// Allocated State means that SCM has allocated this pipeline in its
// namespace. The client needs to create the pipeline on the machines
// which was choosen by the SCM.
if (pipeline.getLifeCycleState() == ALLOCATED) {
createPipeline(client, pipeline);
}
// connect to pipeline leader and allocate container on leader datanode.
client = xceiverClientManager.acquireClient(pipeline);
createContainer(client,
@ -283,10 +267,8 @@ public void deleteContainer(long containerId, Pipeline pipeline,
storageContainerLocationClient
.deleteContainer(containerId);
if (LOG.isDebugEnabled()) {
LOG.debug("Deleted container {}, leader: {}, machines: {} ",
containerId,
pipeline.getLeader(),
pipeline.getMachines());
LOG.debug("Deleted container {}, machines: {} ", containerId,
pipeline.getNodes());
}
} finally {
if (client != null) {
@ -336,10 +318,8 @@ public ContainerData readContainer(long containerID,
ReadContainerResponseProto response =
ContainerProtocolCalls.readContainer(client, containerID, traceID);
if (LOG.isDebugEnabled()) {
LOG.debug("Read container {}, leader: {}, machines: {} ",
containerID,
pipeline.getLeader(),
pipeline.getMachines());
LOG.debug("Read container {}, machines: {} ", containerID,
pipeline.getNodes());
}
return response.getContainerData();
} finally {

View File

@ -19,7 +19,7 @@
package org.apache.hadoop.hdds.scm;
import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos

View File

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

View File

@ -36,7 +36,7 @@
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.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.util.Time;
/**

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.container.common.helpers;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.client.BlockID;
/**

View File

@ -23,6 +23,7 @@
import org.apache.commons.lang3.builder.HashCodeBuilder;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
/**
* Class wraps ozone container info.
@ -50,7 +51,7 @@ public static ContainerWithPipeline fromProtobuf(
HddsProtos.ContainerWithPipeline allocatedContainer) {
return new ContainerWithPipeline(
ContainerInfo.fromProtobuf(allocatedContainer.getContainerInfo()),
Pipeline.getFromProtoBuf(allocatedContainer.getPipeline()));
Pipeline.getFromProtobuf(allocatedContainer.getPipeline()));
}
public HddsProtos.ContainerWithPipeline getProtobuf() {

View File

@ -1,319 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.container.common.helpers;
import com.fasterxml.jackson.annotation.JsonAutoDetect;
import com.fasterxml.jackson.annotation.JsonFilter;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.PropertyAccessor;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.ObjectWriter;
import com.fasterxml.jackson.databind.ser.FilterProvider;
import com.fasterxml.jackson.databind.ser.impl.SimpleBeanPropertyFilter;
import com.fasterxml.jackson.databind.ser.impl.SimpleFilterProvider;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.List;
/**
* A pipeline represents the group of machines over which a container lives.
*/
public class Pipeline {
static final String PIPELINE_INFO = "PIPELINE_INFO_FILTER";
private static final ObjectWriter WRITER;
static {
ObjectMapper mapper = new ObjectMapper();
String[] ignorableFieldNames = {"leaderID", "datanodes"};
FilterProvider filters = new SimpleFilterProvider()
.addFilter(PIPELINE_INFO, SimpleBeanPropertyFilter
.serializeAllExcept(ignorableFieldNames));
mapper.setVisibility(PropertyAccessor.FIELD,
JsonAutoDetect.Visibility.ANY);
mapper.addMixIn(Object.class, MixIn.class);
WRITER = mapper.writer(filters);
}
@JsonIgnore
private String leaderID;
@JsonIgnore
private Map<String, DatanodeDetails> datanodes;
private HddsProtos.LifeCycleState lifeCycleState;
private HddsProtos.ReplicationType type;
private HddsProtos.ReplicationFactor factor;
private PipelineID id;
/**
* Constructs a new pipeline data structure.
*
* @param leaderID - Leader datanode id
* @param lifeCycleState - Pipeline State
* @param replicationType - Replication protocol
* @param replicationFactor - replication count on datanodes
* @param id - pipeline ID
*/
public Pipeline(String leaderID, HddsProtos.LifeCycleState lifeCycleState,
HddsProtos.ReplicationType replicationType,
HddsProtos.ReplicationFactor replicationFactor, PipelineID id) {
this.leaderID = leaderID;
this.lifeCycleState = lifeCycleState;
this.type = replicationType;
this.factor = replicationFactor;
this.id = id;
datanodes = new ConcurrentHashMap<>();
}
@Override
public int hashCode() {
return id.hashCode();
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Pipeline that = (Pipeline) o;
return id.equals(that.id)
&& factor.equals(that.factor)
&& type.equals(that.type)
&& lifeCycleState.equals(that.lifeCycleState)
&& leaderID.equals(that.leaderID);
}
/**
* Gets pipeline object from protobuf.
*
* @param pipelineProto - ProtoBuf definition for the pipeline.
* @return Pipeline Object
*/
public static Pipeline getFromProtoBuf(
HddsProtos.Pipeline pipelineProto) {
Preconditions.checkNotNull(pipelineProto);
Pipeline pipeline =
new Pipeline(pipelineProto.getLeaderID(),
pipelineProto.getState(),
pipelineProto.getType(),
pipelineProto.getFactor(),
PipelineID.getFromProtobuf(pipelineProto.getId()));
for (HddsProtos.DatanodeDetailsProto dataID :
pipelineProto.getMembersList()) {
pipeline.addMember(DatanodeDetails.getFromProtoBuf(dataID));
}
return pipeline;
}
/**
* returns the replication count.
* @return Replication Factor
*/
public HddsProtos.ReplicationFactor getFactor() {
return factor;
}
/**
* Returns the first machine in the set of datanodes.
*
* @return First Machine.
*/
@JsonIgnore
public DatanodeDetails getLeader() {
return getDatanodes().get(leaderID);
}
/**
* Adds a datanode to pipeline
* @param datanodeDetails datanode to be added.
* @return true if the dn was not earlier present, false otherwise
*/
public boolean addMember(DatanodeDetails datanodeDetails) {
return datanodes.put(datanodeDetails.getUuid().toString(),
datanodeDetails) == null;
}
public void resetPipeline() {
// reset datanodes in pipeline and learn about them through
// pipeline reports on SCM restart
datanodes.clear();
}
public Map<String, DatanodeDetails> getDatanodes() {
return datanodes;
}
public boolean isEmpty() {
return datanodes.isEmpty();
}
/**
* Returns the leader host.
*
* @return First Machine.
*/
public String getLeaderHost() {
return getDatanodes()
.get(leaderID).getHostName();
}
/**
*
* @return lead
*/
public String getLeaderID() {
return leaderID;
}
/**
* Returns all machines that make up this pipeline.
*
* @return List of Machines.
*/
@JsonIgnore
public List<DatanodeDetails> getMachines() {
return new ArrayList<>(getDatanodes().values());
}
/**
* Returns all machines that make up this pipeline.
*
* @return List of Machines.
*/
public List<String> getDatanodeHosts() {
List<String> dataHosts = new ArrayList<>();
for (DatanodeDetails datanode : getDatanodes().values()) {
dataHosts.add(datanode.getHostName());
}
return dataHosts;
}
/**
* Return a Protobuf Pipeline message from pipeline.
*
* @return Protobuf message
*/
@JsonIgnore
public HddsProtos.Pipeline getProtobufMessage() {
HddsProtos.Pipeline.Builder builder =
HddsProtos.Pipeline.newBuilder();
for (DatanodeDetails datanode : datanodes.values()) {
builder.addMembers(datanode.getProtoBufMessage());
}
builder.setLeaderID(leaderID);
if (lifeCycleState != null) {
builder.setState(lifeCycleState);
}
if (type != null) {
builder.setType(type);
}
if (factor != null) {
builder.setFactor(factor);
}
if (id != null) {
builder.setId(id.getProtobuf());
}
return builder.build();
}
/**
* Gets the State of the pipeline.
*
* @return - LifeCycleStates.
*/
public HddsProtos.LifeCycleState getLifeCycleState() {
return lifeCycleState;
}
/**
* Update the State of the pipeline.
*/
public void setLifeCycleState(HddsProtos.LifeCycleState nextState) {
lifeCycleState = nextState;
}
/**
* Gets the pipeline id.
*
* @return - Id of the pipeline
*/
public PipelineID getId() {
return id;
}
/**
* Returns the type.
*
* @return type - Standalone, Ratis, Chained.
*/
public HddsProtos.ReplicationType getType() {
return type;
}
@Override
public String toString() {
final StringBuilder b = new StringBuilder(getClass().getSimpleName())
.append("[");
getDatanodes().keySet().forEach(
node -> b.append(node.endsWith(getLeaderID()) ? "*" + id : id));
b.append(" id:").append(id);
if (getType() != null) {
b.append(" type:").append(getType().toString());
}
if (getFactor() != null) {
b.append(" factor:").append(getFactor().toString());
}
if (getLifeCycleState() != null) {
b.append(" State:").append(getLifeCycleState().toString());
}
return b.toString();
}
public void setType(HddsProtos.ReplicationType type) {
this.type = type;
}
/**
* Returns a JSON string of this object.
*
* @return String - json string
* @throws IOException
*/
public String toJsonString() throws IOException {
return WRITER.writeValueAsString(this);
}
@JsonFilter(PIPELINE_INFO)
class MixIn {
}
}

View File

@ -1,97 +0,0 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.container.common.helpers;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.ratis.protocol.RaftGroupId;
import java.util.UUID;
/**
* ID for the pipeline, the ID is based on UUID so that it can be used
* in Ratis as RaftGroupId, GroupID is used by the datanodes to initialize
* the ratis group they are part of.
*/
public final class PipelineID implements Comparable<PipelineID> {
private UUID id;
private RaftGroupId groupId;
private PipelineID(UUID id) {
this.id = id;
this.groupId = RaftGroupId.valueOf(id);
}
public static PipelineID randomId() {
return new PipelineID(UUID.randomUUID());
}
public static PipelineID valueOf(UUID id) {
return new PipelineID(id);
}
public static PipelineID valueOf(RaftGroupId groupId) {
return valueOf(groupId.getUuid());
}
public RaftGroupId getRaftGroupID() {
return groupId;
}
public UUID getId() {
return id;
}
public HddsProtos.PipelineID getProtobuf() {
return HddsProtos.PipelineID.newBuilder().setId(id.toString()).build();
}
public static PipelineID getFromProtobuf(HddsProtos.PipelineID protos) {
return new PipelineID(UUID.fromString(protos.getId()));
}
@Override
public String toString() {
return "pipelineId=" + id;
}
@Override
public int compareTo(PipelineID o) {
return this.id.compareTo(o.id);
}
@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
PipelineID that = (PipelineID) o;
return id.equals(that.id);
}
@Override
public int hashCode() {
return id.hashCode();
}
}

View File

@ -60,7 +60,7 @@ private Pipeline(PipelineID id, ReplicationType type,
*
* @return PipelineID
*/
public PipelineID getID() {
public PipelineID getId() {
return id;
}
@ -87,11 +87,26 @@ public ReplicationFactor getFactor() {
*
* @return - LifeCycleStates.
*/
PipelineState getPipelineState() {
// TODO: See if we need to expose this.
public PipelineState getPipelineState() {
return state;
}
/**
* Returns the list of nodes which form this pipeline.
*
* @return List of DatanodeDetails
*/
public List<DatanodeDetails> getNodes() {
return new ArrayList<>(nodeStatus.keySet());
}
public DatanodeDetails getFirstNode() throws IOException {
if (nodeStatus.isEmpty()) {
throw new IOException(String.format("Pipeline=%s is empty", id));
}
return nodeStatus.keySet().iterator().next();
}
public boolean isClosed() {
return state == PipelineState.CLOSED;
}
@ -117,13 +132,8 @@ boolean isHealthy() {
return true;
}
/**
* Returns the list of nodes which form this pipeline.
*
* @return List of DatanodeDetails
*/
public List<DatanodeDetails> getNodes() {
return new ArrayList<>(nodeStatus.keySet());
public boolean isEmpty() {
return nodeStatus.isEmpty();
}
public HddsProtos.Pipeline getProtobufMessage() {
@ -138,7 +148,7 @@ public HddsProtos.Pipeline getProtobufMessage() {
return builder.build();
}
public static Pipeline fromProtobuf(HddsProtos.Pipeline pipeline) {
public static Pipeline getFromProtobuf(HddsProtos.Pipeline pipeline) {
return new Builder().setId(PipelineID.getFromProtobuf(pipeline.getId()))
.setFactor(pipeline.getFactor())
.setType(pipeline.getType())
@ -164,8 +174,7 @@ public boolean equals(Object o) {
.append(id, that.id)
.append(type, that.type)
.append(factor, that.factor)
.append(state, that.state)
.append(nodeStatus, that.nodeStatus)
.append(getNodes(), that.getNodes())
.isEquals();
}
@ -175,7 +184,6 @@ public int hashCode() {
.append(id)
.append(type)
.append(factor)
.append(state)
.append(nodeStatus)
.toHashCode();
}
@ -244,7 +252,10 @@ public Pipeline build() {
}
}
enum PipelineState {
/**
* Possible Pipeline states in SCM.
*/
public enum PipelineState {
ALLOCATED, OPEN, CLOSED
}
}

View File

@ -0,0 +1,28 @@
package org.apache.hadoop.hdds.scm.pipeline;
import java.io.IOException;
/**
* Signals that a pipeline is missing from PipelineManager.
*/
public class PipelineNotFoundException extends IOException{
/**
* Constructs an {@code PipelineNotFoundException} with {@code null}
* as its error detail message.
*/
public PipelineNotFoundException() {
super();
}
/**
* Constructs an {@code PipelineNotFoundException} with the specified
* detail message.
*
* @param message
* The detail message (which is saved for later retrieval
* by the {@link #getMessage()} method)
*/
public PipelineNotFoundException(String message) {
super(message);
}
}

View File

@ -20,7 +20,7 @@
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerLocationProtocolProtos.ObjectStageChangeRequestProto;

View File

@ -23,7 +23,7 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
@ -103,7 +103,7 @@ public AllocatedBlock allocateBlock(long size,
}
AllocatedBlock.Builder builder = new AllocatedBlock.Builder()
.setBlockID(BlockID.getFromProtobuf(response.getBlockID()))
.setPipeline(Pipeline.getFromProtoBuf(response.getPipeline()))
.setPipeline(Pipeline.getFromProtobuf(response.getPipeline()))
.setShouldCreateContainer(response.getCreateContainer());
return builder.build();
}

View File

@ -29,7 +29,7 @@
import org.apache.hadoop.hdds.scm.ScmInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
@ -292,7 +292,7 @@ public Pipeline createReplicationPipeline(HddsProtos.ReplicationType
PipelineResponseProto.Error.success) {
Preconditions.checkState(response.hasPipeline(), "With success, " +
"must come a pipeline");
return Pipeline.getFromProtoBuf(response.getPipeline());
return Pipeline.getFromProtobuf(response.getPipeline());
} else {
String errorMessage = String.format("create replication pipeline " +
"failed. code : %s Message: %s", response.getErrorCode(),

View File

@ -92,7 +92,7 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
.newBuilder()
.setBlockID(datanodeBlockID)
.setBlockCommitSequenceId(blockCommitSequenceId);
String id = xceiverClient.getPipeline().getLeader().getUuidString();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
@ -125,7 +125,7 @@ public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient,
getBlockLengthRequestBuilder =
ContainerProtos.GetCommittedBlockLengthRequestProto.newBuilder().
setBlockID(blockID.getDatanodeBlockIDProtobuf());
String id = xceiverClient.getPipeline().getLeader().getUuidString();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request =
ContainerCommandRequestProto.newBuilder()
.setCmdType(Type.GetCommittedBlockLength)
@ -152,7 +152,7 @@ public static ContainerProtos.PutBlockResponseProto putBlock(
String traceID) throws IOException {
PutBlockRequestProto.Builder createBlockRequest =
PutBlockRequestProto.newBuilder().setBlockData(containerBlockData);
String id = xceiverClient.getPipeline().getLeader().getUuidString();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request =
ContainerCommandRequestProto.newBuilder().setCmdType(Type.PutBlock)
.setContainerID(containerBlockData.getBlockID().getContainerID())
@ -179,7 +179,7 @@ public static ReadChunkResponseProto readChunk(XceiverClientSpi xceiverClient,
.newBuilder()
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
.setChunkData(chunk);
String id = xceiverClient.getPipeline().getLeader().getUuidString();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.ReadChunk)
@ -211,7 +211,7 @@ public static void writeChunk(XceiverClientSpi xceiverClient, ChunkInfo chunk,
.setBlockID(blockID.getDatanodeBlockIDProtobuf())
.setChunkData(chunk)
.setData(data);
String id = xceiverClient.getPipeline().getLeader().getUuidString();
String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()
.setCmdType(Type.WriteChunk)
@ -260,7 +260,7 @@ public static void writeSmallFile(XceiverClientSpi client,
.setBlock(createBlockRequest).setData(ByteString.copyFrom(data))
.build();
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request =
ContainerCommandRequestProto.newBuilder()
.setCmdType(Type.PutSmallFile)
@ -288,7 +288,7 @@ public static void createContainer(XceiverClientSpi client, long containerID,
createRequest.setContainerType(ContainerProtos.ContainerType
.KeyValueContainer);
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
request.setCmdType(ContainerProtos.Type.CreateContainer);
@ -314,7 +314,7 @@ public static void deleteContainer(XceiverClientSpi client, long containerID,
ContainerProtos.DeleteContainerRequestProto.Builder deleteRequest =
ContainerProtos.DeleteContainerRequestProto.newBuilder();
deleteRequest.setForceDelete(force);
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
@ -338,7 +338,7 @@ public static void deleteContainer(XceiverClientSpi client, long containerID,
*/
public static void closeContainer(XceiverClientSpi client,
long containerID, String traceID) throws IOException {
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
@ -362,7 +362,7 @@ public static void closeContainer(XceiverClientSpi client,
public static ReadContainerResponseProto readContainer(
XceiverClientSpi client, long containerID,
String traceID) throws IOException {
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto.Builder request =
ContainerCommandRequestProto.newBuilder();
@ -396,7 +396,7 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client,
GetSmallFileRequestProto
.newBuilder().setBlock(getBlock)
.build();
String id = client.getPipeline().getLeader().getUuidString();
String id = client.getPipeline().getFirstNode().getUuidString();
ContainerCommandRequestProto request = ContainerCommandRequestProto
.newBuilder()

View File

@ -19,7 +19,7 @@
package org.apache.ratis;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.ratis.client.RaftClient;
@ -40,6 +40,7 @@
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
@ -88,7 +89,7 @@ static RaftPeer toRaftPeer(DatanodeDetails id) {
}
static List<RaftPeer> toRaftPeers(Pipeline pipeline) {
return toRaftPeers(pipeline.getMachines());
return toRaftPeers(pipeline.getNodes());
}
static <E extends DatanodeDetails> List<RaftPeer> toRaftPeers(
@ -125,15 +126,15 @@ static RaftGroup newRaftGroup(RaftGroupId groupId,
}
static RaftGroup newRaftGroup(Pipeline pipeline) {
return RaftGroup.valueOf(pipeline.getId().getRaftGroupID(),
return RaftGroup.valueOf(RaftGroupId.valueOf(pipeline.getId().getId()),
toRaftPeers(pipeline));
}
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
RetryPolicy retryPolicy) {
return newRaftClient(rpcType, toRaftPeerId(pipeline.getLeader()),
newRaftGroup(pipeline.getId().getRaftGroupID(), pipeline.getMachines()),
retryPolicy);
RetryPolicy retryPolicy) throws IOException {
return newRaftClient(rpcType, toRaftPeerId(pipeline.getFirstNode()),
newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()),
pipeline.getNodes()), retryPolicy);
}
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,

View File

@ -27,7 +27,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReport;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.common.helpers.
StorageContainerException;
import org.apache.hadoop.ozone.OzoneConfigKeys;

View File

@ -31,7 +31,7 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineAction;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
@ -372,7 +372,7 @@ private RaftClientRequest createRaftClientRequest(
ContainerCommandRequestProto request, HddsProtos.PipelineID pipelineID,
RaftClientRequest.Type type) {
return new RaftClientRequest(clientId, server.getId(),
PipelineID.getFromProtobuf(pipelineID).getRaftGroupID(),
RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineID).getId()),
nextCallId(), 0, Message.valueOf(request.toByteString()), type);
}
@ -405,7 +405,7 @@ private void handlePipelineFailure(RaftGroupId groupId,
+ roleInfoProto.getRole());
}
PipelineID pipelineID = PipelineID.valueOf(groupId);
PipelineID pipelineID = PipelineID.valueOf(groupId.getUuid());
ClosePipelineInfo.Builder closePipelineInfo =
ClosePipelineInfo.newBuilder()
.setPipelineID(pipelineID.getProtobuf())
@ -429,8 +429,8 @@ public List<PipelineReport> getPipelineReport() {
List<PipelineReport> reports = new ArrayList<>();
for (RaftGroupId groupId : gids) {
reports.add(PipelineReport.newBuilder()
.setPipelineID(PipelineID.valueOf(groupId).getProtobuf())
.build());
.setPipelineID(PipelineID.valueOf(groupId.getUuid()).getProtobuf())
.build());
}
return reports;
} catch (Exception e) {

View File

@ -22,7 +22,7 @@
.StorageContainerDatanodeProtocolProtos.SCMCommandProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.CloseContainerCommandProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
/**
* Asks datanode to close a container.

View File

@ -305,7 +305,7 @@ public AllocatedBlock allocateBlock(final long size,
private AllocatedBlock newBlock(ContainerWithPipeline containerWithPipeline,
HddsProtos.LifeCycleState state) throws IOException {
ContainerInfo containerInfo = containerWithPipeline.getContainerInfo();
if (containerWithPipeline.getPipeline().getDatanodes().size() == 0) {
if (containerWithPipeline.getPipeline().getNodes().size() == 0) {
LOG.error("Pipeline Machine count is zero.");
return null;
}

View File

@ -31,7 +31,7 @@
import java.util.stream.Collectors;
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.pipeline.Pipeline;
/**
* A wrapper class to hold info about datanode and all deleted block
@ -74,7 +74,7 @@ public boolean addTransaction(DeletedBlocksTransaction tx,
}
boolean success = false;
for (DatanodeDetails dd : pipeline.getMachines()) {
for (DatanodeDetails dd : pipeline.getNodes()) {
UUID dnID = dd.getUuid();
if (dnsWithTransactionCommitted == null ||
!dnsWithTransactionCommitted.contains(dnID)) {

View File

@ -31,7 +31,7 @@
.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.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdfs.DFSUtil;
@ -261,8 +261,7 @@ public void commitTransactions(
Pipeline pipeline =
containerManager.getContainerWithPipeline(
ContainerID.valueof(containerId)).getPipeline();
Collection<DatanodeDetails> containerDnsDetails =
pipeline.getDatanodes().values();
Collection<DatanodeDetails> containerDnsDetails = pipeline.getNodes();
// The delete entry can be safely removed from the log if all the
// corresponding nodes commit the txn. It is required to check that
// the nodes returned in the pipeline match the replication factor.

View File

@ -19,7 +19,7 @@
import java.io.IOException;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.IdentifiableEventPayload;
@ -123,7 +123,7 @@ private void fireCloseContainerEvents(
info.getReplicationType(), info.getPipelineID());
Pipeline pipeline = containerWithPipeline.getPipeline();
pipeline.getMachines().stream()
pipeline.getNodes().stream()
.map(node ->
new CommandForDatanode<>(node.getUuid(), closeContainerCommand))
.forEach(command -> publisher.fireEvent(DATANODE_COMMAND, command));

View File

@ -24,7 +24,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
import java.io.Closeable;
import java.io.IOException;
@ -73,7 +73,7 @@ ContainerInfo getContainer(ContainerID containerID)
* @throws IOException
*/
ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException;
throws ContainerNotFoundException, PipelineNotFoundException;
/**
* Returns containers under certain conditions.
@ -175,6 +175,4 @@ void updateDeleteTransactionId(Map<Long, Long> deleteTransactionMap)
ContainerWithPipeline getMatchingContainerWithPipeline(long size,
String owner, ReplicationType type, ReplicationFactor factor,
LifeCycleState state) throws IOException;
PipelineSelector getPipelineSelector();
}

View File

@ -22,11 +22,11 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
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;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleEvent;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
@ -42,6 +42,7 @@
import java.io.IOException;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.NavigableSet;
import java.util.Set;
@ -232,19 +233,28 @@ void loadContainer(final ContainerInfo containerInfo)
/**
* Allocates a new container based on the type, replication etc.
*
* @param selector -- Pipeline selector class.
* @param pipelineManager -- Pipeline Manager class.
* @param type -- Replication type.
* @param replicationFactor - Replication replicationFactor.
* @return ContainerWithPipeline
* @throws IOException on Failure.
*/
ContainerInfo allocateContainer(final PipelineSelector selector,
ContainerInfo allocateContainer(final PipelineManager pipelineManager,
final HddsProtos.ReplicationType type,
final HddsProtos.ReplicationFactor replicationFactor, final String owner)
throws IOException {
final Pipeline pipeline = selector.getReplicationPipeline(type,
replicationFactor);
Pipeline pipeline;
try {
pipeline = pipelineManager.createPipeline(type, replicationFactor);
} catch (IOException e) {
final List<Pipeline> pipelines =
pipelineManager.getPipelines(type, replicationFactor);
if (pipelines.isEmpty()) {
throw new IOException("Could not allocate container");
}
pipeline = pipelines.get((int) containerCount.get() % pipelines.size());
}
Preconditions.checkNotNull(pipeline, "Pipeline type=%s/"
+ "replication=%s couldn't be found for the new container. "
@ -263,7 +273,8 @@ ContainerInfo allocateContainer(final PipelineSelector selector,
.setReplicationFactor(replicationFactor)
.setReplicationType(pipeline.getType())
.build();
selector.addContainerToPipeline(pipeline.getId(), containerID);
pipelineManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(containerID));
Preconditions.checkNotNull(containerInfo);
containers.addContainer(containerInfo);
LOG.trace("New container allocated: {}", containerInfo);

View File

@ -26,13 +26,13 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.LifeCycleState;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
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;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineNotFoundException;
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;
@ -86,7 +86,7 @@ public class SCMContainerManager implements ContainerManager {
private final Lock lock;
private final MetadataStore containerStore;
private final PipelineSelector pipelineSelector;
private final PipelineManager pipelineManager;
private final ContainerStateManager containerStateManager;
private final LeaseManager<ContainerInfo> containerLeaseManager;
private final EventPublisher eventPublisher;
@ -102,12 +102,13 @@ public class SCMContainerManager implements ContainerManager {
* passed to LevelDB and this memory is allocated in Native code space.
* CacheSize is specified
* in MB.
* @param pipelineManager - PipelineManager
* @throws IOException on Failure.
*/
@SuppressWarnings("unchecked")
public SCMContainerManager(final Configuration conf,
final NodeManager nodeManager, final EventPublisher eventPublisher)
throws IOException {
final NodeManager nodeManager, PipelineManager pipelineManager,
final EventPublisher eventPublisher) throws IOException {
final File metaDir = getOzoneMetaDirPath(conf);
final File containerDBPath = new File(metaDir, SCM_CONTAINER_DB);
@ -123,8 +124,7 @@ public SCMContainerManager(final Configuration conf,
this.lock = new ReentrantLock();
this.size = (long) conf.getStorageSize(OZONE_SCM_CONTAINER_SIZE,
OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);
this.pipelineSelector = new PipelineSelector(nodeManager,
conf, eventPublisher, cacheSize);
this.pipelineManager = pipelineManager;
this.containerStateManager = new ContainerStateManager(conf);
this.eventPublisher = eventPublisher;
@ -147,8 +147,10 @@ private void loadExistingContainers() throws IOException {
HddsProtos.SCMContainerInfo.PARSER.parseFrom(entry.getValue()));
Preconditions.checkNotNull(container);
containerStateManager.loadContainer(container);
pipelineSelector.addContainerToPipeline(
container.getPipelineID(), container.getContainerID());
if (container.isOpen()) {
pipelineManager.addContainerToPipeline(container.getPipelineID(),
ContainerID.valueof(container.getContainerID()));
}
}
}
@ -214,28 +216,23 @@ public ContainerInfo getContainer(final ContainerID containerID)
*/
@Override
public ContainerWithPipeline getContainerWithPipeline(ContainerID containerID)
throws ContainerNotFoundException {
throws ContainerNotFoundException, PipelineNotFoundException {
lock.lock();
try {
final ContainerInfo contInfo = getContainer(containerID);
Pipeline pipeline;
String leaderId = "";
if (contInfo.isOpen()) {
// If pipeline with given pipeline Id already exist return it
pipeline = pipelineSelector.getPipeline(contInfo.getPipelineID());
pipeline = pipelineManager.getPipeline(contInfo.getPipelineID());
} else {
// For close containers create pipeline from datanodes with replicas
Set<ContainerReplica> dnWithReplicas = containerStateManager
.getContainerReplicas(contInfo.containerID());
if (!dnWithReplicas.isEmpty()) {
leaderId = dnWithReplicas.iterator().next()
.getDatanodeDetails().getUuidString();
}
pipeline = new Pipeline(leaderId, contInfo.getState(),
ReplicationType.STAND_ALONE, contInfo.getReplicationFactor(),
PipelineID.randomId());
dnWithReplicas.stream().map(ContainerReplica::getDatanodeDetails).
forEach(pipeline::addMember);
List<DatanodeDetails> dns =
dnWithReplicas.stream().map(ContainerReplica::getDatanodeDetails)
.collect(Collectors.toList());
pipeline = pipelineManager.createPipeline(ReplicationType.STAND_ALONE,
contInfo.getReplicationFactor(), dns);
}
return new ContainerWithPipeline(contInfo, pipeline);
} finally {
@ -290,8 +287,8 @@ public ContainerWithPipeline allocateContainer(final ReplicationType type,
lock.lock();
try {
final ContainerInfo containerInfo; containerInfo = containerStateManager
.allocateContainer(pipelineSelector, type, replicationFactor, owner);
final Pipeline pipeline = pipelineSelector.getPipeline(
.allocateContainer(pipelineManager, type, replicationFactor, owner);
final Pipeline pipeline = pipelineManager.getPipeline(
containerInfo.getPipelineID());
try {
@ -360,8 +357,8 @@ public HddsProtos.LifeCycleState updateContainerState(
ContainerInfo updatedContainer =
updateContainerStateInternal(containerID, event);
if (!updatedContainer.isOpen()) {
pipelineSelector.removeContainerFromPipeline(
updatedContainer.getPipelineID(), containerID.getId());
pipelineManager.removeContainerFromPipeline(
updatedContainer.getPipelineID(), containerID);
}
final byte[] dbKey = Longs.toByteArray(containerID.getId());
containerStore.put(dbKey, updatedContainer.getProtobuf().toByteArray());
@ -485,7 +482,7 @@ public ContainerWithPipeline getMatchingContainerWithPipeline(
if (containerInfo == null) {
return null;
}
Pipeline pipeline = pipelineSelector
Pipeline pipeline = pipelineManager
.getPipeline(containerInfo.getPipelineID());
return new ContainerWithPipeline(containerInfo, pipeline);
}
@ -647,13 +644,5 @@ public void close() throws IOException {
if (containerStore != null) {
containerStore.close();
}
if (pipelineSelector != null) {
pipelineSelector.shutdown();
}
}
public PipelineSelector getPipelineSelector() {
return pipelineSelector;
}
}

View File

@ -29,7 +29,6 @@
import org.apache.hadoop.hdds.scm.container.CloseContainerEventHandler
.CloseContainerRetryableReq;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.PipelineReportFromDatanode;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
@ -102,14 +101,6 @@ public final class SCMEvents {
PIPELINE_ACTIONS = new TypedEvent<>(PipelineActionsFromDatanode.class,
"Pipeline_Actions");
/**
* Pipeline close event are triggered to close pipeline because of failure,
* stale node, decommissioning etc.
*/
public static final TypedEvent<PipelineID>
PIPELINE_CLOSE = new TypedEvent<>(PipelineID.class,
"Pipeline_Close");
/**
* A Command status report will be sent by datanodes. This repoort is received
* by SCMDatanodeHeartbeatDispatcher and CommandReport event is generated.

View File

@ -19,8 +19,8 @@
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.NodeReportProto;
import org.apache.hadoop.hdds.scm.container.ContainerID;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;

View File

@ -25,8 +25,8 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.scm.HddsServerUtil;
import org.apache.hadoop.hdds.scm.container.ContainerID;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;

View File

@ -22,8 +22,8 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.container.ContainerID;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.node.states.NodeAlreadyExistsException;
import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException;

View File

@ -19,24 +19,44 @@
package org.apache.hadoop.hdds.scm.node;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.Set;
/**
* Handles Stale node event.
*/
public class StaleNodeHandler implements EventHandler<DatanodeDetails> {
private static final Logger LOG =
LoggerFactory.getLogger(StaleNodeHandler.class);
private final PipelineSelector pipelineSelector;
private final NodeManager nodeManager;
private final PipelineManager pipelineManager;
public StaleNodeHandler(PipelineSelector pipelineSelector) {
this.pipelineSelector = pipelineSelector;
public StaleNodeHandler(NodeManager nodeManager,
PipelineManager pipelineManager) {
this.nodeManager = nodeManager;
this.pipelineManager = pipelineManager;
}
@Override
public void onMessage(DatanodeDetails datanodeDetails,
EventPublisher publisher) {
pipelineSelector.handleStaleNode(datanodeDetails);
Set<PipelineID> pipelineIds =
nodeManager.getPipelineByDnID(datanodeDetails.getUuid());
for (PipelineID pipelineID : pipelineIds) {
try {
pipelineManager.finalizePipeline(pipelineID);
} catch (IOException e) {
LOG.info("Could not finalize pipeline={} for dn={}", pipelineID,
datanodeDetails);
}
}
}
}

View File

@ -19,8 +19,8 @@
package org.apache.hadoop.hdds.scm.node.states;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import java.util.HashSet;
import java.util.Set;
@ -55,7 +55,7 @@ public Set<PipelineID> getPipelines(UUID datanode) {
* @param pipeline Pipeline to be added
*/
public synchronized void addPipeline(Pipeline pipeline) {
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
for (DatanodeDetails details : pipeline.getNodes()) {
UUID dnId = details.getUuid();
dn2ObjectMap.computeIfAbsent(dnId, k -> new HashSet<>())
.add(pipeline.getId());
@ -63,7 +63,7 @@ public synchronized void addPipeline(Pipeline pipeline) {
}
public synchronized void removePipeline(Pipeline pipeline) {
for (DatanodeDetails details : pipeline.getDatanodes().values()) {
for (DatanodeDetails details : pipeline.getNodes()) {
UUID dnId = details.getUuid();
dn2ObjectMap.computeIfPresent(dnId,
(k, v) -> {

View File

@ -50,8 +50,8 @@ public Pipeline create(ReplicationType type, ReplicationFactor factor)
return providers.get(type).create(factor);
}
public Pipeline create(ReplicationType type, List<DatanodeDetails> nodes)
throws IOException {
return providers.get(type).create(nodes);
public Pipeline create(ReplicationType type, ReplicationFactor factor,
List<DatanodeDetails> nodes) {
return providers.get(type).create(factor, nodes);
}
}

View File

@ -36,14 +36,14 @@ public interface PipelineManager extends Closeable {
Pipeline createPipeline(ReplicationType type, ReplicationFactor factor)
throws IOException;
Pipeline createPipeline(ReplicationType type, List<DatanodeDetails> nodes)
throws IOException;
Pipeline createPipeline(ReplicationType type, ReplicationFactor factor,
List<DatanodeDetails> nodes);
Pipeline getPipeline(PipelineID pipelineID) throws IOException;
Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException;
List<Pipeline> getPipelinesByType(ReplicationType type);
List<Pipeline> getPipelines(ReplicationType type);
List<Pipeline> getPipelinesByTypeAndFactor(ReplicationType type,
List<Pipeline> getPipelines(ReplicationType type,
ReplicationFactor factor);
void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)

View File

@ -31,5 +31,5 @@ public interface PipelineProvider {
Pipeline create(ReplicationFactor factor) throws IOException;
Pipeline create(List<DatanodeDetails> nodes) throws IOException;
Pipeline create(ReplicationFactor factor, List<DatanodeDetails> nodes);
}

View File

@ -76,7 +76,13 @@ public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
private void processPipelineReport(PipelineReport report, DatanodeDetails dn)
throws IOException {
PipelineID pipelineID = PipelineID.getFromProtobuf(report.getPipelineID());
Pipeline pipeline = pipelineManager.getPipeline(pipelineID);
Pipeline pipeline = null;
try {
pipeline = pipelineManager.getPipeline(pipelineID);
} catch (PipelineNotFoundException e) {
//TODO: introduce per datanode command for pipeline destroy
return;
}
if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED) {
pipeline.reportDatanode(dn);
@ -87,14 +93,14 @@ private void processPipelineReport(PipelineReport report, DatanodeDetails dn)
} else if (pipeline.isClosed()) {
int numContainers = pipelineManager.getNumberOfContainers(pipelineID);
if (numContainers == 0) {
// if all the containers have been closed the pipeline can be destroyed
// remove the pipeline from the pipeline manager
pipelineManager.removePipeline(pipelineID);
// since all the containers have been closed the pipeline can be
// destroyed
try (XceiverClientRatis client =
XceiverClientRatis.newXceiverClientRatis(pipeline, conf)) {
client.destroyPipeline();
}
// after successfully destroying the pipeline, the pipeline can be
// removed from the pipeline manager
pipelineManager.removePipeline(pipelineID);
}
} else {
// In OPEN state case just report the datanode

View File

@ -23,8 +23,6 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
@ -39,9 +37,6 @@
*/
class PipelineStateManager {
private static final Logger LOG = LoggerFactory.getLogger(
org.apache.hadoop.hdds.scm.pipelines.PipelineStateManager.class);
private final PipelineStateMap pipelineStateMap;
PipelineStateManager(Configuration conf) {
@ -57,17 +52,20 @@ void addContainerToPipeline(PipelineID pipelineId, ContainerID containerID)
pipelineStateMap.addContainerToPipeline(pipelineId, containerID);
}
Pipeline getPipeline(PipelineID pipelineID) throws IOException {
Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException {
return pipelineStateMap.getPipeline(pipelineID);
}
List<Pipeline> getPipelinesByType(ReplicationType type) {
return pipelineStateMap.getPipelinesByType(type);
List<Pipeline> getPipelines(ReplicationType type) {
return pipelineStateMap.getPipelines(type);
}
List<Pipeline> getPipelinesByTypeAndFactor(ReplicationType type,
ReplicationFactor factor) {
return pipelineStateMap.getPipelinesByTypeAndFactor(type, factor);
List<Pipeline> getPipelines(ReplicationType type, ReplicationFactor factor) {
return pipelineStateMap.getPipelines(type, factor);
}
List<Pipeline> getPipelines(ReplicationType type, PipelineState... states) {
return pipelineStateMap.getPipelines(type, states);
}
Set<ContainerID> getContainers(PipelineID pipelineID) throws IOException {
@ -78,8 +76,8 @@ int getNumberOfContainers(PipelineID pipelineID) throws IOException {
return pipelineStateMap.getNumberOfContainers(pipelineID);
}
void removePipeline(PipelineID pipelineID) throws IOException {
pipelineStateMap.removePipeline(pipelineID);
Pipeline removePipeline(PipelineID pipelineID) throws IOException {
return pipelineStateMap.removePipeline(pipelineID);
}
void removeContainerFromPipeline(PipelineID pipelineID,
@ -87,7 +85,8 @@ void removeContainerFromPipeline(PipelineID pipelineID,
pipelineStateMap.removeContainerFromPipeline(pipelineID, containerID);
}
Pipeline finalizePipeline(PipelineID pipelineId) throws IOException {
Pipeline finalizePipeline(PipelineID pipelineId)
throws PipelineNotFoundException {
Pipeline pipeline = pipelineStateMap.getPipeline(pipelineId);
if (!pipeline.isClosed()) {
pipeline = pipelineStateMap

View File

@ -31,8 +31,7 @@
/**
* Holds the data structures which maintain the information about pipeline and
* its state. All the read write operations in this class are protected by a
* lock.
* its state.
* Invariant: If a pipeline exists in PipelineStateMap, both pipelineMap and
* pipeline2container would have a non-null mapping for it.
*/
@ -65,12 +64,12 @@ void addPipeline(Pipeline pipeline) throws IOException {
String.format("Nodes size=%d, replication factor=%d do not match ",
pipeline.getNodes().size(), pipeline.getFactor().getNumber()));
if (pipelineMap.putIfAbsent(pipeline.getID(), pipeline) != null) {
LOG.warn("Duplicate pipeline ID detected. {}", pipeline.getID());
if (pipelineMap.putIfAbsent(pipeline.getId(), pipeline) != null) {
LOG.warn("Duplicate pipeline ID detected. {}", pipeline.getId());
throw new IOException(String
.format("Duplicate pipeline ID %s detected.", pipeline.getID()));
.format("Duplicate pipeline ID %s detected.", pipeline.getId()));
}
pipeline2container.put(pipeline.getID(), new TreeSet<>());
pipeline2container.put(pipeline.getId(), new TreeSet<>());
}
/**
@ -85,12 +84,13 @@ void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
Preconditions.checkNotNull(pipelineID,
"Pipeline Id cannot be null");
Preconditions.checkNotNull(containerID,
"container Id cannot be null");
"Container Id cannot be null");
Pipeline pipeline = getPipeline(pipelineID);
if (!pipeline.isOpen()) {
throw new IOException(
String.format("%s is not in open state", pipelineID));
if (pipeline.isClosed()) {
throw new IOException(String
.format("Cannot add container to pipeline=%s in closed state",
pipelineID));
}
pipeline2container.get(pipelineID).add(containerID);
}
@ -102,10 +102,14 @@ void addContainerToPipeline(PipelineID pipelineID, ContainerID containerID)
* @return Pipeline
* @throws IOException if pipeline is not found
*/
Pipeline getPipeline(PipelineID pipelineID) throws IOException {
Pipeline getPipeline(PipelineID pipelineID) throws PipelineNotFoundException {
Preconditions.checkNotNull(pipelineID,
"Pipeline Id cannot be null");
Pipeline pipeline = pipelineMap.get(pipelineID);
if (pipeline == null) {
throw new IOException(String.format("%s not found", pipelineID));
throw new PipelineNotFoundException(
String.format("%s not found", pipelineID));
}
return pipeline;
}
@ -116,28 +120,51 @@ Pipeline getPipeline(PipelineID pipelineID) throws IOException {
* @param type - ReplicationType
* @return List of pipelines which have the specified replication type
*/
List<Pipeline> getPipelinesByType(ReplicationType type) {
List<Pipeline> getPipelines(ReplicationType type) {
Preconditions.checkNotNull(type, "Replication type cannot be null");
return pipelineMap.values().stream().filter(p -> p.getType().equals(type))
return pipelineMap.values().stream()
.filter(p -> p.getType().equals(type))
.collect(Collectors.toList());
}
/**
* Get open pipeline corresponding to specified replication type and factor.
* Get pipeline corresponding to specified replication type and factor.
*
* @param type - ReplicationType
* @param factor - ReplicationFactor
* @return List of open pipelines with specified replication type and factor
* @return List of pipelines with specified replication type and factor
*/
List<Pipeline> getPipelinesByTypeAndFactor(ReplicationType type,
ReplicationFactor factor) {
List<Pipeline> getPipelines(ReplicationType type, ReplicationFactor factor) {
Preconditions.checkNotNull(type, "Replication type cannot be null");
Preconditions.checkNotNull(factor, "Replication factor cannot be null");
return pipelineMap.values().stream()
.filter(pipeline -> pipeline.isOpen() && pipeline.getType() == type
.filter(pipeline -> pipeline.getType() == type
&& pipeline.getFactor() == factor)
.collect(Collectors.toList());
}
/**
* Get list of pipeline corresponding to specified replication type and
* pipeline states.
*
* @param type - ReplicationType
* @param states - Array of required PipelineState
* @return List of pipelines with specified replication type and states
*/
List<Pipeline> getPipelines(ReplicationType type, PipelineState... states) {
Preconditions.checkNotNull(type, "Replication type cannot be null");
Preconditions.checkNotNull(states, "Pipeline state cannot be null");
Set<PipelineState> pipelineStates = new HashSet<>();
pipelineStates.addAll(Arrays.asList(states));
return pipelineMap.values().stream().filter(
pipeline -> pipeline.getType() == type && pipelineStates
.contains(pipeline.getPipelineState()))
.collect(Collectors.toList());
}
/**
* Get set of containerIDs corresponding to a pipeline.
*
@ -146,10 +173,14 @@ List<Pipeline> getPipelinesByTypeAndFactor(ReplicationType type,
* @throws IOException if pipeline is not found
*/
Set<ContainerID> getContainers(PipelineID pipelineID)
throws IOException {
throws PipelineNotFoundException {
Preconditions.checkNotNull(pipelineID,
"Pipeline Id cannot be null");
Set<ContainerID> containerIDs = pipeline2container.get(pipelineID);
if (containerIDs == null) {
throw new IOException(String.format("%s not found", pipelineID));
throw new PipelineNotFoundException(
String.format("%s not found", pipelineID));
}
return new HashSet<>(containerIDs);
}
@ -161,10 +192,15 @@ Set<ContainerID> getContainers(PipelineID pipelineID)
* @return Number of containers belonging to the pipeline
* @throws IOException if pipeline is not found
*/
int getNumberOfContainers(PipelineID pipelineID) throws IOException {
int getNumberOfContainers(PipelineID pipelineID)
throws PipelineNotFoundException {
Preconditions.checkNotNull(pipelineID,
"Pipeline Id cannot be null");
Set<ContainerID> containerIDs = pipeline2container.get(pipelineID);
if (containerIDs == null) {
throw new IOException(String.format("%s not found", pipelineID));
throw new PipelineNotFoundException(
String.format("%s not found", pipelineID));
}
return containerIDs.size();
}
@ -175,7 +211,7 @@ int getNumberOfContainers(PipelineID pipelineID) throws IOException {
* @param pipelineID - PipelineID of the pipeline to be removed
* @throws IOException if the pipeline is not empty or does not exist
*/
void removePipeline(PipelineID pipelineID) throws IOException {
Pipeline removePipeline(PipelineID pipelineID) throws IOException {
Preconditions.checkNotNull(pipelineID, "Pipeline Id cannot be null");
Pipeline pipeline = getPipeline(pipelineID);
@ -192,6 +228,7 @@ void removePipeline(PipelineID pipelineID) throws IOException {
pipelineMap.remove(pipelineID);
pipeline2container.remove(pipelineID);
return pipeline;
}
/**
@ -210,6 +247,10 @@ void removeContainerFromPipeline(PipelineID pipelineID,
"container Id cannot be null");
Set<ContainerID> containerIDs = pipeline2container.get(pipelineID);
if (containerIDs == null) {
throw new PipelineNotFoundException(
String.format("%s not found", pipelineID));
}
containerIDs.remove(containerID);
}
@ -223,7 +264,7 @@ void removeContainerFromPipeline(PipelineID pipelineID,
* @throws IOException if pipeline does not exist
*/
Pipeline updatePipelineState(PipelineID pipelineID, PipelineState state)
throws IOException {
throws PipelineNotFoundException {
Preconditions.checkNotNull(pipelineID, "Pipeline Id cannot be null");
Preconditions.checkNotNull(state, "Pipeline LifeCycleState cannot be null");

View File

@ -93,7 +93,7 @@ private static ContainerPlacementPolicy createContainerPlacementPolicy(
public Pipeline create(ReplicationFactor factor) throws IOException {
// Get set of datanodes already used for ratis pipeline
Set<DatanodeDetails> dnsUsed = new HashSet<>();
stateManager.getPipelinesByType(ReplicationType.RATIS)
stateManager.getPipelines(ReplicationType.RATIS)
.forEach(p -> dnsUsed.addAll(p.getNodes()));
// Get list of healthy nodes
@ -112,7 +112,7 @@ public Pipeline create(ReplicationFactor factor) throws IOException {
Pipeline pipeline = Pipeline.newBuilder()
.setId(PipelineID.randomId())
.setState(PipelineState.ALLOCATED)
.setState(PipelineState.OPEN)
.setType(ReplicationType.RATIS)
.setFactor(factor)
.setNodes(dns)
@ -122,16 +122,11 @@ public Pipeline create(ReplicationFactor factor) throws IOException {
}
@Override
public Pipeline create(List<DatanodeDetails> nodes) throws IOException {
ReplicationFactor factor = ReplicationFactor.valueOf(nodes.size());
if (factor == null) {
throw new IOException(String
.format("Nodes size=%d does not match any replication factor",
nodes.size()));
}
public Pipeline create(ReplicationFactor factor,
List<DatanodeDetails> nodes) {
return Pipeline.newBuilder()
.setId(PipelineID.randomId())
.setState(PipelineState.ALLOCATED)
.setState(PipelineState.OPEN)
.setType(ReplicationType.RATIS)
.setFactor(factor)
.setNodes(nodes)

View File

@ -78,14 +78,14 @@ public SCMPipelineManager(Configuration conf, NodeManager nodeManager,
File pipelineDBPath = new File(metaDir, SCM_PIPELINE_DB);
this.pipelineStore =
MetadataStoreBuilder.newBuilder()
.setCreateIfMissing(true)
.setConf(conf)
.setDbFile(pipelineDBPath)
.setCacheSize(cacheSize * OzoneConsts.MB)
.build();
initializePipelineState();
this.eventPublisher = eventPublisher;
this.nodeManager = nodeManager;
initializePipelineState();
}
private void initializePipelineState() throws IOException {
@ -97,12 +97,11 @@ private void initializePipelineState() throws IOException {
pipelineStore.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
for (Map.Entry<byte[], byte[]> entry : pipelines) {
Pipeline pipeline = Pipeline
.fromProtobuf(HddsProtos.Pipeline.PARSER.parseFrom(entry.getValue()));
Pipeline pipeline = Pipeline.getFromProtobuf(
HddsProtos.Pipeline.PARSER.parseFrom(entry.getValue()));
Preconditions.checkNotNull(pipeline);
stateManager.addPipeline(pipeline);
// TODO: add pipeline to node manager
// nodeManager.addPipeline(pipeline);
nodeManager.addPipeline(pipeline);
}
}
@ -112,10 +111,10 @@ public synchronized Pipeline createPipeline(
lock.writeLock().lock();
try {
Pipeline pipeline = pipelineFactory.create(type, factor);
pipelineStore.put(pipeline.getID().getProtobuf().toByteArray(),
pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
pipeline.getProtobufMessage().toByteArray());
stateManager.addPipeline(pipeline);
// TODO: add pipeline to node manager
nodeManager.addPipeline(pipeline);
return pipeline;
} finally {
lock.writeLock().unlock();
@ -123,20 +122,20 @@ public synchronized Pipeline createPipeline(
}
@Override
public Pipeline createPipeline(ReplicationType type,
List<DatanodeDetails> nodes)
throws IOException {
public Pipeline createPipeline(ReplicationType type, ReplicationFactor factor,
List<DatanodeDetails> nodes) {
// This will mostly be used to create dummy pipeline for SimplePipelines.
lock.writeLock().lock();
try {
return pipelineFactory.create(type, nodes);
return pipelineFactory.create(type, factor, nodes);
} finally {
lock.writeLock().unlock();
}
}
@Override
public Pipeline getPipeline(PipelineID pipelineID) throws IOException {
public Pipeline getPipeline(PipelineID pipelineID)
throws PipelineNotFoundException {
lock.readLock().lock();
try {
return stateManager.getPipeline(pipelineID);
@ -146,21 +145,21 @@ public Pipeline getPipeline(PipelineID pipelineID) throws IOException {
}
@Override
public List<Pipeline> getPipelinesByType(ReplicationType type) {
public List<Pipeline> getPipelines(ReplicationType type) {
lock.readLock().lock();
try {
return stateManager.getPipelinesByType(type);
return stateManager.getPipelines(type);
} finally {
lock.readLock().unlock();
}
}
@Override
public List<Pipeline> getPipelinesByTypeAndFactor(ReplicationType type,
public List<Pipeline> getPipelines(ReplicationType type,
ReplicationFactor factor) {
lock.readLock().lock();
try {
return stateManager.getPipelinesByTypeAndFactor(type, factor);
return stateManager.getPipelines(type, factor);
} finally {
lock.readLock().unlock();
}
@ -232,9 +231,9 @@ public void openPipeline(PipelineID pipelineId) throws IOException {
public void removePipeline(PipelineID pipelineID) throws IOException {
lock.writeLock().lock();
try {
stateManager.removePipeline(pipelineID);
pipelineStore.delete(pipelineID.getProtobuf().toByteArray());
// TODO: remove pipeline from node manager
Pipeline pipeline = stateManager.removePipeline(pipelineID);
nodeManager.removePipeline(pipeline);
} finally {
lock.writeLock().unlock();
}

View File

@ -54,7 +54,7 @@ public Pipeline create(ReplicationFactor factor) throws IOException {
Collections.shuffle(dns);
return Pipeline.newBuilder()
.setId(PipelineID.randomId())
.setState(PipelineState.ALLOCATED)
.setState(PipelineState.OPEN)
.setType(ReplicationType.STAND_ALONE)
.setFactor(factor)
.setNodes(dns.subList(0, factor.getNumber()))
@ -62,16 +62,11 @@ public Pipeline create(ReplicationFactor factor) throws IOException {
}
@Override
public Pipeline create(List<DatanodeDetails> nodes) throws IOException {
ReplicationFactor factor = ReplicationFactor.valueOf(nodes.size());
if (factor == null) {
throw new IOException(String
.format("Nodes size=%d does not match any replication factor",
nodes.size()));
}
public Pipeline create(ReplicationFactor factor,
List<DatanodeDetails> nodes) {
return Pipeline.newBuilder()
.setId(PipelineID.randomId())
.setState(PipelineState.ALLOCATED)
.setState(PipelineState.OPEN)
.setType(ReplicationType.STAND_ALONE)
.setFactor(factor)
.setNodes(nodes)

View File

@ -1,62 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.pipelines;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineAction;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.PipelineActionsFromDatanode;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Handles pipeline actions from datanode.
*/
public class PipelineActionEventHandler implements
EventHandler<PipelineActionsFromDatanode> {
public static final Logger LOG = LoggerFactory.getLogger(
PipelineActionEventHandler.class);
public PipelineActionEventHandler() {
}
@Override
public void onMessage(PipelineActionsFromDatanode report,
EventPublisher publisher) {
for (PipelineAction action : report.getReport().getPipelineActionsList()) {
switch (action.getAction()) {
case CLOSE:
PipelineID pipelineID = PipelineID.
getFromProtobuf(action.getClosePipeline().getPipelineID());
LOG.info("Closing pipeline " + pipelineID + " for reason:" + action
.getClosePipeline().getDetailedReason());
publisher.fireEvent(SCMEvents.PIPELINE_CLOSE, pipelineID);
break;
default:
LOG.error("unknown pipeline action:{}" + action.getAction());
}
}
}
}

View File

@ -1,52 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.pipelines;
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.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Handles pipeline close event.
*/
public class PipelineCloseHandler implements EventHandler<PipelineID> {
private static final Logger LOG = LoggerFactory
.getLogger(PipelineCloseHandler.class);
private final PipelineSelector pipelineSelector;
public PipelineCloseHandler(PipelineSelector pipelineSelector) {
this.pipelineSelector = pipelineSelector;
}
@Override
public void onMessage(PipelineID pipelineID, EventPublisher publisher) {
Pipeline pipeline = pipelineSelector.getPipeline(pipelineID);
try {
if (pipeline != null) {
pipelineSelector.finalizePipeline(pipeline);
} else {
LOG.debug("pipeline:{} not found", pipelineID);
}
} catch (Exception e) {
LOG.info("failed to close pipeline:{}", pipelineID, e);
}
}
}

View File

@ -1,171 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.pipelines;
import java.util.ArrayList;
import java.util.LinkedList;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
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.PipelineID;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.atomic.AtomicInteger;
/**
* Manage Ozone pipelines.
*/
public abstract class PipelineManager {
private static final Logger LOG =
LoggerFactory.getLogger(PipelineManager.class);
protected final ArrayList<ActivePipelines> activePipelines;
public PipelineManager() {
activePipelines = new ArrayList<>();
for (ReplicationFactor factor : ReplicationFactor.values()) {
activePipelines.add(factor.ordinal(), new ActivePipelines());
}
}
/**
* List of active pipelines.
*/
public static class ActivePipelines {
private final List<PipelineID> activePipelines;
private final AtomicInteger pipelineIndex;
ActivePipelines() {
activePipelines = new LinkedList<>();
pipelineIndex = new AtomicInteger(0);
}
void addPipeline(PipelineID pipelineID) {
if (!activePipelines.contains(pipelineID)) {
activePipelines.add(pipelineID);
}
}
public void removePipeline(PipelineID pipelineID) {
activePipelines.remove(pipelineID);
}
/**
* Find a Pipeline that is operational.
*
* @return - Pipeline or null
*/
PipelineID findOpenPipeline() {
if (activePipelines.size() == 0) {
LOG.error("No Operational pipelines found. Returning null.");
return null;
}
return activePipelines.get(getNextIndex());
}
/**
* gets the next index of the Pipeline to get.
*
* @return index in the link list to get.
*/
private int getNextIndex() {
return pipelineIndex.incrementAndGet() % activePipelines.size();
}
}
/**
* This function is called by the Container Manager while allocating a new
* container. The client specifies what kind of replication pipeline is
* needed and based on the replication type in the request appropriate
* Interface is invoked.
*
* @param replicationFactor - Replication Factor
* @return a Pipeline.
*/
public synchronized final PipelineID getPipeline(
ReplicationFactor replicationFactor, ReplicationType replicationType) {
PipelineID id =
activePipelines.get(replicationFactor.ordinal()).findOpenPipeline();
if (id != null) {
LOG.debug("re-used pipeline:{} for container with " +
"replicationType:{} replicationFactor:{}",
id, replicationType, replicationFactor);
}
if (id == null) {
LOG.error("Get pipeline call failed. We are not able to find" +
" operational pipeline.");
return null;
} else {
return id;
}
}
void addOpenPipeline(Pipeline pipeline) {
activePipelines.get(pipeline.getFactor().ordinal())
.addPipeline(pipeline.getId());
}
public abstract Pipeline allocatePipeline(
ReplicationFactor replicationFactor);
/**
* Initialize the pipeline.
* TODO: move the initialization to Ozone Client later
*/
public abstract void initializePipeline(Pipeline pipeline) throws IOException;
public void processPipelineReport(Pipeline pipeline, DatanodeDetails dn) {
if (pipeline.addMember(dn)
&&(pipeline.getDatanodes().size() == pipeline.getFactor().getNumber())
&& pipeline.getLifeCycleState() == HddsProtos.LifeCycleState.OPEN) {
addOpenPipeline(pipeline);
}
}
/**
* Creates a pipeline with a specified replication factor and type.
* @param replicationFactor - Replication Factor.
* @param replicationType - Replication Type.
*/
public Pipeline createPipeline(ReplicationFactor replicationFactor,
ReplicationType replicationType) throws IOException {
Pipeline pipeline = allocatePipeline(replicationFactor);
if (pipeline != null) {
LOG.debug("created new pipeline:{} for container with "
+ "replicationType:{} replicationFactor:{}",
pipeline.getId(), replicationType, replicationFactor);
}
return pipeline;
}
/**
* Remove the pipeline from active allocation.
* @param pipeline pipeline to be finalized
*/
public abstract boolean finalizePipeline(Pipeline pipeline);
/**
*
* @param pipeline
*/
public abstract void closePipeline(Pipeline pipeline) throws IOException;
}

View File

@ -1,59 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.pipelines;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.server
.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventHandler;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* Handles Node Reports from datanode.
*/
public class PipelineReportHandler implements
EventHandler<PipelineReportFromDatanode> {
private static final Logger LOGGER = LoggerFactory
.getLogger(PipelineReportHandler.class);
private final PipelineSelector pipelineSelector;
public PipelineReportHandler(PipelineSelector pipelineSelector) {
Preconditions.checkNotNull(pipelineSelector);
this.pipelineSelector = pipelineSelector;
}
@Override
public void onMessage(PipelineReportFromDatanode pipelineReportFromDatanode,
EventPublisher publisher) {
Preconditions.checkNotNull(pipelineReportFromDatanode);
DatanodeDetails dn = pipelineReportFromDatanode.getDatanodeDetails();
PipelineReportsProto pipelineReport =
pipelineReportFromDatanode.getReport();
Preconditions.checkNotNull(dn, "Pipeline Report is "
+ "missing DatanodeDetails.");
LOGGER.trace("Processing pipeline report for dn: {}", dn);
pipelineSelector.processPipelineReport(dn, pipelineReport);
}
}

View File

@ -1,481 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.pipelines;
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.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.ContainerID;
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;
import org.apache.hadoop.hdds.scm.container.placement.algorithms
.SCMContainerPlacementRandom;
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.pipelines.ratis.RatisManagerImpl;
import org.apache.hadoop.hdds.scm.pipelines.standalone.StandaloneManagerImpl;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
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.server.events.EventPublisher;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.lease.Lease;
import org.apache.hadoop.ozone.lease.LeaseException;
import org.apache.hadoop.ozone.lease.LeaseManager;
import org.apache.hadoop.utils.MetadataStore;
import org.apache.hadoop.utils.MetadataStoreBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.File;
import java.io.IOException;
import java.lang.reflect.Constructor;
import java.lang.reflect.InvocationTargetException;
import java.util.HashSet;
import java.util.List;
import java.util.HashMap;
import java.util.Set;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_CHANGE_PIPELINE_STATE;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.FAILED_TO_FIND_ACTIVE_PIPELINE;
import static org.apache.hadoop.hdds.server
.ServerUtils.getOzoneMetaDirPath;
import static org.apache.hadoop.ozone
.OzoneConsts.SCM_PIPELINE_DB;
/**
* Sends the request to the right pipeline manager.
*/
public class PipelineSelector {
private static final Logger LOG =
LoggerFactory.getLogger(PipelineSelector.class);
private final ContainerPlacementPolicy placementPolicy;
private final Map<ReplicationType, PipelineManager> pipelineManagerMap;
private final Configuration conf;
private final EventPublisher eventPublisher;
private final long containerSize;
private final MetadataStore pipelineStore;
private final PipelineStateManager stateManager;
private final NodeManager nodeManager;
private final Map<PipelineID, HashSet<ContainerID>> pipeline2ContainerMap;
private final Map<PipelineID, Pipeline> pipelineMap;
private final LeaseManager<Pipeline> pipelineLeaseManager;
/**
* Constructs a pipeline Selector.
*
* @param nodeManager - node manager
* @param conf - Ozone Config
*/
public PipelineSelector(NodeManager nodeManager, Configuration conf,
EventPublisher eventPublisher, int cacheSizeMB) throws IOException {
this.conf = conf;
this.eventPublisher = eventPublisher;
this.placementPolicy = createContainerPlacementPolicy(nodeManager, conf);
this.containerSize = (long)this.conf.getStorageSize(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT,
StorageUnit.BYTES);
pipelineMap = new ConcurrentHashMap<>();
pipelineManagerMap = new HashMap<>();
pipelineManagerMap.put(ReplicationType.STAND_ALONE,
new StandaloneManagerImpl(nodeManager, placementPolicy,
containerSize));
pipelineManagerMap.put(ReplicationType.RATIS,
new RatisManagerImpl(nodeManager, placementPolicy,
containerSize, conf));
long pipelineCreationLeaseTimeout = conf.getTimeDuration(
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT,
ScmConfigKeys.OZONE_SCM_PIPELINE_CREATION_LEASE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
pipelineLeaseManager = new LeaseManager<>("PipelineCreation",
pipelineCreationLeaseTimeout);
pipelineLeaseManager.start();
stateManager = new PipelineStateManager();
this.nodeManager = nodeManager;
pipeline2ContainerMap = new HashMap<>();
// Write the container name to pipeline mapping.
File metaDir = getOzoneMetaDirPath(conf);
File containerDBPath = new File(metaDir, SCM_PIPELINE_DB);
pipelineStore = MetadataStoreBuilder.newBuilder()
.setConf(conf)
.setDbFile(containerDBPath)
.setCacheSize(cacheSizeMB * OzoneConsts.MB)
.build();
reloadExistingPipelines();
}
private void reloadExistingPipelines() throws IOException {
if (pipelineStore.isEmpty()) {
// Nothing to do just return
return;
}
List<Map.Entry<byte[], byte[]>> range =
pipelineStore.getSequentialRangeKVs(null, Integer.MAX_VALUE, null);
// Transform the values into the pipelines.
// TODO: filter by pipeline state
for (Map.Entry<byte[], byte[]> entry : range) {
Pipeline pipeline = Pipeline.getFromProtoBuf(
HddsProtos.Pipeline.PARSER.parseFrom(entry.getValue()));
Preconditions.checkNotNull(pipeline);
addExistingPipeline(pipeline);
}
}
@VisibleForTesting
public Set<ContainerID> getOpenContainerIDsByPipeline(PipelineID pipelineID) {
return pipeline2ContainerMap.get(pipelineID);
}
public void addContainerToPipeline(PipelineID pipelineID, long containerID) {
pipeline2ContainerMap.get(pipelineID)
.add(ContainerID.valueof(containerID));
}
public void removeContainerFromPipeline(PipelineID pipelineID,
long containerID) throws IOException {
pipeline2ContainerMap.get(pipelineID)
.remove(ContainerID.valueof(containerID));
closePipelineIfNoOpenContainers(pipelineMap.get(pipelineID));
}
/**
* Translates a list of nodes, ordered such that the first is the leader, into
* a corresponding {@link Pipeline} object.
*
* @param nodes - list of datanodes on which we will allocate the container.
* The first of the list will be the leader node.
* @return pipeline corresponding to nodes
*/
public static Pipeline newPipelineFromNodes(
List<DatanodeDetails> nodes, ReplicationType replicationType,
ReplicationFactor replicationFactor, PipelineID id) {
Preconditions.checkNotNull(nodes);
Preconditions.checkArgument(nodes.size() > 0);
String leaderId = nodes.get(0).getUuidString();
// A new pipeline always starts in allocated state
Pipeline pipeline = new Pipeline(leaderId, LifeCycleState.ALLOCATED,
replicationType, replicationFactor, id);
for (DatanodeDetails node : nodes) {
pipeline.addMember(node);
}
return pipeline;
}
/**
* Create pluggable container placement policy implementation instance.
*
* @param nodeManager - SCM node manager.
* @param conf - configuration.
* @return SCM container placement policy implementation instance.
*/
@SuppressWarnings("unchecked")
private static ContainerPlacementPolicy createContainerPlacementPolicy(
final NodeManager nodeManager, final Configuration conf) {
Class<? extends ContainerPlacementPolicy> implClass =
(Class<? extends ContainerPlacementPolicy>) conf.getClass(
ScmConfigKeys.OZONE_SCM_CONTAINER_PLACEMENT_IMPL_KEY,
SCMContainerPlacementRandom.class);
try {
Constructor<? extends ContainerPlacementPolicy> ctor =
implClass.getDeclaredConstructor(NodeManager.class,
Configuration.class);
return ctor.newInstance(nodeManager, conf);
} catch (RuntimeException e) {
throw e;
} catch (InvocationTargetException e) {
throw new RuntimeException(implClass.getName()
+ " could not be constructed.", e.getCause());
} catch (Exception e) {
LOG.error("Unhandled exception occurred, Placement policy will not be " +
"functional.");
throw new IllegalArgumentException("Unable to load " +
"ContainerPlacementPolicy", e);
}
}
/**
* This function is called by the Container Manager while allocating a new
* container. The client specifies what kind of replication pipeline is needed
* and based on the replication type in the request appropriate Interface is
* invoked.
*/
public Pipeline getReplicationPipeline(ReplicationType replicationType,
HddsProtos.ReplicationFactor replicationFactor)
throws IOException {
PipelineManager manager = pipelineManagerMap.get(replicationType);
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Getting replication pipeline forReplicationType {} :" +
" ReplicationFactor {}", replicationType.toString(),
replicationFactor.toString());
/**
* In the Ozone world, we have a very simple policy.
*
* 1. Try to create a pipeline if there are enough free nodes.
*
* 2. This allows all nodes to part of a pipeline quickly.
*
* 3. if there are not enough free nodes, return already allocated pipeline
* in a round-robin fashion.
*
* TODO: Might have to come up with a better algorithm than this.
* Create a new placement policy that returns pipelines in round robin
* fashion.
*/
Pipeline pipeline =
manager.createPipeline(replicationFactor, replicationType);
if (pipeline == null) {
// try to return a pipeline from already allocated pipelines
PipelineID pipelineId =
manager.getPipeline(replicationFactor, replicationType);
if (pipelineId == null) {
throw new SCMException(FAILED_TO_FIND_ACTIVE_PIPELINE);
}
pipeline = pipelineMap.get(pipelineId);
Preconditions.checkArgument(pipeline.getLifeCycleState() ==
LifeCycleState.OPEN);
} else {
pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
pipeline.getProtobufMessage().toByteArray());
// if a new pipeline is created, initialize its state machine
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CREATE);
//TODO: move the initialization of pipeline to Ozone Client
manager.initializePipeline(pipeline);
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CREATED);
}
return pipeline;
}
/**
* This function to return pipeline for given pipeline id.
*/
public Pipeline getPipeline(PipelineID pipelineID) {
return pipelineMap.get(pipelineID);
}
/**
* Finalize a given pipeline.
*/
public void finalizePipeline(Pipeline pipeline) throws IOException {
PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
if (pipeline.getLifeCycleState() == LifeCycleState.CLOSING ||
pipeline.getLifeCycleState() == LifeCycleState.CLOSED) {
LOG.debug("pipeline:{} already in closing state, skipping",
pipeline.getId());
// already in closing/closed state
return;
}
// Remove the pipeline from active allocation
if (manager.finalizePipeline(pipeline)) {
LOG.info("Finalizing pipeline. pipelineID: {}", pipeline.getId());
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.FINALIZE);
closePipelineIfNoOpenContainers(pipeline);
}
}
/**
* Close a given pipeline.
*/
private void closePipelineIfNoOpenContainers(Pipeline pipeline)
throws IOException {
if (pipeline.getLifeCycleState() != LifeCycleState.CLOSING) {
return;
}
HashSet<ContainerID> containerIDS =
pipeline2ContainerMap.get(pipeline.getId());
if (containerIDS.size() == 0) {
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.CLOSE);
LOG.info("Closing pipeline. pipelineID: {}", pipeline.getId());
}
}
/**
* Close a given pipeline.
*/
private void closePipeline(Pipeline pipeline) throws IOException {
PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Closing pipeline. pipelineID: {}", pipeline.getId());
HashSet<ContainerID> containers =
pipeline2ContainerMap.get(pipeline.getId());
Preconditions.checkArgument(containers.size() == 0);
manager.closePipeline(pipeline);
}
/**
* Add to a given pipeline.
*/
private void addOpenPipeline(Pipeline pipeline) {
PipelineManager manager = pipelineManagerMap.get(pipeline.getType());
Preconditions.checkNotNull(manager, "Found invalid pipeline manager");
LOG.debug("Adding Open pipeline. pipelineID: {}", pipeline.getId());
manager.addOpenPipeline(pipeline);
}
private void closeContainersByPipeline(Pipeline pipeline) {
HashSet<ContainerID> containers =
pipeline2ContainerMap.get(pipeline.getId());
for (ContainerID id : containers) {
eventPublisher.fireEvent(SCMEvents.CLOSE_CONTAINER, id);
}
}
private void addExistingPipeline(Pipeline pipeline) throws IOException {
LifeCycleState state = pipeline.getLifeCycleState();
switch (state) {
case ALLOCATED:
// a pipeline in allocated state is only present in SCM and does not exist
// on datanode, on SCM restart, this pipeline can be ignored.
break;
case CREATING:
case OPEN:
case CLOSING:
//TODO: process pipeline report and move pipeline to active queue
// when all the nodes have reported.
pipelineMap.put(pipeline.getId(), pipeline);
pipeline2ContainerMap.put(pipeline.getId(), new HashSet<>());
nodeManager.addPipeline(pipeline);
// reset the datanodes in the pipeline
// they will be reset on
pipeline.resetPipeline();
break;
case CLOSED:
// if the pipeline is in closed state, nothing to do.
break;
default:
throw new IOException("invalid pipeline state:" + state);
}
}
public void handleStaleNode(DatanodeDetails dn) {
Set<PipelineID> pipelineIDs = nodeManager.getPipelineByDnID(dn.getUuid());
for (PipelineID id : pipelineIDs) {
LOG.info("closing pipeline {}.", id);
eventPublisher.fireEvent(SCMEvents.PIPELINE_CLOSE, id);
}
}
void processPipelineReport(DatanodeDetails dn,
PipelineReportsProto pipelineReport) {
Set<PipelineID> reportedPipelines = new HashSet<>();
pipelineReport.getPipelineReportList().
forEach(p ->
reportedPipelines.add(
processPipelineReport(p.getPipelineID(), dn)));
//TODO: handle missing pipelines and new pipelines later
}
private PipelineID processPipelineReport(
HddsProtos.PipelineID id, DatanodeDetails dn) {
PipelineID pipelineID = PipelineID.getFromProtobuf(id);
Pipeline pipeline = pipelineMap.get(pipelineID);
if (pipeline != null) {
pipelineManagerMap.get(pipeline.getType())
.processPipelineReport(pipeline, dn);
}
return pipelineID;
}
/**
* Update the Pipeline State to the next state.
*
* @param pipeline - Pipeline
* @param event - LifeCycle Event
* @throws SCMException on Failure.
*/
public void updatePipelineState(Pipeline pipeline,
HddsProtos.LifeCycleEvent event) throws IOException {
try {
switch (event) {
case CREATE:
pipelineMap.put(pipeline.getId(), pipeline);
pipeline2ContainerMap.put(pipeline.getId(), new HashSet<>());
nodeManager.addPipeline(pipeline);
// Acquire lease on pipeline
Lease<Pipeline> pipelineLease = pipelineLeaseManager.acquire(pipeline);
// Register callback to be executed in case of timeout
pipelineLease.registerCallBack(() -> {
updatePipelineState(pipeline, HddsProtos.LifeCycleEvent.TIMEOUT);
return null;
});
break;
case CREATED:
// Release the lease on pipeline
pipelineLeaseManager.release(pipeline);
addOpenPipeline(pipeline);
break;
case FINALIZE:
closeContainersByPipeline(pipeline);
break;
case CLOSE:
case TIMEOUT:
closePipeline(pipeline);
pipeline2ContainerMap.remove(pipeline.getId());
nodeManager.removePipeline(pipeline);
pipelineMap.remove(pipeline.getId());
break;
default:
throw new SCMException("Unsupported pipeline LifeCycleEvent.",
FAILED_TO_CHANGE_PIPELINE_STATE);
}
stateManager.updatePipelineState(pipeline, event);
pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
pipeline.getProtobufMessage().toByteArray());
} catch (LeaseException e) {
throw new IOException("Lease Exception.", e);
}
}
public void shutdown() throws IOException {
if (pipelineLeaseManager != null) {
pipelineLeaseManager.shutdown();
}
if (pipelineStore != null) {
pipelineStore.close();
}
}
}

View File

@ -1,136 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.pipelines;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.ozone.common.statemachine
.InvalidStateTransitionException;
import org.apache.hadoop.ozone.common.statemachine.StateMachine;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.Set;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes
.FAILED_TO_CHANGE_PIPELINE_STATE;
/**
* Manages Pipeline states.
*/
public class PipelineStateManager {
private static final Logger LOG =
LoggerFactory.getLogger(PipelineStateManager.class);
private final StateMachine<HddsProtos.LifeCycleState,
HddsProtos.LifeCycleEvent> stateMachine;
PipelineStateManager() {
// Initialize the container state machine.
Set<HddsProtos.LifeCycleState> finalStates = new HashSet<>();
// These are the steady states of a container.
finalStates.add(HddsProtos.LifeCycleState.OPEN);
finalStates.add(HddsProtos.LifeCycleState.CLOSED);
this.stateMachine = new StateMachine<>(HddsProtos.LifeCycleState.ALLOCATED,
finalStates);
initializeStateMachine();
}
/**
* Event and State Transition Mapping.
*
* State: ALLOCATED ---------------> CREATING
* Event: CREATE
*
* State: CREATING ---------------> OPEN
* Event: CREATED
*
* State: OPEN ---------------> CLOSING
* Event: FINALIZE
*
* State: CLOSING ---------------> CLOSED
* Event: CLOSE
*
* State: CREATING ---------------> CLOSED
* Event: TIMEOUT
*
*
* Container State Flow:
*
* [ALLOCATED]---->[CREATING]------>[OPEN]-------->[CLOSING]
* (CREATE) | (CREATED) (FINALIZE) |
* | |
* | |
* |(TIMEOUT) |(CLOSE)
* | |
* +--------> [CLOSED] <--------+
*/
private void initializeStateMachine() {
stateMachine.addTransition(HddsProtos.LifeCycleState.ALLOCATED,
HddsProtos.LifeCycleState.CREATING,
HddsProtos.LifeCycleEvent.CREATE);
stateMachine.addTransition(HddsProtos.LifeCycleState.CREATING,
HddsProtos.LifeCycleState.OPEN,
HddsProtos.LifeCycleEvent.CREATED);
stateMachine.addTransition(HddsProtos.LifeCycleState.OPEN,
HddsProtos.LifeCycleState.CLOSING,
HddsProtos.LifeCycleEvent.FINALIZE);
stateMachine.addTransition(HddsProtos.LifeCycleState.CLOSING,
HddsProtos.LifeCycleState.CLOSED,
HddsProtos.LifeCycleEvent.CLOSE);
stateMachine.addTransition(HddsProtos.LifeCycleState.CREATING,
HddsProtos.LifeCycleState.CLOSED,
HddsProtos.LifeCycleEvent.TIMEOUT);
}
/**
* Update the Pipeline State to the next state.
*
* @param pipeline - Pipeline
* @param event - LifeCycle Event
* @throws SCMException on Failure.
*/
public void updatePipelineState(Pipeline pipeline,
HddsProtos.LifeCycleEvent event) throws IOException {
HddsProtos.LifeCycleState newState;
try {
newState = stateMachine.getNextState(pipeline.getLifeCycleState(), event);
} catch (InvalidStateTransitionException ex) {
String error = String.format("Failed to update pipeline state %s, " +
"reason: invalid state transition from state: %s upon " +
"event: %s.",
pipeline.getId(), pipeline.getLifeCycleState(), event);
LOG.error(error);
throw new SCMException(error, FAILED_TO_CHANGE_PIPELINE_STATE);
}
// This is a post condition after executing getNextState.
Preconditions.checkNotNull(newState);
Preconditions.checkNotNull(pipeline);
pipeline.setLifeCycleState(newState);
}
}

View File

@ -1,38 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.pipelines;
/**
Ozone supports the notion of different kind of pipelines.
That means that we can have a replication pipeline build on
Ratis, Standalone or some other protocol. All Pipeline managers
the entities in charge of pipelines reside in the package.
Here is the high level Arch.
1. A pipeline selector class is instantiated in the Container manager class.
2. A client when creating a container -- will specify what kind of
replication type it wants to use. We support 2 types now, Ratis and StandAlone.
3. Based on the replication type, the pipeline selector class asks the
corresponding pipeline manager for a pipeline.
4. We have supported the ability for clients to specify a set of nodes in
the pipeline or rely in the pipeline manager to select the datanodes if they
are not specified.
*/

View File

@ -1,129 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p/>
* http://www.apache.org/licenses/LICENSE-2.0
* <p/>
* 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.pipelines.ratis;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.scm.XceiverClientRatis;
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;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
/**
* Implementation of {@link PipelineManager}.
*
* TODO : Introduce a state machine.
*/
public class RatisManagerImpl extends PipelineManager {
private static final Logger LOG =
LoggerFactory.getLogger(RatisManagerImpl.class);
private final Configuration conf;
private final NodeManager nodeManager;
private final Set<DatanodeDetails> ratisMembers;
/**
* Constructs a Ratis Pipeline Manager.
*
* @param nodeManager
*/
public RatisManagerImpl(NodeManager nodeManager,
ContainerPlacementPolicy placementPolicy, long size, Configuration conf) {
super();
this.conf = conf;
this.nodeManager = nodeManager;
ratisMembers = new HashSet<>();
}
/**
* Allocates a new ratis Pipeline from the free nodes.
*
* @param factor - One or Three
* @return Pipeline.
*/
public Pipeline allocatePipeline(ReplicationFactor factor) {
List<DatanodeDetails> newNodesList = new LinkedList<>();
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
//TODO: Add Raft State to the Nodes, so we can query and skip nodes from
// data from datanode instead of maintaining a set.
for (DatanodeDetails datanode : datanodes) {
Preconditions.checkNotNull(datanode);
if (!ratisMembers.contains(datanode)) {
newNodesList.add(datanode);
if (newNodesList.size() == factor.getNumber()) {
// once a datanode has been added to a pipeline, exclude it from
// further allocations
ratisMembers.addAll(newNodesList);
PipelineID pipelineID = PipelineID.randomId();
LOG.info("Allocating a new ratis pipeline of size: {} id: {}",
factor.getNumber(), pipelineID);
return PipelineSelector.newPipelineFromNodes(newNodesList,
ReplicationType.RATIS, factor, pipelineID);
}
}
}
return null;
}
public void initializePipeline(Pipeline pipeline) throws IOException {
//TODO:move the initialization from SCM to client
try (XceiverClientRatis client =
XceiverClientRatis.newXceiverClientRatis(pipeline, conf)) {
client.createPipeline();
}
}
public void processPipelineReport(Pipeline pipeline, DatanodeDetails dn) {
super.processPipelineReport(pipeline, dn);
ratisMembers.add(dn);
}
public synchronized boolean finalizePipeline(Pipeline pipeline) {
activePipelines.get(pipeline.getFactor().ordinal())
.removePipeline(pipeline.getId());
return true;
}
/**
* Close the pipeline.
*/
public void closePipeline(Pipeline pipeline) throws IOException {
try (XceiverClientRatis client =
XceiverClientRatis.newXceiverClientRatis(pipeline, conf)) {
client.destroyPipeline();
}
for (DatanodeDetails node : pipeline.getMachines()) {
// A node should always be the in ratis members list.
Preconditions.checkArgument(ratisMembers.remove(node));
}
}
}

View File

@ -1,18 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.pipelines.ratis;

View File

@ -1,122 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.pipelines.standalone;
import com.google.common.base.Preconditions;
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;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineManager;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import java.io.IOException;
import java.util.HashSet;
import java.util.LinkedList;
import java.util.List;
import java.util.Set;
/**
* Standalone Manager Impl to prove that pluggable interface
* works with current tests.
*/
public class StandaloneManagerImpl extends PipelineManager {
private static final Logger LOG =
LoggerFactory.getLogger(StandaloneManagerImpl.class);
private final NodeManager nodeManager;
private final ContainerPlacementPolicy placementPolicy;
private final long containerSize;
private final Set<DatanodeDetails> standAloneMembers;
/**
* Constructor for Standalone Node Manager Impl.
* @param nodeManager - Node Manager.
* @param placementPolicy - Placement Policy
* @param containerSize - Container Size.
*/
public StandaloneManagerImpl(NodeManager nodeManager,
ContainerPlacementPolicy placementPolicy, long containerSize) {
super();
this.nodeManager = nodeManager;
this.placementPolicy = placementPolicy;
this.containerSize = containerSize;
this.standAloneMembers = new HashSet<>();
}
/**
* Allocates a new standalone Pipeline from the free nodes.
*
* @param factor - One
* @return Pipeline.
*/
public Pipeline allocatePipeline(ReplicationFactor factor) {
List<DatanodeDetails> newNodesList = new LinkedList<>();
List<DatanodeDetails> datanodes = nodeManager.getNodes(NodeState.HEALTHY);
for (DatanodeDetails datanode : datanodes) {
Preconditions.checkNotNull(datanode);
if (!standAloneMembers.contains(datanode)) {
newNodesList.add(datanode);
if (newNodesList.size() == factor.getNumber()) {
// once a datanode has been added to a pipeline, exclude it from
// further allocations
standAloneMembers.addAll(newNodesList);
// Standalone pipeline use node id as pipeline
PipelineID pipelineID =
PipelineID.valueOf(newNodesList.get(0).getUuid());
LOG.info("Allocating a new standalone pipeline of size: {} id: {}",
factor.getNumber(), pipelineID);
return PipelineSelector.newPipelineFromNodes(newNodesList,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE, pipelineID);
}
}
}
return null;
}
public void initializePipeline(Pipeline pipeline) {
// Nothing to be done for standalone pipeline
}
public void processPipelineReport(Pipeline pipeline, DatanodeDetails dn) {
super.processPipelineReport(pipeline, dn);
standAloneMembers.add(dn);
}
public synchronized boolean finalizePipeline(Pipeline pipeline) {
activePipelines.get(pipeline.getFactor().ordinal())
.removePipeline(pipeline.getId());
return false;
}
/**
* Close the pipeline.
*/
public void closePipeline(Pipeline pipeline) throws IOException {
for (DatanodeDetails node : pipeline.getMachines()) {
// A node should always be the in standalone members list.
Preconditions.checkArgument(standAloneMembers.remove(node));
}
}
}

View File

@ -1,18 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.hdds.scm.pipelines.standalone;

View File

@ -37,7 +37,7 @@
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.ContainerInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes;
import org.apache.hadoop.hdds.scm.protocol.StorageContainerLocationProtocol;

View File

@ -63,9 +63,10 @@
import org.apache.hadoop.hdds.scm.node.NodeReportHandler;
import org.apache.hadoop.hdds.scm.node.SCMNodeManager;
import org.apache.hadoop.hdds.scm.node.StaleNodeHandler;
import org.apache.hadoop.hdds.scm.pipelines.PipelineCloseHandler;
import org.apache.hadoop.hdds.scm.pipelines.PipelineActionEventHandler;
import org.apache.hadoop.hdds.scm.pipelines.PipelineReportHandler;
import org.apache.hadoop.hdds.scm.pipeline.PipelineActionHandler;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineReportHandler;
import org.apache.hadoop.hdds.server.ServiceRuntimeInfoImpl;
import org.apache.hadoop.hdds.server.events.EventPublisher;
import org.apache.hadoop.hdds.server.events.EventQueue;
@ -149,6 +150,7 @@ public final class StorageContainerManager extends ServiceRuntimeInfoImpl
* State Managers of SCM.
*/
private final NodeManager scmNodeManager;
private final PipelineManager pipelineManager;
private final ContainerManager containerManager;
private final BlockManager scmBlockManager;
private final SCMStorage scmStorage;
@ -201,8 +203,9 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
scmNodeManager = new SCMNodeManager(
conf, scmStorage.getClusterID(), this, eventQueue);
pipelineManager = new SCMPipelineManager(conf, scmNodeManager, eventQueue);
containerManager = new SCMContainerManager(
conf, scmNodeManager, eventQueue);
conf, scmNodeManager, pipelineManager, eventQueue);
scmBlockManager = new BlockManagerImpl(
conf, scmNodeManager, containerManager, eventQueue);
@ -213,14 +216,13 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
NodeReportHandler nodeReportHandler =
new NodeReportHandler(scmNodeManager);
PipelineReportHandler pipelineReportHandler =
new PipelineReportHandler(
containerManager.getPipelineSelector());
new PipelineReportHandler(pipelineManager, conf);
CommandStatusReportHandler cmdStatusReportHandler =
new CommandStatusReportHandler();
NewNodeHandler newNodeHandler = new NewNodeHandler(scmNodeManager);
StaleNodeHandler staleNodeHandler =
new StaleNodeHandler(containerManager.getPipelineSelector());
new StaleNodeHandler(scmNodeManager, pipelineManager);
DeadNodeHandler deadNodeHandler = new DeadNodeHandler(scmNodeManager,
containerManager);
ContainerActionsHandler actionsHandler = new ContainerActionsHandler();
@ -231,11 +233,8 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
new ContainerReportHandler(containerManager, scmNodeManager,
replicationStatus);
PipelineActionEventHandler pipelineActionEventHandler =
new PipelineActionEventHandler();
PipelineCloseHandler pipelineCloseHandler =
new PipelineCloseHandler(containerManager.getPipelineSelector());
PipelineActionHandler pipelineActionHandler =
new PipelineActionHandler(pipelineManager);
long watcherTimeout =
conf.getTimeDuration(ScmConfigKeys.HDDS_SCM_WATCHER_TIMEOUT,
@ -294,10 +293,7 @@ private StorageContainerManager(OzoneConfiguration conf) throws IOException {
.addHandler(SCMEvents.PENDING_DELETE_STATUS, pendingDeleteHandler);
eventQueue.addHandler(SCMEvents.DELETE_BLOCK_STATUS,
(DeletedBlockLogImpl) scmBlockManager.getDeletedBlockLog());
eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS,
pipelineActionEventHandler);
eventQueue.addHandler(SCMEvents.PIPELINE_CLOSE, pipelineCloseHandler);
eventQueue.addHandler(SCMEvents.PIPELINE_ACTIONS, pipelineActionHandler);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, clientProtocolServer);
eventQueue.addHandler(SCMEvents.PIPELINE_REPORT, pipelineReportHandler);
@ -771,6 +767,7 @@ public void stop() {
LOG.error("SCM Event Queue stop failed", ex);
}
IOUtils.cleanupWithLogger(LOG, containerManager);
IOUtils.cleanupWithLogger(LOG, pipelineManager);
}
/**
@ -815,6 +812,16 @@ public NodeManager getScmNodeManager() {
return scmNodeManager;
}
/**
* Returns pipeline manager.
*
* @return - Pipeline Manager
*/
@VisibleForTesting
public PipelineManager getPipelineManager() {
return pipelineManager;
}
@VisibleForTesting
public BlockManager getScmBlockManager() {
return scmBlockManager;

View File

@ -27,6 +27,8 @@
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.scm.server.SCMStorage;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.hdds.server.events.EventHandler;
@ -58,6 +60,7 @@
public class TestBlockManager implements EventHandler<Boolean> {
private static SCMContainerManager mapping;
private static MockNodeManager nodeManager;
private static PipelineManager pipelineManager;
private static BlockManagerImpl blockManager;
private static File testDir;
private final static long DEFAULT_BLOCK_SIZE = 128 * MB;
@ -83,7 +86,10 @@ public void setUp() throws Exception {
throw new IOException("Unable to create test directory path");
}
nodeManager = new MockNodeManager(true, 10);
mapping = new SCMContainerManager(conf, nodeManager, eventQueue);
pipelineManager =
new SCMPipelineManager(conf, nodeManager, eventQueue);
mapping = new SCMContainerManager(conf, nodeManager, pipelineManager,
eventQueue);
blockManager = new BlockManagerImpl(conf,
nodeManager, mapping, eventQueue);
eventQueue.addHandler(SCMEvents.CHILL_MODE_STATUS, blockManager);
@ -101,6 +107,7 @@ public void setUp() throws Exception {
@After
public void cleanup() throws IOException {
blockManager.close();
pipelineManager.close();
mapping.close();
FileUtil.fullyDelete(testDir);
}

View File

@ -24,12 +24,11 @@
import org.apache.hadoop.hdds.scm.container.ContainerManager;
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;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdfs.DFSUtil;
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
@ -102,12 +101,13 @@ private void setupContainerManager() throws IOException {
ContainerInfo containerInfo =
new ContainerInfo.Builder().setContainerID(1).build();
Pipeline pipeline =
new Pipeline(null, LifeCycleState.CLOSED,
ReplicationType.RATIS, ReplicationFactor.THREE, null);
pipeline.addMember(dnList.get(0));
pipeline.addMember(dnList.get(1));
pipeline.addMember(dnList.get(2));
Pipeline pipeline = Pipeline.newBuilder()
.setType(ReplicationType.RATIS)
.setFactor(ReplicationFactor.THREE)
.setState(Pipeline.PipelineState.CLOSED)
.setId(PipelineID.randomId())
.setNodes(dnList)
.build();
ContainerWithPipeline containerWithPipeline =
new ContainerWithPipeline(containerInfo, pipeline);
when(containerManager.getContainerWithPipeline(anyObject()))
@ -383,11 +383,15 @@ public void testDeletedBlockTransactions() throws IOException {
private void mockContainerInfo(long containerID, DatanodeDetails dd)
throws IOException {
Pipeline pipeline =
new Pipeline("fake", LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
PipelineID.randomId());
pipeline.addMember(dd);
List<DatanodeDetails> dns = new ArrayList<>();
dns.add(dd);
Pipeline pipeline = Pipeline.newBuilder()
.setType(ReplicationType.STAND_ALONE)
.setFactor(ReplicationFactor.ONE)
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setNodes(dns)
.build();
ContainerInfo.Builder builder = new ContainerInfo.Builder();
builder.setPipelineID(pipeline.getId())

View File

@ -19,8 +19,8 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.TestUtils;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;

View File

@ -25,6 +25,8 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers
.ContainerWithPipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
@ -65,8 +67,11 @@ public static void setUp() throws Exception {
configuration
.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
nodeManager = new MockNodeManager(true, 10);
containerManager = new SCMContainerManager(configuration, nodeManager,
new EventQueue());
PipelineManager pipelineManager =
new SCMPipelineManager(configuration, nodeManager, eventQueue);
containerManager = new
SCMContainerManager(configuration, nodeManager,
pipelineManager, new EventQueue());
eventQueue = new EventQueue();
eventQueue.addHandler(CLOSE_CONTAINER,
new CloseContainerEventHandler(containerManager));
@ -110,11 +115,12 @@ public void testCloseContainerEventWithValidContainers() throws IOException {
GenericTestUtils.LogCapturer logCapturer = GenericTestUtils.LogCapturer
.captureLogs(CloseContainerEventHandler.LOG);
ContainerWithPipeline containerWithPipeline = containerManager
.allocateContainer(HddsProtos.ReplicationType.STAND_ALONE,
.allocateContainer(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.ONE, "ozone");
ContainerID id = new ContainerID(
containerWithPipeline.getContainerInfo().getContainerID());
DatanodeDetails datanode = containerWithPipeline.getPipeline().getLeader();
DatanodeDetails datanode =
containerWithPipeline.getPipeline().getFirstNode();
int closeCount = nodeManager.getCommandCount(datanode);
eventQueue.fireEvent(CLOSE_CONTAINER, id);
eventQueue.processAll(1000);
@ -149,13 +155,13 @@ public void testCloseContainerEventWithRatis() throws IOException {
eventQueue.processAll(1000);
int i = 0;
for (DatanodeDetails details : containerWithPipeline.getPipeline()
.getMachines()) {
.getNodes()) {
closeCount[i] = nodeManager.getCommandCount(details);
i++;
}
i = 0;
for (DatanodeDetails details : containerWithPipeline.getPipeline()
.getMachines()) {
.getNodes()) {
Assert.assertEquals(closeCount[i], nodeManager.getCommandCount(details));
i++;
}
@ -166,7 +172,7 @@ public void testCloseContainerEventWithRatis() throws IOException {
i = 0;
// Make sure close is queued for each datanode on the pipeline
for (DatanodeDetails details : containerWithPipeline.getPipeline()
.getMachines()) {
.getNodes()) {
Assert.assertEquals(closeCount[i] + 1,
nodeManager.getCommandCount(details));
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSING,

View File

@ -32,6 +32,8 @@
.ReplicationActivityStatus;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationRequest;
import org.apache.hadoop.hdds.scm.node.NodeManager;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher
.ContainerReportFromDatanode;
import org.apache.hadoop.hdds.server.events.Event;
@ -73,8 +75,11 @@ public void test() throws IOException {
//GIVEN
OzoneConfiguration conf = new OzoneConfiguration();
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir);
EventQueue eventQueue = new EventQueue();
PipelineManager pipelineManager =
new SCMPipelineManager(conf, nodeManager, eventQueue);
SCMContainerManager containerManager = new SCMContainerManager(
conf, nodeManager, new EventQueue());
conf, nodeManager, pipelineManager, eventQueue);
ReplicationActivityStatus replicationActivityStatus =
new ReplicationActivityStatus();

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.hdds.scm.container;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Set;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
@ -25,9 +26,10 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.TestUtils;
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.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
@ -102,19 +104,20 @@ private void addReplica(ContainerInfo cont, DatanodeDetails node)
private ContainerInfo allocateContainer() throws IOException {
PipelineSelector pipelineSelector = Mockito.mock(PipelineSelector.class);
PipelineManager pipelineManager = Mockito.mock(SCMPipelineManager.class);
Pipeline pipeline = new Pipeline("leader", HddsProtos.LifeCycleState.CLOSED,
Pipeline pipeline =
Pipeline.newBuilder().setState(Pipeline.PipelineState.CLOSED)
.setId(PipelineID.randomId())
.setType(HddsProtos.ReplicationType.STAND_ALONE)
.setFactor(HddsProtos.ReplicationFactor.THREE)
.setNodes(new ArrayList<>()).build();
when(pipelineManager.createPipeline(HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
return containerStateManager.allocateContainer(pipelineManager,
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

@ -24,13 +24,15 @@
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
@ -59,6 +61,7 @@
public class TestSCMContainerManager {
private static SCMContainerManager containerManager;
private static MockNodeManager nodeManager;
private static PipelineManager pipelineManager;
private static File testDir;
private static XceiverClientManager xceiverClientManager;
private static String containerOwner = "OZONE";
@ -85,8 +88,10 @@ public static void setUp() throws Exception {
throw new IOException("Unable to create test directory path");
}
nodeManager = new MockNodeManager(true, 10);
pipelineManager =
new SCMPipelineManager(conf, nodeManager, new EventQueue());
containerManager = new SCMContainerManager(conf, nodeManager,
new EventQueue());
pipelineManager, new EventQueue());
xceiverClientManager = new XceiverClientManager(conf);
random = new Random();
}
@ -96,6 +101,9 @@ public static void cleanup() throws IOException {
if(containerManager != null) {
containerManager.close();
}
if (pipelineManager != null) {
pipelineManager.close();
}
FileUtil.fullyDelete(testDir);
}
@ -130,7 +138,7 @@ public void testallocateContainerDistributesAllocation() throws Exception {
Assert.assertNotNull(containerInfo);
Assert.assertNotNull(containerInfo.getPipeline());
pipelineList.add(containerInfo.getPipeline().getLeader()
pipelineList.add(containerInfo.getPipeline().getFirstNode()
.getUuid());
}
Assert.assertTrue(pipelineList.size() > 5);
@ -145,8 +153,8 @@ public void testGetContainer() throws IOException {
Pipeline pipeline = containerInfo.getPipeline();
Assert.assertNotNull(pipeline);
Pipeline newPipeline = containerInfo.getPipeline();
Assert.assertEquals(pipeline.getLeader().getUuid(),
newPipeline.getLeader().getUuid());
Assert.assertEquals(pipeline.getFirstNode().getUuid(),
newPipeline.getFirstNode().getUuid());
}
@Test
@ -191,15 +199,15 @@ public void testGetContainerWithPipeline() throws Exception {
contInfo = containerManager.getContainer(contInfo.containerID());
Assert.assertEquals(contInfo.getState(), LifeCycleState.CLOSED);
Pipeline pipeline = containerWithPipeline.getPipeline();
containerManager.getPipelineSelector().finalizePipeline(pipeline);
pipelineManager.finalizePipeline(pipeline.getId());
ContainerWithPipeline containerWithPipeline2 = containerManager
.getContainerWithPipeline(contInfo.containerID());
pipeline = containerWithPipeline2.getPipeline();
Assert.assertNotEquals(containerWithPipeline, containerWithPipeline2);
Assert.assertNotNull("Pipeline should not be null", pipeline);
Assert.assertTrue(pipeline.getDatanodeHosts().contains(dn1.getHostName()));
Assert.assertTrue(pipeline.getDatanodeHosts().contains(dn2.getHostName()));
Assert.assertTrue(pipeline.getNodes().contains(dn1));
Assert.assertTrue(pipeline.getNodes().contains(dn2));
}
@Test

View File

@ -20,23 +20,22 @@
import java.util.ArrayList;
import java.util.Arrays;
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;
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.ReplicateContainerCommandProto;
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.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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.replication.ReplicationManager.ReplicationRequestToRepeat;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
@ -227,18 +226,16 @@ public void testCommandWatcher() throws InterruptedException, IOException {
public static Pipeline createPipeline(Iterable<DatanodeDetails> ids)
throws IOException {
Objects.requireNonNull(ids, "ids == null");
final Iterator<DatanodeDetails> i = ids.iterator();
Preconditions.checkArgument(i.hasNext());
final DatanodeDetails leader = i.next();
final Pipeline pipeline =
new Pipeline(leader.getUuidString(), LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
PipelineID.randomId());
pipeline.addMember(leader);
while (i.hasNext()) {
pipeline.addMember(i.next());
}
return pipeline;
Preconditions.checkArgument(ids.iterator().hasNext());
List<DatanodeDetails> dns = new ArrayList<>();
ids.forEach(dns::add);
return Pipeline.newBuilder()
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setType(HddsProtos.ReplicationType.STAND_ALONE)
.setFactor(ReplicationFactor.ONE)
.setNodes(dns)
.build();
}
}

View File

@ -33,6 +33,8 @@
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.events.SCMEvents;
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -102,7 +104,9 @@ SCMContainerManager createContainerManager(Configuration config,
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,
PipelineManager pipelineManager =
new SCMPipelineManager(config, scmNodeManager, eventQueue);
return new SCMContainerManager(config, scmNodeManager, pipelineManager,
eventQueue);
}
@ -156,7 +160,7 @@ public void testContainerPlacementCapacity() throws IOException,
xceiverClientManager.getType(),
xceiverClientManager.getFactor(), "OZONE");
assertEquals(xceiverClientManager.getFactor().getNumber(),
containerWithPipeline.getPipeline().getMachines().size());
containerWithPipeline.getPipeline().getNodes().size());
} finally {
IOUtils.closeQuietly(containerManager);
IOUtils.closeQuietly(nodeManager);

View File

@ -42,6 +42,8 @@
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.pipeline.PipelineManager;
import org.apache.hadoop.hdds.scm.pipeline.SCMPipelineManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.NodeReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventPublisher;
@ -75,7 +77,10 @@ public void setup() throws IOException {
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, storageDir);
eventQueue = new EventQueue();
nodeManager = new SCMNodeManager(conf, "cluster1", null, eventQueue);
containerManager = new SCMContainerManager(conf, nodeManager, eventQueue);
PipelineManager pipelineManager =
new SCMPipelineManager(conf, nodeManager, eventQueue);
containerManager = new SCMContainerManager(conf, nodeManager,
pipelineManager, eventQueue);
deadNodeHandler = new DeadNodeHandler(nodeManager, containerManager);
eventQueue.addHandler(SCMEvents.DEAD_NODE, deadNodeHandler);
publisher = Mockito.mock(EventPublisher.class);

View File

@ -21,7 +21,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;

View File

@ -51,7 +51,7 @@
.StorageContainerDatanodeProtocolProtos.StorageReportProto;
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.SCMVersionResponseProto;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.container.common.statemachine

View File

@ -20,8 +20,8 @@
import org.apache.hadoop.hdds.protocol.proto
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
import org.apache.hadoop.hdds.scm.container.ContainerID;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeMetric;
import org.apache.hadoop.hdds.scm.container.placement.metrics.SCMNodeStat;
import org.apache.hadoop.hdds.scm.exceptions.SCMException;

View File

@ -82,10 +82,7 @@ public Void call() throws Exception {
LOG.info("Container Metadata: {}", metadataStr);
// Print pipeline of an existing container.
LOG.info("LeaderID: {}", container.getPipeline()
.getLeader().getHostName());
String machinesStr = container.getPipeline()
.getMachines().stream().map(
String machinesStr = container.getPipeline().getNodes().stream().map(
DatanodeDetails::getHostName).collect(Collectors.joining(","));
LOG.info("Datanodes: [{}]", machinesStr);
return null;

View File

@ -24,7 +24,7 @@
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
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.pipeline.Pipeline;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
@ -282,7 +282,10 @@ public static LengthInputStream getFromOmKeyInfo(
// irrespective of the container state, we will always read via Standalone
// protocol.
pipeline.setType(HddsProtos.ReplicationType.STAND_ALONE);
if (pipeline.getType() != HddsProtos.ReplicationType.STAND_ALONE) {
pipeline = Pipeline.newBuilder(pipeline)
.setType(HddsProtos.ReplicationType.STAND_ALONE).build();
}
XceiverClientSpi xceiverClient = xceiverClientManager
.acquireClient(pipeline);
boolean success = false;

View File

@ -24,9 +24,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.exceptions.SCMException;
import org.apache.hadoop.hdds.scm.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.After;
@ -50,7 +47,7 @@ public class TestNode2PipelineMap {
private static StorageContainerManager scm;
private static ContainerWithPipeline ratisContainer;
private static ContainerManager containerManager;
private static PipelineSelector pipelineSelector;
private static PipelineManager pipelineManager;
/**
* Create a MiniDFSCluster for testing.
@ -66,7 +63,7 @@ public void init() throws Exception {
containerManager = scm.getContainerManager();
ratisContainer = containerManager.allocateContainer(
RATIS, THREE, "testOwner");
pipelineSelector = containerManager.getPipelineSelector();
pipelineManager = scm.getPipelineManager();
}
/**
@ -83,15 +80,15 @@ public void shutdown() {
@Test
public void testPipelineMap() throws IOException {
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
ratisContainer.getPipeline().getId());
Set<ContainerID> set = pipelineManager
.getContainersInPipeline(ratisContainer.getPipeline().getId());
ContainerID cId = ratisContainer.getContainerInfo().containerID();
Assert.assertEquals(1, set.size());
set.forEach(containerID ->
Assert.assertEquals(containerID, cId));
List<DatanodeDetails> dns = ratisContainer.getPipeline().getMachines();
List<DatanodeDetails> dns = ratisContainer.getPipeline().getNodes();
Assert.assertEquals(3, dns.size());
// get pipeline details by dnid
@ -112,18 +109,14 @@ public void testPipelineMap() throws IOException {
.updateContainerState(cId, HddsProtos.LifeCycleEvent.FINALIZE);
containerManager
.updateContainerState(cId, HddsProtos.LifeCycleEvent.CLOSE);
Set<ContainerID> set2 = pipelineSelector.getOpenContainerIDsByPipeline(
Set<ContainerID> set2 = pipelineManager.getContainersInPipeline(
ratisContainer.getPipeline().getId());
Assert.assertEquals(0, set2.size());
try {
pipelineSelector.updatePipelineState(ratisContainer.getPipeline(),
HddsProtos.LifeCycleEvent.CLOSE);
Assert.fail("closing of pipeline without finalize should fail");
} catch (Exception e) {
Assert.assertTrue(e instanceof SCMException);
Assert.assertEquals(((SCMException)e).getResult(),
SCMException.ResultCodes.FAILED_TO_CHANGE_PIPELINE_STATE);
}
pipelineManager.finalizePipeline(ratisContainer.getPipeline().getId());
pipelineManager.removePipeline(ratisContainer.getPipeline().getId());
pipelines = scm.getScmNodeManager()
.getPipelineByDnID(dns.get(0).getUuid());
Assert.assertEquals(0, pipelines.size());
}
}

View File

@ -20,12 +20,10 @@
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.ScmConfigKeys;
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.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -53,6 +51,7 @@ public class TestNodeFailure {
private static ContainerWithPipeline ratisContainer1;
private static ContainerWithPipeline ratisContainer2;
private static ContainerManager containerManager;
private static PipelineManager pipelineManager;
private static long timeForFailure;
/**
@ -76,6 +75,7 @@ public static void init() throws Exception {
cluster.waitForClusterToBeReady();
StorageContainerManager scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
pipelineManager = scm.getPipelineManager();
ratisContainer1 = containerManager.allocateContainer(
RATIS, THREE, "testOwner");
ratisContainer2 = containerManager.allocateContainer(
@ -102,21 +102,21 @@ public static void shutdown() {
@Test
public void testPipelineFail() throws InterruptedException, IOException,
TimeoutException {
Assert.assertEquals(ratisContainer1.getPipeline().getLifeCycleState(),
HddsProtos.LifeCycleState.OPEN);
Assert.assertEquals(ratisContainer1.getPipeline().getPipelineState(),
Pipeline.PipelineState.OPEN);
Pipeline pipelineToFail = ratisContainer1.getPipeline();
DatanodeDetails dnToFail = pipelineToFail.getMachines().get(0);
DatanodeDetails dnToFail = pipelineToFail.getFirstNode();
cluster.shutdownHddsDatanode(dnToFail);
// wait for sufficient time for the callback to be triggered
Thread.sleep(3 * timeForFailure);
Assert.assertEquals(HddsProtos.LifeCycleState.CLOSED,
ratisContainer1.getPipeline().getLifeCycleState());
Assert.assertEquals(HddsProtos.LifeCycleState.OPEN,
ratisContainer2.getPipeline().getLifeCycleState());
Assert.assertNull(containerManager.getPipelineSelector()
.getPipeline(pipelineToFail.getId()));
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
pipelineManager.getPipeline(ratisContainer1.getPipeline().getId())
.getPipelineState());
Assert.assertEquals(Pipeline.PipelineState.OPEN,
pipelineManager.getPipeline(ratisContainer2.getPipeline().getId())
.getPipelineState());
// Now restart the datanode and make sure that a new pipeline is created.
cluster.restartHddsDatanode(dnToFail);
ContainerWithPipeline ratisContainer3 =

View File

@ -24,8 +24,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.pipelines.PipelineSelector;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.AfterClass;
@ -50,7 +48,7 @@ public class TestPipelineClose {
private static ContainerWithPipeline ratisContainer1;
private static ContainerWithPipeline ratisContainer2;
private static ContainerManager containerManager;
private static PipelineSelector pipelineSelector;
private static PipelineManager pipelineManager;
/**
* Create a MiniDFSCluster for testing.
@ -68,7 +66,7 @@ public static void init() throws Exception {
.allocateContainer(RATIS, THREE, "testOwner");
ratisContainer2 = containerManager
.allocateContainer(RATIS, THREE, "testOwner");
pipelineSelector = containerManager.getPipelineSelector();
pipelineManager = scm.getPipelineManager();
// At this stage, there should be 2 pipeline one with 1 open container each.
// Try closing the both the pipelines, one with a closed container and
// the other with an open container.
@ -87,8 +85,8 @@ public static void shutdown() {
@Test
public void testPipelineCloseWithClosedContainer() throws IOException {
Set<ContainerID> set = pipelineSelector.getOpenContainerIDsByPipeline(
ratisContainer1.getPipeline().getId());
Set<ContainerID> set = pipelineManager
.getContainersInPipeline(ratisContainer1.getPipeline().getId());
ContainerID cId = ratisContainer1.getContainerInfo().containerID();
Assert.assertEquals(1, set.size());
@ -105,17 +103,17 @@ public void testPipelineCloseWithClosedContainer() throws IOException {
containerManager
.updateContainerState(cId, HddsProtos.LifeCycleEvent.CLOSE);
Set<ContainerID> setClosed = pipelineSelector.getOpenContainerIDsByPipeline(
ratisContainer1.getPipeline().getId());
Set<ContainerID> setClosed = pipelineManager
.getContainersInPipeline(ratisContainer1.getPipeline().getId());
Assert.assertEquals(0, setClosed.size());
pipelineSelector.finalizePipeline(ratisContainer1.getPipeline());
Pipeline pipeline1 = pipelineSelector
pipelineManager.finalizePipeline(ratisContainer1.getPipeline().getId());
Pipeline pipeline1 = pipelineManager
.getPipeline(ratisContainer1.getPipeline().getId());
Assert.assertNull(pipeline1);
Assert.assertEquals(ratisContainer1.getPipeline().getLifeCycleState(),
HddsProtos.LifeCycleState.CLOSED);
for (DatanodeDetails dn : ratisContainer1.getPipeline().getMachines()) {
Assert.assertEquals(pipeline1.getPipelineState(),
Pipeline.PipelineState.CLOSED);
pipelineManager.removePipeline(pipeline1.getId());
for (DatanodeDetails dn : ratisContainer1.getPipeline().getNodes()) {
// Assert that the pipeline has been removed from Node2PipelineMap as well
Assert.assertEquals(scm.getScmNodeManager().getPipelineByDnID(
dn.getUuid()).size(), 0);
@ -125,7 +123,7 @@ public void testPipelineCloseWithClosedContainer() throws IOException {
@Test
public void testPipelineCloseWithOpenContainer() throws IOException,
TimeoutException, InterruptedException {
Set<ContainerID> setOpen = pipelineSelector.getOpenContainerIDsByPipeline(
Set<ContainerID> setOpen = pipelineManager.getContainersInPipeline(
ratisContainer2.getPipeline().getId());
Assert.assertEquals(1, setOpen.size());
@ -134,12 +132,13 @@ public void testPipelineCloseWithOpenContainer() throws IOException,
.updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATE);
containerManager
.updateContainerState(cId2, HddsProtos.LifeCycleEvent.CREATED);
pipelineSelector.finalizePipeline(ratisContainer2.getPipeline());
Assert.assertEquals(ratisContainer2.getPipeline().getLifeCycleState(),
HddsProtos.LifeCycleState.CLOSING);
Pipeline pipeline2 = pipelineSelector
pipelineManager.finalizePipeline(ratisContainer2.getPipeline().getId());
Assert.assertEquals(
pipelineManager.getPipeline(ratisContainer2.getPipeline().getId())
.getPipelineState(), Pipeline.PipelineState.CLOSED);
Pipeline pipeline2 = pipelineManager
.getPipeline(ratisContainer2.getPipeline().getId());
Assert.assertEquals(pipeline2.getLifeCycleState(),
HddsProtos.LifeCycleState.CLOSING);
Assert.assertEquals(pipeline2.getPipelineState(),
Pipeline.PipelineState.CLOSED);
}
}

View File

@ -91,7 +91,7 @@ public void testAddAndGetPipeline() throws IOException {
}
// verify pipeline returned is same
Pipeline pipeline1 = stateManager.getPipeline(pipeline.getID());
Pipeline pipeline1 = stateManager.getPipeline(pipeline.getId());
Assert.assertTrue(pipeline == pipeline1);
// clean up
@ -102,15 +102,17 @@ public void testAddAndGetPipeline() throws IOException {
public void testGetPipelines() throws IOException {
Set<Pipeline> pipelines = new HashSet<>();
Pipeline pipeline = createDummyPipeline(1);
pipelines.add(pipeline);
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getId());
pipelines.add(pipeline);
pipeline = createDummyPipeline(1);
pipelines.add(pipeline);
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getId());
pipelines.add(pipeline);
Set<Pipeline> pipelines1 = new HashSet<>(stateManager.getPipelinesByType(
Set<Pipeline> pipelines1 = new HashSet<>(stateManager.getPipelines(
HddsProtos.ReplicationType.RATIS));
Assert.assertEquals(pipelines, pipelines1);
Assert.assertEquals(pipelines1.size(), pipelines.size());
// clean up
for (Pipeline pipeline1 : pipelines) {
removePipeline(pipeline1);
@ -131,16 +133,16 @@ public void testGetPipelinesByTypeAndFactor() throws IOException {
stateManager.addPipeline(pipeline);
pipelines.add(pipeline);
// 5 pipelines in allocated state for each type and factor
// 5 pipelines in open state for each type and factor
pipeline = createDummyPipeline(type, factor, factor.getNumber());
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
pipelines.add(pipeline);
// 5 pipelines in allocated state for each type and factor
// 5 pipelines in closed state for each type and factor
pipeline = createDummyPipeline(type, factor, factor.getNumber());
stateManager.addPipeline(pipeline);
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
pipelines.add(pipeline);
}
}
@ -152,8 +154,8 @@ public void testGetPipelinesByTypeAndFactor() throws IOException {
.values()) {
// verify pipelines received
List<Pipeline> pipelines1 =
stateManager.getPipelinesByTypeAndFactor(type, factor);
Assert.assertEquals(5, pipelines1.size());
stateManager.getPipelines(type, factor);
Assert.assertEquals(15, pipelines1.size());
pipelines1.stream().forEach(p -> {
Assert.assertEquals(p.getType(), type);
Assert.assertEquals(p.getFactor(), factor);
@ -167,41 +169,80 @@ public void testGetPipelinesByTypeAndFactor() throws IOException {
}
}
@Test
public void testGetPipelinesByTypeAndState() throws IOException {
Set<Pipeline> pipelines = new HashSet<>();
for (HddsProtos.ReplicationType type : HddsProtos.ReplicationType
.values()) {
HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE;
for (int i = 0; i < 5; i++) {
// 5 pipelines in allocated state for each type and factor
Pipeline pipeline =
createDummyPipeline(type, factor, factor.getNumber());
stateManager.addPipeline(pipeline);
pipelines.add(pipeline);
// 5 pipelines in open state for each type and factor
pipeline = createDummyPipeline(type, factor, factor.getNumber());
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getId());
pipelines.add(pipeline);
// 5 pipelines in closed state for each type and factor
pipeline = createDummyPipeline(type, factor, factor.getNumber());
stateManager.addPipeline(pipeline);
stateManager.finalizePipeline(pipeline.getId());
pipelines.add(pipeline);
}
}
for (HddsProtos.ReplicationType type : HddsProtos.ReplicationType
.values()) {
// verify pipelines received
List<Pipeline> pipelines1 = stateManager
.getPipelines(type, Pipeline.PipelineState.OPEN);
Assert.assertEquals(5, pipelines1.size());
pipelines1.forEach(p -> {
Assert.assertEquals(p.getType(), type);
Assert.assertEquals(p.getPipelineState(), Pipeline.PipelineState.OPEN);
});
pipelines1 = stateManager
.getPipelines(type, Pipeline.PipelineState.OPEN,
Pipeline.PipelineState.CLOSED, Pipeline.PipelineState.ALLOCATED);
Assert.assertEquals(15, pipelines1.size());
}
//clean up
for (Pipeline pipeline : pipelines) {
removePipeline(pipeline);
}
}
@Test
public void testAddAndGetContainer() throws IOException {
long containerID = 0;
Pipeline pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
pipeline = stateManager.getPipeline(pipeline.getID());
try {
stateManager.addContainerToPipeline(pipeline.getID(),
ContainerID.valueof(++containerID));
Assert.fail("Container should not have been added");
} catch (IOException e) {
// add container possible only in container with open state
Assert.assertTrue(e.getMessage().contains("is not in open state"));
}
pipeline = stateManager.getPipeline(pipeline.getId());
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
// move pipeline to open state
stateManager.openPipeline(pipeline.getID());
// add three containers
stateManager.addContainerToPipeline(pipeline.getID(),
ContainerID.valueof(containerID));
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.openPipeline(pipeline.getId());
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
//verify the number of containers returned
Set<ContainerID> containerIDs =
stateManager.getContainers(pipeline.getID());
stateManager.getContainers(pipeline.getId());
Assert.assertEquals(containerIDs.size(), containerID);
removePipeline(pipeline);
try {
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
Assert.fail("Container should not have been added");
} catch (IOException e) {
@ -215,12 +256,12 @@ public void testRemovePipeline() throws IOException {
Pipeline pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
// close the pipeline
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
stateManager
.addContainerToPipeline(pipeline.getID(), ContainerID.valueof(1));
.addContainerToPipeline(pipeline.getId(), ContainerID.valueof(1));
try {
stateManager.removePipeline(pipeline.getID());
stateManager.removePipeline(pipeline.getId());
Assert.fail("Pipeline should not have been removed");
} catch (IOException e) {
// can not remove a pipeline which already has containers
@ -228,10 +269,10 @@ public void testRemovePipeline() throws IOException {
}
// close the pipeline
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
try {
stateManager.removePipeline(pipeline.getID());
stateManager.removePipeline(pipeline.getId());
Assert.fail("Pipeline should not have been removed");
} catch (IOException e) {
// can not remove a pipeline which already has containers
@ -248,33 +289,33 @@ public void testRemoveContainer() throws IOException {
Pipeline pipeline = createDummyPipeline(1);
// create an open pipeline in stateMap
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(containerID));
Assert.assertEquals(1, stateManager.getContainers(pipeline.getID()).size());
stateManager.removeContainerFromPipeline(pipeline.getID(),
Assert.assertEquals(1, stateManager.getContainers(pipeline.getId()).size());
stateManager.removeContainerFromPipeline(pipeline.getId(),
ContainerID.valueof(containerID));
Assert.assertEquals(0, stateManager.getContainers(pipeline.getID()).size());
Assert.assertEquals(0, stateManager.getContainers(pipeline.getId()).size());
// add two containers in the pipeline
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
stateManager.addContainerToPipeline(pipeline.getID(),
stateManager.addContainerToPipeline(pipeline.getId(),
ContainerID.valueof(++containerID));
Assert.assertEquals(2, stateManager.getContainers(pipeline.getID()).size());
Assert.assertEquals(2, stateManager.getContainers(pipeline.getId()).size());
// move pipeline to closing state
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
stateManager.removeContainerFromPipeline(pipeline.getID(),
stateManager.removeContainerFromPipeline(pipeline.getId(),
ContainerID.valueof(containerID));
stateManager.removeContainerFromPipeline(pipeline.getID(),
stateManager.removeContainerFromPipeline(pipeline.getId(),
ContainerID.valueof(--containerID));
Assert.assertEquals(0, stateManager.getContainers(pipeline.getID()).size());
Assert.assertEquals(0, stateManager.getContainers(pipeline.getId()).size());
// clean up
stateManager.removePipeline(pipeline.getID());
stateManager.removePipeline(pipeline.getId());
}
@Test
@ -282,30 +323,30 @@ public void testFinalizePipeline() throws IOException {
Pipeline pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
// finalize on ALLOCATED pipeline
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
stateManager.getPipeline(pipeline.getID()).getPipelineState());
stateManager.getPipeline(pipeline.getId()).getPipelineState());
// clean up
removePipeline(pipeline);
pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
// finalize on OPEN pipeline
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
stateManager.getPipeline(pipeline.getID()).getPipelineState());
stateManager.getPipeline(pipeline.getId()).getPipelineState());
// clean up
removePipeline(pipeline);
pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
stateManager.openPipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
stateManager.finalizePipeline(pipeline.getId());
// finalize should work on already closed pipeline
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
Assert.assertEquals(Pipeline.PipelineState.CLOSED,
stateManager.getPipeline(pipeline.getID()).getPipelineState());
stateManager.getPipeline(pipeline.getId()).getPipelineState());
// clean up
removePipeline(pipeline);
}
@ -315,25 +356,25 @@ public void testOpenPipeline() throws IOException {
Pipeline pipeline = createDummyPipeline(1);
stateManager.addPipeline(pipeline);
// open on ALLOCATED pipeline
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
Assert.assertEquals(Pipeline.PipelineState.OPEN,
stateManager.getPipeline(pipeline.getID()).getPipelineState());
stateManager.getPipeline(pipeline.getId()).getPipelineState());
stateManager.openPipeline(pipeline.getID());
stateManager.openPipeline(pipeline.getId());
// open should work on already open pipeline
Assert.assertEquals(Pipeline.PipelineState.OPEN,
stateManager.getPipeline(pipeline.getID()).getPipelineState());
stateManager.getPipeline(pipeline.getId()).getPipelineState());
// clean up
removePipeline(pipeline);
}
private void removePipeline(Pipeline pipeline) throws IOException {
stateManager.finalizePipeline(pipeline.getID());
stateManager.finalizePipeline(pipeline.getId());
Set<ContainerID> containerIDs =
stateManager.getContainers(pipeline.getID());
stateManager.getContainers(pipeline.getId());
for (ContainerID containerID : containerIDs) {
stateManager.removeContainerFromPipeline(pipeline.getID(), containerID);
stateManager.removeContainerFromPipeline(pipeline.getId(), containerID);
}
stateManager.removePipeline(pipeline.getID());
stateManager.removePipeline(pipeline.getId());
}
}

View File

@ -58,7 +58,7 @@ public void testCreatePipelineWithFactor() throws IOException {
Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(pipeline.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
factor = HddsProtos.ReplicationFactor.ONE;
@ -71,7 +71,7 @@ public void testCreatePipelineWithFactor() throws IOException {
Assert.assertEquals(pipeline1.getType(), HddsProtos.ReplicationType.RATIS);
Assert.assertEquals(pipeline1.getFactor(), factor);
Assert.assertEquals(pipeline1.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline1.getNodes().size(), factor.getNumber());
}
@ -86,19 +86,20 @@ private List<DatanodeDetails> createListOfNodes(int nodeCount) {
@Test
public void testCreatePipelineWithNodes() throws IOException {
HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE;
Pipeline pipeline = provider.create(createListOfNodes(factor.getNumber()));
Pipeline pipeline =
provider.create(factor, createListOfNodes(factor.getNumber()));
Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(
pipeline.getPipelineState(), Pipeline.PipelineState.ALLOCATED);
pipeline.getPipelineState(), Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
factor = HddsProtos.ReplicationFactor.ONE;
pipeline = provider.create(createListOfNodes(factor.getNumber()));
pipeline = provider.create(factor, createListOfNodes(factor.getNumber()));
Assert.assertEquals(pipeline.getType(), HddsProtos.ReplicationType.RATIS);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(pipeline.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
}
}

View File

@ -26,7 +26,6 @@
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.ContainerID;
import org.apache.hadoop.hdds.scm.container.MockNodeManager;
import org.apache.hadoop.hdds.scm.container.TestSCMContainerManager;
import org.apache.hadoop.hdds.scm.server.SCMDatanodeHeartbeatDispatcher.PipelineReportFromDatanode;
import org.apache.hadoop.hdds.server.events.EventQueue;
import org.apache.hadoop.ozone.OzoneConfigKeys;
@ -54,7 +53,7 @@ public class TestSCMPipelineManager {
public static void setUp() throws Exception {
conf = new OzoneConfiguration();
testDir = GenericTestUtils
.getTestDir(TestSCMContainerManager.class.getSimpleName());
.getTestDir(TestSCMPipelineManager.class.getSimpleName());
conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, testDir.getAbsolutePath());
boolean folderExisted = testDir.exists() || testDir.mkdirs();
if (!folderExisted) {
@ -83,16 +82,18 @@ public void testPipelineReload() throws IOException {
// new pipeline manager should be able to load the pipelines from the db
pipelineManager =
new SCMPipelineManager(conf, nodeManager,
new EventQueue());
new SCMPipelineManager(conf, nodeManager, new EventQueue());
for (Pipeline p : pipelines) {
pipelineManager.openPipeline(p.getId());
}
List<Pipeline> pipelineList =
pipelineManager.getPipelinesByType(HddsProtos.ReplicationType.RATIS);
pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS);
Assert.assertEquals(pipelines, new HashSet<>(pipelineList));
// clean up
for (Pipeline pipeline : pipelines) {
pipelineManager.finalizePipeline(pipeline.getID());
pipelineManager.removePipeline(pipeline.getID());
pipelineManager.finalizePipeline(pipeline.getId());
pipelineManager.removePipeline(pipeline.getId());
}
pipelineManager.close();
}
@ -104,13 +105,13 @@ public void testRemovePipeline() throws IOException {
Pipeline pipeline = pipelineManager
.createPipeline(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE);
pipelineManager.openPipeline(pipeline.getID());
pipelineManager.openPipeline(pipeline.getId());
pipelineManager
.addContainerToPipeline(pipeline.getID(), ContainerID.valueof(1));
pipelineManager.finalizePipeline(pipeline.getID());
.addContainerToPipeline(pipeline.getId(), ContainerID.valueof(1));
pipelineManager.finalizePipeline(pipeline.getId());
pipelineManager
.removeContainerFromPipeline(pipeline.getID(), ContainerID.valueof(1));
pipelineManager.removePipeline(pipeline.getID());
.removeContainerFromPipeline(pipeline.getId(), ContainerID.valueof(1));
pipelineManager.removePipeline(pipeline.getId());
pipelineManager.close();
// new pipeline manager should not be able to load removed pipelines
@ -118,7 +119,7 @@ public void testRemovePipeline() throws IOException {
new SCMPipelineManager(conf, nodeManager,
new EventQueue());
try {
pipelineManager.getPipeline(pipeline.getID());
pipelineManager.getPipeline(pipeline.getId());
Assert.fail("Pipeline should not have been retrieved");
} catch (IOException e) {
Assert.assertTrue(e.getMessage().contains("not found"));
@ -138,36 +139,36 @@ public void testPipelineReport() throws IOException {
.createPipeline(HddsProtos.ReplicationType.RATIS,
HddsProtos.ReplicationFactor.THREE);
Assert
.assertFalse(pipelineManager.getPipeline(pipeline.getID()).isHealthy());
.assertFalse(pipelineManager.getPipeline(pipeline.getId()).isHealthy());
Assert
.assertFalse(pipelineManager.getPipeline(pipeline.getID()).isOpen());
.assertTrue(pipelineManager.getPipeline(pipeline.getId()).isOpen());
// get pipeline report from each dn in the pipeline
PipelineReportHandler pipelineReportHandler =
new PipelineReportHandler(pipelineManager, conf);
for (DatanodeDetails dn: pipeline.getNodes()) {
PipelineReportFromDatanode pipelineReportFromDatanode =
TestUtils.getRandomPipelineReportFromDatanode(dn, pipeline.getID());
TestUtils.getRandomPipelineReportFromDatanode(dn, pipeline.getId());
// pipeline is not healthy until all dns report
Assert.assertFalse(
pipelineManager.getPipeline(pipeline.getID()).isHealthy());
pipelineManager.getPipeline(pipeline.getId()).isHealthy());
pipelineReportHandler
.onMessage(pipelineReportFromDatanode, new EventQueue());
}
// pipeline is healthy when all dns report
Assert
.assertTrue(pipelineManager.getPipeline(pipeline.getID()).isHealthy());
.assertTrue(pipelineManager.getPipeline(pipeline.getId()).isHealthy());
// pipeline should now move to open state
Assert
.assertTrue(pipelineManager.getPipeline(pipeline.getID()).isOpen());
.assertTrue(pipelineManager.getPipeline(pipeline.getId()).isOpen());
// close the pipeline
pipelineManager.finalizePipeline(pipeline.getID());
pipelineManager.finalizePipeline(pipeline.getId());
for (DatanodeDetails dn: pipeline.getNodes()) {
PipelineReportFromDatanode pipelineReportFromDatanode =
TestUtils.getRandomPipelineReportFromDatanode(dn, pipeline.getID());
TestUtils.getRandomPipelineReportFromDatanode(dn, pipeline.getId());
// pipeline report for a closed pipeline should destroy the pipeline
// and remove it from the pipeline manager
pipelineReportHandler
@ -175,7 +176,7 @@ public void testPipelineReport() throws IOException {
}
try {
pipelineManager.getPipeline(pipeline.getID());
pipelineManager.getPipeline(pipeline.getId());
Assert.fail("Pipeline should not have been retrieved");
} catch (IOException e) {
Assert.assertTrue(e.getMessage().contains("not found"));

View File

@ -19,9 +19,7 @@
package org.apache.hadoop.hdds.scm.pipeline;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.ContainerManager;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.junit.AfterClass;
@ -48,6 +46,7 @@ public class TestSCMRestart {
private static Pipeline ratisPipeline2;
private static ContainerManager containerManager;
private static ContainerManager newContainerManager;
private static PipelineManager pipelineManager;
/**
* Create a MiniDFSCluster for testing.
@ -65,6 +64,7 @@ public static void init() throws Exception {
cluster.waitForClusterToBeReady();
StorageContainerManager scm = cluster.getStorageContainerManager();
containerManager = scm.getContainerManager();
pipelineManager = scm.getPipelineManager();
ratisPipeline1 = containerManager.allocateContainer(
RATIS, THREE, "Owner1").getPipeline();
ratisPipeline2 = containerManager.allocateContainer(
@ -75,6 +75,7 @@ public static void init() throws Exception {
cluster.restartStorageContainerManager();
newContainerManager = cluster.getStorageContainerManager()
.getContainerManager();
pipelineManager = cluster.getStorageContainerManager().getPipelineManager();
}
/**
@ -90,25 +91,15 @@ public static void shutdown() {
@Test
public void testPipelineWithScmRestart() throws IOException {
// After restart make sure that the pipeline are still present
Pipeline ratisPipeline1AfterRestart = newContainerManager
.getPipelineSelector().getPipeline(ratisPipeline1.getId());
Pipeline ratisPipeline2AfterRestart = newContainerManager
.getPipelineSelector().getPipeline(ratisPipeline2.getId());
Pipeline ratisPipeline1AfterRestart =
pipelineManager.getPipeline(ratisPipeline1.getId());
Pipeline ratisPipeline2AfterRestart =
pipelineManager.getPipeline(ratisPipeline2.getId());
Assert.assertNotSame(ratisPipeline1AfterRestart, ratisPipeline1);
Assert.assertNotSame(ratisPipeline2AfterRestart, ratisPipeline2);
Assert.assertEquals(ratisPipeline1AfterRestart, ratisPipeline1);
Assert.assertEquals(ratisPipeline2AfterRestart, ratisPipeline2);
for (DatanodeDetails dn : ratisPipeline1.getMachines()) {
Assert.assertEquals(dn, ratisPipeline1AfterRestart.getDatanodes()
.get(dn.getUuidString()));
}
for (DatanodeDetails dn : ratisPipeline2.getMachines()) {
Assert.assertEquals(dn, ratisPipeline2AfterRestart.getDatanodes()
.get(dn.getUuidString()));
}
// Try creating a new ratis pipeline, it should be from the same pipeline
// as was before restart
Pipeline newRatisPipeline = newContainerManager

View File

@ -57,7 +57,7 @@ public void testCreatePipelineWithFactor() throws IOException {
HddsProtos.ReplicationType.STAND_ALONE);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(pipeline.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
factor = HddsProtos.ReplicationFactor.ONE;
@ -67,7 +67,7 @@ public void testCreatePipelineWithFactor() throws IOException {
HddsProtos.ReplicationType.STAND_ALONE);
Assert.assertEquals(pipeline1.getFactor(), factor);
Assert.assertEquals(pipeline1.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline1.getNodes().size(), factor.getNumber());
}
@ -82,21 +82,22 @@ private List<DatanodeDetails> createListOfNodes(int nodeCount) {
@Test
public void testCreatePipelineWithNodes() throws IOException {
HddsProtos.ReplicationFactor factor = HddsProtos.ReplicationFactor.THREE;
Pipeline pipeline = provider.create(createListOfNodes(factor.getNumber()));
Pipeline pipeline =
provider.create(factor, createListOfNodes(factor.getNumber()));
Assert.assertEquals(pipeline.getType(),
HddsProtos.ReplicationType.STAND_ALONE);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(pipeline.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
factor = HddsProtos.ReplicationFactor.ONE;
pipeline = provider.create(createListOfNodes(factor.getNumber()));
pipeline = provider.create(factor, createListOfNodes(factor.getNumber()));
Assert.assertEquals(pipeline.getType(),
HddsProtos.ReplicationType.STAND_ALONE);
Assert.assertEquals(pipeline.getFactor(), factor);
Assert.assertEquals(pipeline.getPipelineState(),
Pipeline.PipelineState.ALLOCATED);
Pipeline.PipelineState.OPEN);
Assert.assertEquals(pipeline.getNodes().size(), factor.getNumber());
}
}

View File

@ -20,7 +20,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.client.protocol.ClientProtocol;
import org.apache.hadoop.ozone.client.rpc.RpcClient;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;

View File

@ -28,7 +28,7 @@
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.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.
ContainerPlacementPolicy;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.

View File

@ -21,7 +21,7 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.container.common.SCMTestUtils;
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
@ -29,7 +29,7 @@
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.test.PathUtils;
import org.apache.hadoop.test.TestGenericTestUtils;
import org.junit.AfterClass;
@ -40,6 +40,7 @@
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
@ -91,18 +92,20 @@ public void testStartMultipleDatanodes() throws Exception {
assertEquals(numberOfNodes, datanodes.size());
for(HddsDatanodeService dn : datanodes) {
// Create a single member pipe line
DatanodeDetails datanodeDetails = dn.getDatanodeDetails();
final Pipeline pipeline =
new Pipeline(datanodeDetails.getUuidString(),
HddsProtos.LifeCycleState.OPEN,
HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.ONE, PipelineID.randomId());
pipeline.addMember(datanodeDetails);
List<DatanodeDetails> dns = new ArrayList<>();
dns.add(dn.getDatanodeDetails());
Pipeline pipeline = Pipeline.newBuilder()
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setType(HddsProtos.ReplicationType.STAND_ALONE)
.setFactor(HddsProtos.ReplicationFactor.ONE)
.setNodes(dns)
.build();
// Verify client is able to connect to the container
try (XceiverClientGrpc client = new XceiverClientGrpc(pipeline, conf)){
client.connect();
assertTrue(client.isConnected(pipeline.getLeader()));
assertTrue(client.isConnected(pipeline.getFirstNode()));
}
}
}

View File

@ -140,7 +140,7 @@ private void testRpcPermissionWithConf(
if (expectPermissionDenied) {
fail("Operation should fail, expecting an IOException here.");
} else {
Assert.assertEquals(1, container2.getPipeline().getMachines().size());
Assert.assertEquals(1, container2.getPipeline().getNodes().size());
}
} catch (Exception e) {
verifyPermissionDeniedException(e, fakeRemoteUsername);
@ -153,7 +153,7 @@ private void testRpcPermissionWithConf(
if (expectPermissionDenied) {
fail("Operation should fail, expecting an IOException here.");
} else {
Assert.assertEquals(1, container3.getPipeline().getMachines().size());
Assert.assertEquals(1, container3.getPipeline().getNodes().size());
}
} catch (Exception e) {
verifyPermissionDeniedException(e, fakeRemoteUsername);

View File

@ -162,9 +162,10 @@ private MetadataStore getContainerMetadata(Long containerID)
.getStorageContainerManager().getClientProtocolServer()
.getContainerWithPipeline(containerID);
DatanodeDetails leadDN = containerWithPipeline.getPipeline().getLeader();
DatanodeDetails dn =
containerWithPipeline.getPipeline().getFirstNode();
OzoneContainer containerServer =
getContainerServerByDatanodeUuid(leadDN.getUuidString());
getContainerServerByDatanodeUuid(dn.getUuidString());
KeyValueContainerData containerData =
(KeyValueContainerData) containerServer.getContainerSet()
.getContainer(containerID).getContainerData();

View File

@ -24,7 +24,7 @@
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.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.ozone.client.*;
import org.apache.hadoop.hdds.client.OzoneQuota;
@ -449,7 +449,7 @@ public void testGetKeyDetails() throws IOException, OzoneException {
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);

View File

@ -28,7 +28,7 @@
StorageContainerException;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneConsts;
@ -306,7 +306,7 @@ private void waitForContainerClose(String keyName,
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline();
pipelineList.add(pipeline);
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
for (DatanodeDetails details : datanodes) {
Assert.assertFalse(ContainerTestHelper
.isContainerClosed(cluster, containerID, details));
@ -319,7 +319,7 @@ private void waitForContainerClose(String keyName,
int index = 0;
for (long containerID : containerIdList) {
Pipeline pipeline = pipelineList.get(index);
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
for (DatanodeDetails datanodeDetails : datanodes) {
GenericTestUtils.waitFor(() -> ContainerTestHelper
.isContainerClosed(cluster, containerID, datanodeDetails), 500,
@ -352,7 +352,7 @@ public void testDiscardPreallocatedBlocks() throws Exception {
List<DatanodeDetails> datanodes =
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline().getMachines();
.getPipeline().getNodes();
Assert.assertEquals(1, datanodes.size());
waitForContainerClose(keyName, key, HddsProtos.ReplicationType.STAND_ALONE);
dataString = fixedLengthString(keyString, (1 * blockSize));
@ -455,7 +455,7 @@ public void testRetriesOnBlockNotCommittedException() throws Exception {
List<DatanodeDetails> datanodes =
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(ContainerID.valueof(containerID))
.getPipeline().getMachines();
.getPipeline().getNodes();
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

@ -30,7 +30,7 @@
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.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.*;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.ozone.client.*;
@ -649,7 +649,7 @@ public void testPutKeyAndGetKeyThreeNodes()
cluster.getStorageContainerManager().getContainerManager()
.getContainerWithPipeline(new ContainerID(containerID));
Pipeline pipeline = container.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
DatanodeDetails datanodeDetails = datanodes.get(0);
Assert.assertNotNull(datanodeDetails);
@ -754,7 +754,7 @@ public void testGetKeyDetails() throws IOException, OzoneException {
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);

View File

@ -21,7 +21,8 @@
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.StorageUnit;
import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.PipelineID;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
@ -35,14 +36,12 @@
.ContainerCommandRequestProto;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
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.datanode.proto.ContainerProtos.KeyValue;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.junit.Assert;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -128,17 +127,16 @@ public static Pipeline createPipeline(int numNodes)
public static Pipeline createPipeline(
Iterable<DatanodeDetails> ids) throws IOException {
Objects.requireNonNull(ids, "ids == null");
final Iterator<DatanodeDetails> i = ids.iterator();
Preconditions.checkArgument(i.hasNext());
final DatanodeDetails leader = i.next();
final Pipeline pipeline =
new Pipeline(leader.getUuidString(), LifeCycleState.OPEN,
ReplicationType.STAND_ALONE, ReplicationFactor.ONE,
PipelineID.randomId());
pipeline.addMember(leader);
for(; i.hasNext();) {
pipeline.addMember(i.next());
}
Preconditions.checkArgument(ids.iterator().hasNext());
List<DatanodeDetails> dns = new ArrayList<>();
ids.forEach(dns::add);
Pipeline pipeline = Pipeline.newBuilder()
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setType(HddsProtos.ReplicationType.STAND_ALONE)
.setFactor(ReplicationFactor.ONE)
.setNodes(dns)
.build();
return pipeline;
}
@ -218,7 +216,7 @@ public static ContainerCommandRequestProto getWriteChunkRequest(
request.setContainerID(blockID.getContainerID());
request.setWriteChunk(writeRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -260,7 +258,7 @@ public static ContainerCommandRequestProto getWriteSmallFileRequest(
request.setContainerID(blockID.getContainerID());
request.setPutSmallFile(smallFileRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -279,7 +277,7 @@ public static ContainerCommandRequestProto getReadSmallFileRequest(
request.setContainerID(getKey.getGetBlock().getBlockID().getContainerID());
request.setGetSmallFile(smallFileRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -309,7 +307,7 @@ public static ContainerCommandRequestProto getReadChunkRequest(
newRequest.setContainerID(readRequest.getBlockID().getContainerID());
newRequest.setReadChunk(readRequest);
newRequest.setTraceID(UUID.randomUUID().toString());
newRequest.setDatanodeUuid(pipeline.getLeader().getUuidString());
newRequest.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return newRequest.build();
}
@ -342,7 +340,7 @@ public static ContainerCommandRequestProto getDeleteChunkRequest(
request.setContainerID(writeRequest.getBlockID().getContainerID());
request.setDeleteChunk(deleteRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -363,7 +361,7 @@ public static ContainerCommandRequestProto getCreateContainerRequest(
request.setCreateContainer(
ContainerProtos.CreateContainerRequestProto.getDefaultInstance());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -398,7 +396,7 @@ public static ContainerCommandRequestProto getUpdateContainerRequest(
request.setContainerID(containerID);
request.setUpdateContainer(updateRequestBuilder.build());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
/**
@ -427,7 +425,8 @@ public static ContainerCommandRequestProto getUpdateContainerRequest(
* @return - Request
*/
public static ContainerCommandRequestProto getPutBlockRequest(
Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest) {
Pipeline pipeline, ContainerProtos.WriteChunkRequestProto writeRequest)
throws IOException {
LOG.trace("putBlock: {} to pipeline={}",
writeRequest.getBlockID());
@ -448,7 +447,7 @@ public static ContainerCommandRequestProto getPutBlockRequest(
request.setContainerID(blockData.getContainerID());
request.setPutBlock(putRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -460,7 +459,8 @@ public static ContainerCommandRequestProto getPutBlockRequest(
* immediately.
*/
public static ContainerCommandRequestProto getBlockRequest(
Pipeline pipeline, ContainerProtos.PutBlockRequestProto putBlockRequest) {
Pipeline pipeline, ContainerProtos.PutBlockRequestProto putBlockRequest)
throws IOException {
ContainerProtos.DatanodeBlockID blockID =
putBlockRequest.getBlockData().getBlockID();
LOG.trace("getKey: blockID={}", blockID);
@ -475,7 +475,7 @@ public static ContainerCommandRequestProto getBlockRequest(
request.setContainerID(blockID.getContainerID());
request.setGetBlock(getRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -499,7 +499,8 @@ public static void verifyGetBlock(ContainerCommandRequestProto request,
* @return - Request
*/
public static ContainerCommandRequestProto getDeleteBlockRequest(
Pipeline pipeline, ContainerProtos.PutBlockRequestProto putBlockRequest) {
Pipeline pipeline, ContainerProtos.PutBlockRequestProto putBlockRequest)
throws IOException {
ContainerProtos.DatanodeBlockID blockID = putBlockRequest.getBlockData()
.getBlockID();
LOG.trace("deleteBlock: name={}", blockID);
@ -512,7 +513,7 @@ public static ContainerCommandRequestProto getDeleteBlockRequest(
request.setContainerID(blockID.getContainerID());
request.setDeleteBlock(delRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
return request.build();
}
@ -523,7 +524,7 @@ public static ContainerCommandRequestProto getDeleteBlockRequest(
* @return ContainerCommandRequestProto.
*/
public static ContainerCommandRequestProto getCloseContainer(
Pipeline pipeline, long containerID) {
Pipeline pipeline, long containerID) throws IOException {
ContainerProtos.ContainerCommandRequestProto cmd =
ContainerCommandRequestProto.newBuilder()
.setCmdType(ContainerProtos.Type.CloseContainer)
@ -531,7 +532,7 @@ public static ContainerCommandRequestProto getCloseContainer(
.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
.setTraceID(UUID.randomUUID().toString())
.setDatanodeUuid(pipeline.getLeader().getUuidString())
.setDatanodeUuid(pipeline.getFirstNode().getUuidString())
.build();
return cmd;
@ -544,7 +545,7 @@ public static ContainerCommandRequestProto getCloseContainer(
* @return ContainerCommandRequestProto without traceId.
*/
public static ContainerCommandRequestProto getRequestWithoutTraceId(
Pipeline pipeline, long containerID) {
Pipeline pipeline, long containerID) throws IOException {
Preconditions.checkNotNull(pipeline);
ContainerProtos.ContainerCommandRequestProto cmd =
ContainerCommandRequestProto.newBuilder()
@ -552,7 +553,7 @@ public static ContainerCommandRequestProto getRequestWithoutTraceId(
.setContainerID(containerID)
.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance())
.setDatanodeUuid(pipeline.getLeader().getUuidString())
.setDatanodeUuid(pipeline.getFirstNode().getUuidString())
.build();
return cmd;
}
@ -563,7 +564,8 @@ public static ContainerCommandRequestProto getRequestWithoutTraceId(
* @return ContainerCommandRequestProto.
*/
public static ContainerCommandRequestProto getDeleteContainer(
Pipeline pipeline, long containerID, boolean forceDelete) {
Pipeline pipeline, long containerID, boolean forceDelete)
throws IOException {
Preconditions.checkNotNull(pipeline);
ContainerProtos.DeleteContainerRequestProto deleteRequest =
ContainerProtos.DeleteContainerRequestProto.newBuilder().
@ -575,7 +577,7 @@ public static ContainerCommandRequestProto getDeleteContainer(
ContainerProtos.DeleteContainerRequestProto.getDefaultInstance())
.setDeleteContainer(deleteRequest)
.setTraceID(UUID.randomUUID().toString())
.setDatanodeUuid(pipeline.getLeader().getUuidString())
.setDatanodeUuid(pipeline.getFirstNode().getUuidString())
.build();
}

View File

@ -36,7 +36,7 @@
.DatanodeBlockID;
import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
@ -123,7 +123,7 @@ public void testContainerReplication() throws Exception {
cluster.getStorageContainerManager().getScmNodeManager()
.addDatanodeCommand(destinationDatanode.getDatanodeDetails().getUuid(),
new ReplicateContainerCommand(containerId,
sourcePipelines.getMachines()));
sourcePipelines.getNodes()));
Thread.sleep(3000);
@ -163,7 +163,7 @@ public void testContainerReplication() throws Exception {
private HddsDatanodeService chooseDatanodeWithoutContainer(Pipeline pipeline,
List<HddsDatanodeService> dataNodes) {
for (HddsDatanodeService datanode : dataNodes) {
if (!pipeline.getMachines().contains(datanode.getDatanodeDetails())) {
if (!pipeline.getNodes().contains(datanode.getDatanodeDetails())) {
return datanode;
}
}

View File

@ -31,7 +31,7 @@
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler;
import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.junit.AfterClass;
import org.junit.BeforeClass;
@ -144,7 +144,7 @@ private List<ChunkInfo> writeChunkBuilder(BlockID blockID, Pipeline pipeline,
request.setContainerID(blockID.getContainerID());
request.setWriteChunk(writeRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
dispatcher.dispatch(request.build());
chunkList.add(info);
}
@ -179,7 +179,7 @@ public void testPutKeyWithMultipleChunks()
request.setContainerID(blockID.getContainerID());
request.setPutBlock(putBlockRequestProto);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
dispatcher.dispatch(request.build());
//the open block should be removed from Map
@ -217,7 +217,7 @@ public void testDeleteChunk() throws Exception {
request.setDeleteChunk(deleteChunkProto);
request.setWriteChunk(writeRequest);
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
dispatcher.dispatch(request.build());
Assert.assertTrue(
openContainerBlockMap.getBlockDataMap(testContainerID)
@ -250,7 +250,7 @@ public void testCloseContainer() throws Exception {
request.setCloseContainer(
ContainerProtos.CloseContainerRequestProto.getDefaultInstance());
request.setTraceID(UUID.randomUUID().toString());
request.setDatanodeUuid(pipeline.getLeader().getUuidString());
request.setDatanodeUuid(pipeline.getFirstNode().getUuidString());
dispatcher.dispatch(request.build());
Assert.assertNull(
openContainerBlockMap.getBlockDataMap(testContainerID));

View File

@ -23,7 +23,7 @@
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.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.HddsDatanodeService;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.ObjectStore;
@ -106,7 +106,7 @@ public void testIfCloseContainerCommandHandlerIsInvoked() throws Exception {
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);
@ -162,7 +162,7 @@ public void testCloseContainerViaStandAlone()
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(datanodes.size(), 1);
DatanodeDetails datanodeDetails = datanodes.get(0);
@ -220,7 +220,7 @@ public void testCloseContainerViaRatis() throws IOException,
.getContainerManager().getContainerWithPipeline(
ContainerID.valueof(containerID))
.getPipeline();
List<DatanodeDetails> datanodes = pipeline.getMachines();
List<DatanodeDetails> datanodes = pipeline.getNodes();
Assert.assertEquals(3, datanodes.size());
GenericTestUtils.LogCapturer logCapturer =

View File

@ -20,7 +20,7 @@
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.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;

View File

@ -33,7 +33,7 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto;
import org.apache.hadoop.hdds.scm.*;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.RatisTestHelper;
@ -93,7 +93,7 @@ static void runContainerStateMachineMetrics(
final OzoneConfiguration conf = new OzoneConfiguration();
initConf.accept(pipeline, conf);
for (DatanodeDetails dn : pipeline.getMachines()) {
for (DatanodeDetails dn : pipeline.getNodes()) {
final XceiverServerSpi s = createServer.apply(dn, conf);
servers.add(s);
s.start();

View File

@ -41,7 +41,7 @@
import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc;
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.ozone.container.replication.GrpcReplicationService;
import org.apache.hadoop.ozone.container.replication.OnDemandContainerReplicationSource;
import org.apache.hadoop.test.GenericTestUtils;
@ -75,7 +75,7 @@ public void testContainerMetrics() throws Exception {
.createSingleNodePipeline();
OzoneConfiguration conf = new OzoneConfiguration();
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader()
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
conf.setInt(DFSConfigKeys.DFS_METRICS_PERCENTILES_INTERVALS_KEY,
interval);

View File

@ -28,7 +28,7 @@
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
@ -66,7 +66,8 @@ public void testCreateOzoneContainer() throws Exception {
// independently in our test path.
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
conf.set(HDDS_DATANODE_DIR_KEY, tempFolder.getRoot().getPath());
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT, pipeline.getLeader()
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
conf.setBoolean(
OzoneConfigKeys.DFS_CONTAINER_IPC_RANDOM_PORT, false);
@ -108,7 +109,7 @@ public void testOzoneContainerViaDataNode() throws Exception {
Pipeline pipeline =
ContainerTestHelper.createSingleNodePipeline();
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader()
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
cluster = MiniOzoneCluster.newBuilder(conf)
@ -514,7 +515,7 @@ private static XceiverClientGrpc createClientForTesting(
Pipeline pipeline =
ContainerTestHelper.createSingleNodePipeline();
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader()
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
// This client talks to ozone container via datanode.

View File

@ -25,7 +25,7 @@
import org.apache.hadoop.ozone.container.ContainerTestHelper;
import org.apache.hadoop.ozone.web.utils.OzoneUtils;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.rpc.SupportedRpcType;
import org.apache.ratis.util.CheckedBiConsumer;
@ -88,12 +88,12 @@ private static void runTest(
// Create Ratis cluster
// final String ratisId = "ratis1";
// final PipelineManager manager = RatisManagerImpl.newRatisManager(conf);
// manager.createPipeline(ratisId, pipeline.getMachines());
// manager.createPipeline(ratisId, pipeline.getNodes());
// LOG.info("Created RatisCluster " + ratisId);
//
// // check Ratis cluster members
// final List<DatanodeDetails> dns = manager.getMembers(ratisId);
// Assert.assertEquals(pipeline.getMachines(), dns);
// Assert.assertEquals(pipeline.getNodes(), dns);
//
// // run test
// final XceiverClientSpi client = XceiverClientRatis

View File

@ -44,7 +44,7 @@
import org.apache.hadoop.hdds.scm.XceiverClientGrpc;
import org.apache.hadoop.hdds.scm.XceiverClientRatis;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.ratis.rpc.RpcType;
import org.apache.ratis.util.CheckedBiConsumer;
@ -56,7 +56,6 @@
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.function.BiConsumer;
import static org.apache.ratis.rpc.SupportedRpcType.GRPC;
import static org.apache.ratis.rpc.SupportedRpcType.NETTY;
@ -80,15 +79,15 @@ private GrpcReplicationService createReplicationService(
public void testClientServer() throws Exception {
DatanodeDetails datanodeDetails = TestUtils.randomDatanodeDetails();
ContainerSet containerSet = new ContainerSet();
runTestClientServer(1,
(pipeline, conf) -> conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue()),
runTestClientServer(1, (pipeline, conf) -> conf
.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue()),
XceiverClientGrpc::new,
(dn, conf) -> new XceiverServerGrpc(datanodeDetails, conf,
new TestContainerDispatcher(),
createReplicationService(containerSet)),
(dn, p) -> {});
createReplicationService(containerSet)), (dn, p) -> {
});
}
@FunctionalInterface
@ -131,7 +130,7 @@ static void runTestClientServerRatis(RpcType rpc, int numNodes)
static void runTestClientServer(
int numDatanodes,
BiConsumer<Pipeline, OzoneConfiguration> initConf,
CheckedBiConsumer<Pipeline, OzoneConfiguration, IOException> initConf,
CheckedBiFunction<Pipeline, OzoneConfiguration, XceiverClientSpi,
IOException> createClient,
CheckedBiFunction<DatanodeDetails, OzoneConfiguration, XceiverServerSpi,
@ -146,7 +145,7 @@ static void runTestClientServer(
final OzoneConfiguration conf = new OzoneConfiguration();
initConf.accept(pipeline, conf);
for(DatanodeDetails dn : pipeline.getMachines()) {
for (DatanodeDetails dn : pipeline.getNodes()) {
final XceiverServerSpi s = createServer.apply(dn, conf);
servers.add(s);
s.start();
@ -181,7 +180,7 @@ public void testClientServerWithContainerDispatcher() throws Exception {
Pipeline pipeline = ContainerTestHelper.createSingleNodePipeline();
OzoneConfiguration conf = new OzoneConfiguration();
conf.setInt(OzoneConfigKeys.DFS_CONTAINER_IPC_PORT,
pipeline.getLeader()
pipeline.getFirstNode()
.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue());
ContainerSet containerSet = new ContainerSet();

View File

@ -69,7 +69,7 @@ public void testAllocate() throws Exception {
xceiverClientManager.getFactor(),
containerOwner);
Assert.assertNotNull(container);
Assert.assertNotNull(container.getPipeline().getLeader());
Assert.assertNotNull(container.getPipeline().getFirstNode());
}

View File

@ -28,7 +28,7 @@
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.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.container.common.helpers.
StorageContainerException;
import org.apache.hadoop.hdds.scm.container.placement.algorithms.

View File

@ -27,8 +27,8 @@
import org.apache.hadoop.hdds.scm.TestUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
import org.apache.hadoop.hdds.scm.container.common.helpers.DeleteBlockResult;
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.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol;
import org.apache.hadoop.ozone.common.BlockGroup;
import org.apache.hadoop.ozone.common.DeleteBlockGroupResult;
@ -127,12 +127,15 @@ public AllocatedBlock allocateBlock(long size,
}
private Pipeline createPipeline(DatanodeDetails datanode) {
final Pipeline pipeline =
new Pipeline(datanode.getUuidString(), HddsProtos.LifeCycleState.OPEN,
HddsProtos.ReplicationType.STAND_ALONE,
HddsProtos.ReplicationFactor.ONE,
PipelineID.randomId());
pipeline.addMember(datanode);
List<DatanodeDetails> dns = new ArrayList<>();
dns.add(datanode);
Pipeline pipeline = Pipeline.newBuilder()
.setState(Pipeline.PipelineState.OPEN)
.setId(PipelineID.randomId())
.setType(HddsProtos.ReplicationType.STAND_ALONE)
.setFactor(HddsProtos.ReplicationFactor.ONE)
.setNodes(dns)
.build();
return pipeline;
}

Some files were not shown because too many files have changed in this diff Show More