HDDS-1406. Avoid usage of commonPool in RatisPipelineUtils. (#714)
This commit is contained in:
parent
ef1cc725b8
commit
77c49f2963
|
@ -61,4 +61,8 @@ public final class PipelineFactory {
|
|||
List<DatanodeDetails> nodes) {
|
||||
return providers.get(type).create(factor, nodes);
|
||||
}
|
||||
|
||||
public void shutdown() {
|
||||
providers.values().forEach(provider -> provider.shutdown());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -33,4 +33,5 @@ public interface PipelineProvider {
|
|||
|
||||
Pipeline create(ReplicationFactor factor, List<DatanodeDetails> nodes);
|
||||
|
||||
void shutdown();
|
||||
}
|
||||
|
|
|
@ -24,17 +24,39 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.NodeState;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
||||
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.node.NodeManager;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline.PipelineState;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.ratis.RatisHelper;
|
||||
import org.apache.ratis.client.RaftClient;
|
||||
import org.apache.ratis.grpc.GrpcTlsConfig;
|
||||
import org.apache.ratis.protocol.RaftClientReply;
|
||||
import org.apache.ratis.protocol.RaftGroup;
|
||||
import org.apache.ratis.protocol.RaftPeer;
|
||||
import org.apache.ratis.retry.RetryPolicy;
|
||||
import org.apache.ratis.rpc.SupportedRpcType;
|
||||
import org.apache.ratis.util.TimeDuration;
|
||||
import org.apache.ratis.util.function.CheckedBiConsumer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ForkJoinPool;
|
||||
import java.util.concurrent.ForkJoinWorkerThread;
|
||||
import java.util.concurrent.RejectedExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
|
@ -42,10 +64,28 @@ import java.util.stream.Collectors;
|
|||
*/
|
||||
public class RatisPipelineProvider implements PipelineProvider {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(RatisPipelineProvider.class);
|
||||
|
||||
private final NodeManager nodeManager;
|
||||
private final PipelineStateManager stateManager;
|
||||
private final Configuration conf;
|
||||
|
||||
// Set parallelism at 3, as now in Ratis we create 1 and 3 node pipelines.
|
||||
private final int parallelismForPool = 3;
|
||||
|
||||
private final ForkJoinPool.ForkJoinWorkerThreadFactory factory =
|
||||
(pool -> {
|
||||
final ForkJoinWorkerThread worker = ForkJoinPool.
|
||||
defaultForkJoinWorkerThreadFactory.newThread(pool);
|
||||
worker.setName("RATISCREATEPIPELINE" + worker.getPoolIndex());
|
||||
return worker;
|
||||
});
|
||||
|
||||
private final ForkJoinPool forkJoinPool = new ForkJoinPool(
|
||||
parallelismForPool, factory, null, false);
|
||||
|
||||
|
||||
RatisPipelineProvider(NodeManager nodeManager,
|
||||
PipelineStateManager stateManager, Configuration conf) {
|
||||
this.nodeManager = nodeManager;
|
||||
|
@ -53,6 +93,7 @@ public class RatisPipelineProvider implements PipelineProvider {
|
|||
this.conf = conf;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Create pluggable container placement policy implementation instance.
|
||||
*
|
||||
|
@ -133,7 +174,81 @@ public class RatisPipelineProvider implements PipelineProvider {
|
|||
.build();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
forkJoinPool.shutdownNow();
|
||||
try {
|
||||
forkJoinPool.awaitTermination(60, TimeUnit.SECONDS);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Unexpected exception occurred during shutdown of " +
|
||||
"RatisPipelineProvider", e);
|
||||
}
|
||||
}
|
||||
|
||||
protected void initializePipeline(Pipeline pipeline) throws IOException {
|
||||
RatisPipelineUtils.createPipeline(pipeline, conf);
|
||||
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
|
||||
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
|
||||
callRatisRpc(pipeline.getNodes(),
|
||||
(raftClient, peer) -> {
|
||||
RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
|
||||
if (reply == null || !reply.isSuccess()) {
|
||||
String msg = "Pipeline initialization failed for pipeline:"
|
||||
+ pipeline.getId() + " node:" + peer.getId();
|
||||
LOG.error(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
private void callRatisRpc(List<DatanodeDetails> datanodes,
|
||||
CheckedBiConsumer< RaftClient, RaftPeer, IOException> rpc)
|
||||
throws IOException {
|
||||
if (datanodes.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final String rpcType = conf
|
||||
.get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
|
||||
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
|
||||
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(conf);
|
||||
final List< IOException > exceptions =
|
||||
Collections.synchronizedList(new ArrayList<>());
|
||||
final int maxOutstandingRequests =
|
||||
HddsClientUtils.getMaxOutstandingRequests(conf);
|
||||
final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
|
||||
SecurityConfig(conf));
|
||||
final TimeDuration requestTimeout =
|
||||
RatisHelper.getClientRequestTimeout(conf);
|
||||
try {
|
||||
forkJoinPool.submit(() -> {
|
||||
datanodes.parallelStream().forEach(d -> {
|
||||
final RaftPeer p = RatisHelper.toRaftPeer(d);
|
||||
try (RaftClient client = RatisHelper
|
||||
.newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
|
||||
retryPolicy, maxOutstandingRequests, tlsConfig,
|
||||
requestTimeout)) {
|
||||
rpc.accept(client, p);
|
||||
} catch (IOException ioe) {
|
||||
String errMsg =
|
||||
"Failed invoke Ratis rpc " + rpc + " for " + d.getUuid();
|
||||
LOG.error(errMsg, ioe);
|
||||
exceptions.add(new IOException(errMsg, ioe));
|
||||
}
|
||||
});
|
||||
}).get();
|
||||
} catch (ExecutionException | RejectedExecutionException ex) {
|
||||
LOG.error(ex.getClass().getName() + " exception occurred during " +
|
||||
"createPipeline", ex);
|
||||
throw new IOException(ex.getClass().getName() + " exception occurred " +
|
||||
"during createPipeline", ex);
|
||||
} catch (InterruptedException ex) {
|
||||
Thread.currentThread().interrupt();
|
||||
throw new IOException("Interrupt exception occurred during " +
|
||||
"createPipeline", ex);
|
||||
}
|
||||
if (!exceptions.isEmpty()) {
|
||||
throw MultipleIOException.createIOException(exceptions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,66 +17,37 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdds.scm.pipeline;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.ratis.RatisHelper;
|
||||
import org.apache.ratis.client.RaftClient;
|
||||
import org.apache.ratis.grpc.GrpcTlsConfig;
|
||||
import org.apache.ratis.protocol.RaftClientReply;
|
||||
import org.apache.ratis.protocol.RaftGroup;
|
||||
import org.apache.ratis.protocol.RaftGroupId;
|
||||
import org.apache.ratis.protocol.RaftPeer;
|
||||
import org.apache.ratis.retry.RetryPolicy;
|
||||
import org.apache.ratis.rpc.SupportedRpcType;
|
||||
import org.apache.ratis.util.TimeDuration;
|
||||
import org.apache.ratis.util.function.CheckedBiConsumer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Utility class for Ratis pipelines. Contains methods to create and destroy
|
||||
* ratis pipelines.
|
||||
*/
|
||||
final class RatisPipelineUtils {
|
||||
public final class RatisPipelineUtils {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(RatisPipelineUtils.class);
|
||||
|
||||
private RatisPipelineUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Sends ratis command to create pipeline on all the datanodes.
|
||||
*
|
||||
* @param pipeline - Pipeline to be created
|
||||
* @param ozoneConf - Ozone Confinuration
|
||||
* @throws IOException if creation fails
|
||||
*/
|
||||
public static void createPipeline(Pipeline pipeline, Configuration ozoneConf)
|
||||
throws IOException {
|
||||
final RaftGroup group = RatisHelper.newRaftGroup(pipeline);
|
||||
LOG.debug("creating pipeline:{} with {}", pipeline.getId(), group);
|
||||
callRatisRpc(pipeline.getNodes(), ozoneConf,
|
||||
(raftClient, peer) -> {
|
||||
RaftClientReply reply = raftClient.groupAdd(group, peer.getId());
|
||||
if (reply == null || !reply.isSuccess()) {
|
||||
String msg = "Pipeline initialization failed for pipeline:"
|
||||
+ pipeline.getId() + " node:" + peer.getId();
|
||||
LOG.error(msg);
|
||||
throw new IOException(msg);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes pipeline from SCM. Sends ratis command to destroy pipeline on all
|
||||
* the datanodes.
|
||||
|
@ -125,42 +96,4 @@ final class RatisPipelineUtils {
|
|||
client
|
||||
.groupRemove(RaftGroupId.valueOf(pipelineID.getId()), true, p.getId());
|
||||
}
|
||||
|
||||
private static void callRatisRpc(List<DatanodeDetails> datanodes,
|
||||
Configuration ozoneConf,
|
||||
CheckedBiConsumer<RaftClient, RaftPeer, IOException> rpc)
|
||||
throws IOException {
|
||||
if (datanodes.isEmpty()) {
|
||||
return;
|
||||
}
|
||||
|
||||
final String rpcType = ozoneConf
|
||||
.get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
|
||||
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
|
||||
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
|
||||
final List<IOException> exceptions =
|
||||
Collections.synchronizedList(new ArrayList<>());
|
||||
final int maxOutstandingRequests =
|
||||
HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
|
||||
final GrpcTlsConfig tlsConfig = RatisHelper.createTlsClientConfig(new
|
||||
SecurityConfig(ozoneConf));
|
||||
final TimeDuration requestTimeout =
|
||||
RatisHelper.getClientRequestTimeout(ozoneConf);
|
||||
datanodes.parallelStream().forEach(d -> {
|
||||
final RaftPeer p = RatisHelper.toRaftPeer(d);
|
||||
try (RaftClient client = RatisHelper
|
||||
.newRaftClient(SupportedRpcType.valueOfIgnoreCase(rpcType), p,
|
||||
retryPolicy, maxOutstandingRequests, tlsConfig, requestTimeout)) {
|
||||
rpc.accept(client, p);
|
||||
} catch (IOException ioe) {
|
||||
String errMsg =
|
||||
"Failed invoke Ratis rpc " + rpc + " for " + d.getUuid();
|
||||
LOG.error(errMsg, ioe);
|
||||
exceptions.add(new IOException(errMsg, ioe));
|
||||
}
|
||||
});
|
||||
if (!exceptions.isEmpty()) {
|
||||
throw MultipleIOException.createIOException(exceptions);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -87,7 +87,8 @@ public class SCMPipelineManager implements PipelineManager {
|
|||
this.lock = new ReentrantReadWriteLock();
|
||||
this.conf = conf;
|
||||
this.stateManager = new PipelineStateManager(conf);
|
||||
this.pipelineFactory = new PipelineFactory(nodeManager, stateManager, conf);
|
||||
this.pipelineFactory = new PipelineFactory(nodeManager, stateManager,
|
||||
conf);
|
||||
// TODO: See if thread priority needs to be set for these threads
|
||||
scheduler = new Scheduler("RatisPipelineUtilsThread", false, 1);
|
||||
this.backgroundPipelineCreator =
|
||||
|
@ -419,5 +420,7 @@ public class SCMPipelineManager implements PipelineManager {
|
|||
if(metrics != null) {
|
||||
metrics.unRegister();
|
||||
}
|
||||
// shutdown pipeline provider.
|
||||
pipelineFactory.shutdown();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,4 +72,9 @@ public class SimplePipelineProvider implements PipelineProvider {
|
|||
.setNodes(nodes)
|
||||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
// Do nothing.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -37,4 +37,9 @@ public class MockRatisPipelineProvider extends RatisPipelineProvider {
|
|||
protected void initializePipeline(Pipeline pipeline) throws IOException {
|
||||
// do nothing as the datanodes do not exists
|
||||
}
|
||||
|
||||
@Override
|
||||
public void shutdown() {
|
||||
// Do nothing.
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
|||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.scm.server.StorageContainerManager;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.hadoop.ozone.HddsDatanodeService;
|
||||
import org.apache.hadoop.ozone.MiniOzoneCluster;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
|
@ -40,7 +39,7 @@ import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTER
|
|||
/**
|
||||
* Tests for RatisPipelineUtils.
|
||||
*/
|
||||
public class TestRatisPipelineUtils {
|
||||
public class TestRatisPipelineCreateAndDestory {
|
||||
|
||||
private static MiniOzoneCluster cluster;
|
||||
private OzoneConfiguration conf = new OzoneConfiguration();
|
||||
|
@ -98,11 +97,13 @@ public class TestRatisPipelineUtils {
|
|||
|
||||
// try creating another pipeline now
|
||||
try {
|
||||
RatisPipelineUtils.createPipeline(pipelines.get(0), conf);
|
||||
pipelineManager.createPipeline(HddsProtos.ReplicationType.RATIS,
|
||||
HddsProtos.ReplicationFactor.THREE);
|
||||
Assert.fail("pipeline creation should fail after shutting down pipeline");
|
||||
} catch (IOException ioe) {
|
||||
// in case the pipeline creation fails, MultipleIOException is thrown
|
||||
Assert.assertTrue(ioe instanceof MultipleIOException);
|
||||
// As now all datanodes are shutdown, they move to stale state, there
|
||||
// will be no sufficient datanodes to create the pipeline.
|
||||
Assert.assertTrue(ioe instanceof InsufficientDatanodesException);
|
||||
}
|
||||
|
||||
// make sure pipelines is destroyed
|
Loading…
Reference in New Issue