HDDS-694. Plugin new Pipeline management code in SCM.
Contributed by Lokesh Jain.
This commit is contained in:
parent
e28c00c290
commit
dce4ebe814
@ -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(
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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 {
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -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;
|
||||
|
||||
/**
|
||||
|
@ -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() {
|
||||
|
@ -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 {
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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(),
|
||||
|
@ -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()
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
|
@ -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,7 +429,7 @@ public List<PipelineReport> getPipelineReport() {
|
||||
List<PipelineReport> reports = new ArrayList<>();
|
||||
for (RaftGroupId groupId : gids) {
|
||||
reports.add(PipelineReport.newBuilder()
|
||||
.setPipelineID(PipelineID.valueOf(groupId).getProtobuf())
|
||||
.setPipelineID(PipelineID.valueOf(groupId.getUuid()).getProtobuf())
|
||||
.build());
|
||||
}
|
||||
return reports;
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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)) {
|
||||
|
@ -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.
|
||||
|
@ -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));
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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) -> {
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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");
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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.
|
||||
*/
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
@ -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));
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
@ -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())
|
||||
|
@ -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;
|
||||
|
@ -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,
|
||||
|
@ -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();
|
||||
|
@ -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,
|
||||
HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.THREE,
|
||||
PipelineID.randomId());
|
||||
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(pipelineSelector
|
||||
.getReplicationPipeline(HddsProtos.ReplicationType.STAND_ALONE,
|
||||
when(pipelineManager.createPipeline(HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.THREE)).thenReturn(pipeline);
|
||||
|
||||
return containerStateManager.allocateContainer(
|
||||
pipelineSelector, HddsProtos.ReplicationType.STAND_ALONE,
|
||||
return containerStateManager.allocateContainer(pipelineManager,
|
||||
HddsProtos.ReplicationType.STAND_ALONE,
|
||||
HddsProtos.ReplicationFactor.THREE, "root");
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
@ -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 =
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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(),
|
||||
pipeline = stateManager.getPipeline(pipeline.getId());
|
||||
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"));
|
||||
}
|
||||
|
||||
// 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());
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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"));
|
||||
|
@ -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
|
||||
|
@ -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());
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
@ -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));
|
||||
|
@ -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 =
|
||||
|
@ -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;
|
||||
|
@ -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();
|
||||
|
@ -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);
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
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();
|
||||
|
@ -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());
|
||||
|
||||
}
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user