Compare commits
89 Commits
Author | SHA1 | Date |
---|---|---|
Ajay Kumar | 9d3a22ebc0 | |
Ajay Yadav | 43d5f892ef | |
Elek, Márton | be269feb6a | |
Xiaoyu Yao | 679c2bfa79 | |
Ajay Yadav | 8e5e5f3641 | |
Ajay Yadav | 75dc0cc92f | |
Eric Yang | 3e88cb3c26 | |
Elek, Márton | 3431333988 | |
Nanda kumar | 50d7c91de1 | |
Anu Engineer | c74df79aee | |
Elek, Márton | d758c97873 | |
Elek, Márton | 7ec6a31eb3 | |
Xiaoyu Yao | 3e8669cbe2 | |
Ajay Kumar | 7030e6ef92 | |
Yiqun Lin | 0781a7174f | |
Ajay Kumar | fb7844db4e | |
Shashikant Banerjee | 0b84c2d5d6 | |
Shashikant Banerjee | bb20c80955 | |
Elek, Márton | f0640e2eba | |
Doroszlai, Attila | 4345e6e10f | |
nilotpalnandi | f2dee89b4f | |
Shashikant Banerjee | c53408c3e3 | |
Xiaoyu Yao | f8f85fcc31 | |
Xiaoyu Yao | 5f8ded542a | |
Shweta Yakkali | 299177eaa7 | |
Ajay Yadav | 447d53476a | |
Sandeep Nemuri | f6acbc9caf | |
Shashikant Banerjee | eed623ad61 | |
Xiaoyu Yao | 8d1c218428 | |
Vivek Ratnavel Subramanian | 6b3e0b3edc | |
Xiaoyu Yao | 9b989c49c9 | |
Arpit Agarwal | 6d73e7bdf4 | |
Arpit Agarwal | 8f422d7b5e | |
Arpit Agarwal | 6c009a3d04 | |
Xiaoyu Yao | b2c63c3ce1 | |
Hanisha Koneru | 598c16c1fc | |
Bharat Viswanadham | 3edb697a37 | |
Xiaoyu Yao | a55a295c7a | |
Ajay Yadav | d2e115d8c8 | |
Arpit Agarwal | 7cd82b4547 | |
Xiaoyu Yao | 6fd95b8595 | |
Vivek Ratnavel Subramanian | 328bfcc0b6 | |
Elek, Márton | 0d35cfc006 | |
Elek, Márton | a009255723 | |
Xiaoyu Yao | 3ca1024eec | |
Elek, Márton | 522976e405 | |
Márton Elek | 3fd9c0db17 | |
Shashikant Banerjee | 8bd13ffa10 | |
Vivek Ratnavel Subramanian | 64726dd8fe | |
Vivek Ratnavel Subramanian | c00e704496 | |
Shashikant Banerjee | 8614da3723 | |
Mukul Kumar Singh | 8e66b6220b | |
Mukul Kumar Singh | f38a8a9630 | |
Ajay Yadav | 0f4ae392d7 | |
Xiaoyu Yao | eeb7d63046 | |
Aravindan Vijayan | 34f0ad064d | |
Lokesh Jain | b5b3f4d823 | |
Vivek Ratnavel Subramanian | 32477ed1e6 | |
Ajay Yadav | 15b6e38ddf | |
Xiaoyu Yao | 262260fadf | |
Mukul Kumar Singh | 97fd3025a4 | |
Shashikant Banerjee | 4373c12701 | |
Xiaoyu Yao | e673bebcfe | |
Ajay Yadav | 6028aefb57 | |
Xiaoyu Yao | 1050bf42fa | |
Xiaoyu Yao | fe41e67ba4 | |
Xiaoyu Yao | 89fb60ffd0 | |
Mukul Kumar Singh | 5d67a17877 | |
Mukul Kumar Singh | aa9de9a0fa | |
Lokesh Jain | 60b5e178e3 | |
Bharat Viswanadham | e65717af25 | |
Xiaoyu Yao | 49217971b5 | |
Ajay Yadav | 6465125a5a | |
Elek, Márton | 118986cde7 | |
Lokesh Jain | c334bc8baf | |
Vivek Ratnavel Subramanian | 62ab3425dc | |
Vivek Ratnavel Subramanian | 67a26ac793 | |
Márton Elek | 9cd3fe49cf | |
Ajay Kumar | 5502da4ae2 | |
Márton Elek | a50ffd8d05 | |
Márton Elek | ab5da9a984 | |
Shashikant Banerjee | 8655abb353 | |
Shashikant Banerjee | b99ae7b4e2 | |
Elek, Márton | 8d906d7217 | |
Anu Engineer | 6eb79815fa | |
Bharat Viswanadham | 4cf50e3972 | |
Arpit Agarwal | c57e1ddd63 | |
Elek, Márton | f4918acdae | |
Ajay Yadav | c3986dc53b |
|
@ -249,7 +249,7 @@ public class NodeHealthScriptRunner extends AbstractService {
|
|||
}
|
||||
|
||||
/**
|
||||
* Sets if the node is healhty or not considering disks' health also.
|
||||
* Sets if the node is healthy or not considering disks' health also.
|
||||
*
|
||||
* @param isHealthy
|
||||
* if or not node is healthy
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
|
|||
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
||||
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.tracing.GrpcClientInterceptor;
|
||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
@ -136,7 +138,8 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|||
NettyChannelBuilder channelBuilder = NettyChannelBuilder.forAddress(dn
|
||||
.getIpAddress(), port).usePlaintext()
|
||||
.maxInboundMessageSize(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE)
|
||||
.intercept(new ClientCredentialInterceptor(userName, encodedToken));
|
||||
.intercept(new ClientCredentialInterceptor(userName, encodedToken),
|
||||
new GrpcClientInterceptor());
|
||||
if (secConfig.isGrpcTlsEnabled()) {
|
||||
File trustCertCollectionFile = secConfig.getTrustStoreFile();
|
||||
File privateKeyFile = secConfig.getClientPrivateKeyFile();
|
||||
|
@ -204,7 +207,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|||
ContainerCommandRequestProto request) throws IOException {
|
||||
try {
|
||||
XceiverClientReply reply;
|
||||
reply = sendCommandWithRetry(request, null);
|
||||
reply = sendCommandWithTraceIDAndRetry(request, null);
|
||||
ContainerCommandResponseProto responseProto = reply.getResponse().get();
|
||||
return responseProto;
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
|
@ -217,7 +220,21 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
||||
throws IOException {
|
||||
Preconditions.checkState(HddsUtils.isReadOnly(request));
|
||||
return sendCommandWithRetry(request, excludeDns);
|
||||
return sendCommandWithTraceIDAndRetry(request, excludeDns);
|
||||
}
|
||||
|
||||
private XceiverClientReply sendCommandWithTraceIDAndRetry(
|
||||
ContainerCommandRequestProto request, List<DatanodeDetails> excludeDns)
|
||||
throws IOException {
|
||||
try (Scope scope = GlobalTracer.get()
|
||||
.buildSpan("XceiverClientGrpc." + request.getCmdType().name())
|
||||
.startActive(true)) {
|
||||
ContainerCommandRequestProto finalPayload =
|
||||
ContainerCommandRequestProto.newBuilder(request)
|
||||
.setTraceID(TracingUtil.exportCurrentSpan())
|
||||
.build();
|
||||
return sendCommandWithRetry(finalPayload, excludeDns);
|
||||
}
|
||||
}
|
||||
|
||||
private XceiverClientReply sendCommandWithRetry(
|
||||
|
@ -253,7 +270,7 @@ public class XceiverClientGrpc extends XceiverClientSpi {
|
|||
break;
|
||||
}
|
||||
} catch (ExecutionException | InterruptedException e) {
|
||||
LOG.warn("Failed to execute command " + request + " on datanode " + dn
|
||||
LOG.debug("Failed to execute command " + request + " on datanode " + dn
|
||||
.getUuidString(), e);
|
||||
if (Status.fromThrowable(e.getCause()).getCode()
|
||||
== Status.UNAUTHENTICATED.getCode()) {
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdds.scm;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
import org.apache.hadoop.metrics2.MetricsSystem;
|
||||
|
@ -37,7 +38,9 @@ public class XceiverClientMetrics {
|
|||
.getSimpleName();
|
||||
|
||||
private @Metric MutableCounterLong pendingOps;
|
||||
private @Metric MutableCounterLong totalOps;
|
||||
private MutableCounterLong[] pendingOpsArray;
|
||||
private MutableCounterLong[] opsArray;
|
||||
private MutableRate[] containerOpsLatency;
|
||||
private MetricsRegistry registry;
|
||||
|
||||
|
@ -46,12 +49,17 @@ public class XceiverClientMetrics {
|
|||
this.registry = new MetricsRegistry(SOURCE_NAME);
|
||||
|
||||
this.pendingOpsArray = new MutableCounterLong[numEnumEntries];
|
||||
this.opsArray = new MutableCounterLong[numEnumEntries];
|
||||
this.containerOpsLatency = new MutableRate[numEnumEntries];
|
||||
for (int i = 0; i < numEnumEntries; i++) {
|
||||
pendingOpsArray[i] = registry.newCounter(
|
||||
"numPending" + ContainerProtos.Type.forNumber(i + 1),
|
||||
"number of pending" + ContainerProtos.Type.forNumber(i + 1) + " ops",
|
||||
(long) 0);
|
||||
opsArray[i] = registry
|
||||
.newCounter("opCount" + ContainerProtos.Type.forNumber(i + 1),
|
||||
"number of" + ContainerProtos.Type.forNumber(i + 1) + " ops",
|
||||
(long) 0);
|
||||
|
||||
containerOpsLatency[i] = registry.newRate(
|
||||
ContainerProtos.Type.forNumber(i + 1) + "Latency",
|
||||
|
@ -68,6 +76,8 @@ public class XceiverClientMetrics {
|
|||
|
||||
public void incrPendingContainerOpsMetrics(ContainerProtos.Type type) {
|
||||
pendingOps.incr();
|
||||
totalOps.incr();
|
||||
opsArray[type.ordinal()].incr();
|
||||
pendingOpsArray[type.ordinal()].incr();
|
||||
}
|
||||
|
||||
|
@ -85,6 +95,16 @@ public class XceiverClientMetrics {
|
|||
return pendingOpsArray[type.ordinal()].value();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getTotalOpCount() {
|
||||
return totalOps.value();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getContainerOpCountMetrics(ContainerProtos.Type type) {
|
||||
return opsArray[type.ordinal()].value();
|
||||
}
|
||||
|
||||
public void unRegister() {
|
||||
MetricsSystem ms = DefaultMetricsSystem.instance();
|
||||
ms.unregisterSource(SOURCE_NAME);
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
package org.apache.hadoop.hdds.scm;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hdds.HddsUtils;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
|
@ -25,6 +26,7 @@ import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
|||
|
||||
import io.opentracing.Scope;
|
||||
import io.opentracing.util.GlobalTracer;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.ratis.grpc.GrpcTlsConfig;
|
||||
import org.apache.ratis.proto.RaftProtos;
|
||||
import org.apache.ratis.protocol.RaftRetryFailureException;
|
||||
|
@ -47,6 +49,7 @@ import org.apache.ratis.protocol.RaftClientReply;
|
|||
import org.apache.ratis.rpc.RpcType;
|
||||
import org.apache.ratis.rpc.SupportedRpcType;
|
||||
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
|
||||
import org.apache.ratis.util.TimeDuration;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -74,6 +77,8 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
final String rpcType = ozoneConf
|
||||
.get(ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_KEY,
|
||||
ScmConfigKeys.DFS_CONTAINER_RATIS_RPC_TYPE_DEFAULT);
|
||||
final TimeDuration clientRequestTimeout =
|
||||
RatisHelper.getClientRequestTimeout(ozoneConf);
|
||||
final int maxOutstandingRequests =
|
||||
HddsClientUtils.getMaxOutstandingRequests(ozoneConf);
|
||||
final RetryPolicy retryPolicy = RatisHelper.createRetryPolicy(ozoneConf);
|
||||
|
@ -81,7 +86,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
SecurityConfig(ozoneConf));
|
||||
return new XceiverClientRatis(pipeline,
|
||||
SupportedRpcType.valueOfIgnoreCase(rpcType), maxOutstandingRequests,
|
||||
retryPolicy, tlsConfig);
|
||||
retryPolicy, tlsConfig, clientRequestTimeout);
|
||||
}
|
||||
|
||||
private final Pipeline pipeline;
|
||||
|
@ -90,6 +95,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
private final int maxOutstandingRequests;
|
||||
private final RetryPolicy retryPolicy;
|
||||
private final GrpcTlsConfig tlsConfig;
|
||||
private final TimeDuration clientRequestTimeout;
|
||||
|
||||
// Map to track commit index at every server
|
||||
private final ConcurrentHashMap<UUID, Long> commitInfoMap;
|
||||
|
@ -97,12 +103,14 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
// create a separate RaftClient for watchForCommit API
|
||||
private RaftClient watchClient;
|
||||
|
||||
private XceiverClientMetrics metrics;
|
||||
|
||||
/**
|
||||
* Constructs a client.
|
||||
*/
|
||||
private XceiverClientRatis(Pipeline pipeline, RpcType rpcType,
|
||||
int maxOutStandingChunks, RetryPolicy retryPolicy,
|
||||
GrpcTlsConfig tlsConfig) {
|
||||
GrpcTlsConfig tlsConfig, TimeDuration timeout) {
|
||||
super();
|
||||
this.pipeline = pipeline;
|
||||
this.rpcType = rpcType;
|
||||
|
@ -111,6 +119,8 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
commitInfoMap = new ConcurrentHashMap<>();
|
||||
watchClient = null;
|
||||
this.tlsConfig = tlsConfig;
|
||||
this.clientRequestTimeout = timeout;
|
||||
metrics = XceiverClientManager.getXceiverClientMetrics();
|
||||
}
|
||||
|
||||
private void updateCommitInfosMap(
|
||||
|
@ -160,7 +170,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
// requests to be handled by raft client
|
||||
if (!client.compareAndSet(null,
|
||||
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
||||
maxOutstandingRequests, tlsConfig))) {
|
||||
maxOutstandingRequests, tlsConfig, clientRequestTimeout))) {
|
||||
throw new IllegalStateException("Client is already connected.");
|
||||
}
|
||||
}
|
||||
|
@ -194,6 +204,12 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
return Objects.requireNonNull(client.get(), "client is null");
|
||||
}
|
||||
|
||||
|
||||
@VisibleForTesting
|
||||
public ConcurrentHashMap<UUID, Long> getCommitInfoMap() {
|
||||
return commitInfoMap;
|
||||
}
|
||||
|
||||
private CompletableFuture<RaftClientReply> sendRequestAsync(
|
||||
ContainerCommandRequestProto request) {
|
||||
try (Scope scope = GlobalTracer.get()
|
||||
|
@ -243,7 +259,7 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
if (watchClient == null) {
|
||||
watchClient =
|
||||
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
||||
maxOutstandingRequests, tlsConfig);
|
||||
maxOutstandingRequests, tlsConfig, clientRequestTimeout);
|
||||
}
|
||||
CompletableFuture<RaftClientReply> replyFuture = watchClient
|
||||
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
|
||||
|
@ -260,9 +276,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
// TODO : need to remove the code to create the new RaftClient instance
|
||||
// here once the watch request bypassing sliding window in Raft Client
|
||||
// gets fixed.
|
||||
watchClient = RatisHelper
|
||||
.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
||||
maxOutstandingRequests, tlsConfig);
|
||||
watchClient =
|
||||
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy,
|
||||
maxOutstandingRequests, tlsConfig, clientRequestTimeout);
|
||||
reply = watchClient
|
||||
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
|
||||
.get(timeout, TimeUnit.MILLISECONDS);
|
||||
|
@ -296,15 +312,20 @@ public final class XceiverClientRatis extends XceiverClientSpi {
|
|||
public XceiverClientReply sendCommandAsync(
|
||||
ContainerCommandRequestProto request) {
|
||||
XceiverClientReply asyncReply = new XceiverClientReply(null);
|
||||
long requestTime = Time.monotonicNowNanos();
|
||||
CompletableFuture<RaftClientReply> raftClientReply =
|
||||
sendRequestAsync(request);
|
||||
metrics.incrPendingContainerOpsMetrics(request.getCmdType());
|
||||
CompletableFuture<ContainerCommandResponseProto> containerCommandResponse =
|
||||
raftClientReply.whenComplete((reply, e) -> LOG.debug(
|
||||
"received reply {} for request: cmdType={} containerID={}"
|
||||
raftClientReply.whenComplete((reply, e) -> {
|
||||
LOG.debug("received reply {} for request: cmdType={} containerID={}"
|
||||
+ " pipelineID={} traceID={} exception: {}", reply,
|
||||
request.getCmdType(), request.getContainerID(),
|
||||
request.getPipelineID(), request.getTraceID(), e))
|
||||
.thenApply(reply -> {
|
||||
request.getPipelineID(), request.getTraceID(), e);
|
||||
metrics.decrPendingContainerOpsMetrics(request.getCmdType());
|
||||
metrics.addContainerOpsLatency(request.getCmdType(),
|
||||
Time.monotonicNowNanos() - requestTime);
|
||||
}).thenApply(reply -> {
|
||||
try {
|
||||
// we need to handle RaftRetryFailure Exception
|
||||
RaftRetryFailureException raftRetryFailureException =
|
||||
|
|
|
@ -17,6 +17,7 @@
|
|||
*/
|
||||
|
||||
package org.apache.hadoop.hdds.scm.storage;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
|
@ -40,13 +41,22 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.Buffer;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.Collections;
|
||||
import java.util.UUID;
|
||||
import java.util.List;
|
||||
import java.util.ArrayList;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.CompletableFuture;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.CompletionException;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
|
||||
.putBlockAsync;
|
||||
|
@ -87,11 +97,11 @@ public class BlockOutputStream extends OutputStream {
|
|||
private final long streamBufferFlushSize;
|
||||
private final long streamBufferMaxSize;
|
||||
private final long watchTimeout;
|
||||
private List<ByteBuffer> bufferList;
|
||||
private BufferPool bufferPool;
|
||||
// The IOException will be set by response handling thread in case there is an
|
||||
// exception received in the response. If the exception is set, the next
|
||||
// request will fail upfront.
|
||||
private IOException ioException;
|
||||
private AtomicReference<IOException> ioException;
|
||||
private ExecutorService responseExecutor;
|
||||
|
||||
// the effective length of data flushed so far
|
||||
|
@ -104,14 +114,25 @@ public class BlockOutputStream extends OutputStream {
|
|||
// by all servers
|
||||
private long totalAckDataLength;
|
||||
|
||||
// List containing buffers for which the putBlock call will
|
||||
// update the length in the datanodes. This list will just maintain
|
||||
// references to the buffers in the BufferPool which will be cleared
|
||||
// when the watchForCommit acknowledges a putBlock logIndex has been
|
||||
// committed on all datanodes. This list will be a place holder for buffers
|
||||
// which got written between successive putBlock calls.
|
||||
private List<ByteBuffer> bufferList;
|
||||
|
||||
// future Map to hold up all putBlock futures
|
||||
private ConcurrentHashMap<Long,
|
||||
CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
|
||||
futureMap;
|
||||
// map containing mapping for putBlock logIndex to to flushedDataLength Map.
|
||||
private ConcurrentHashMap<Long, Long> commitIndex2flushedDataMap;
|
||||
|
||||
private int currentBufferIndex;
|
||||
// The map should maintain the keys (logIndexes) in order so that while
|
||||
// removing we always end up updating incremented data flushed length.
|
||||
// Also, corresponding to the logIndex, the corresponding list of buffers will
|
||||
// be released from the buffer pool.
|
||||
private ConcurrentSkipListMap<Long, List<ByteBuffer>>
|
||||
commitIndex2flushedDataMap;
|
||||
|
||||
private List<DatanodeDetails> failedServers;
|
||||
|
||||
|
@ -124,7 +145,7 @@ public class BlockOutputStream extends OutputStream {
|
|||
* @param pipeline pipeline where block will be written
|
||||
* @param traceID container protocol call args
|
||||
* @param chunkSize chunk size
|
||||
* @param bufferList list of byte buffers
|
||||
* @param bufferPool pool of buffers
|
||||
* @param streamBufferFlushSize flush size
|
||||
* @param streamBufferMaxSize max size of the currentBuffer
|
||||
* @param watchTimeout watch timeout
|
||||
|
@ -135,7 +156,7 @@ public class BlockOutputStream extends OutputStream {
|
|||
public BlockOutputStream(BlockID blockID, String key,
|
||||
XceiverClientManager xceiverClientManager, Pipeline pipeline,
|
||||
String traceID, int chunkSize, long streamBufferFlushSize,
|
||||
long streamBufferMaxSize, long watchTimeout, List<ByteBuffer> bufferList,
|
||||
long streamBufferMaxSize, long watchTimeout, BufferPool bufferPool,
|
||||
ChecksumType checksumType, int bytesPerChecksum)
|
||||
throws IOException {
|
||||
this.blockID = blockID;
|
||||
|
@ -154,26 +175,28 @@ public class BlockOutputStream extends OutputStream {
|
|||
this.streamBufferFlushSize = streamBufferFlushSize;
|
||||
this.streamBufferMaxSize = streamBufferMaxSize;
|
||||
this.watchTimeout = watchTimeout;
|
||||
this.bufferList = bufferList;
|
||||
this.bufferPool = bufferPool;
|
||||
this.checksumType = checksumType;
|
||||
this.bytesPerChecksum = bytesPerChecksum;
|
||||
|
||||
// A single thread executor handle the responses of async requests
|
||||
responseExecutor = Executors.newSingleThreadExecutor();
|
||||
commitIndex2flushedDataMap = new ConcurrentHashMap<>();
|
||||
commitIndex2flushedDataMap = new ConcurrentSkipListMap<>();
|
||||
totalAckDataLength = 0;
|
||||
futureMap = new ConcurrentHashMap<>();
|
||||
totalDataFlushedLength = 0;
|
||||
currentBufferIndex = 0;
|
||||
writtenDataLength = 0;
|
||||
failedServers = Collections.emptyList();
|
||||
bufferList = null;
|
||||
ioException = new AtomicReference<>(null);
|
||||
}
|
||||
|
||||
|
||||
public BlockID getBlockID() {
|
||||
return blockID;
|
||||
}
|
||||
|
||||
public long getTotalSuccessfulFlushedData() {
|
||||
public long getTotalAckDataLength() {
|
||||
return totalAckDataLength;
|
||||
}
|
||||
|
||||
|
@ -181,17 +204,34 @@ public class BlockOutputStream extends OutputStream {
|
|||
return writtenDataLength;
|
||||
}
|
||||
|
||||
private long computeBufferData() {
|
||||
int dataLength =
|
||||
bufferList.stream().mapToInt(Buffer::position).sum();
|
||||
Preconditions.checkState(dataLength <= streamBufferMaxSize);
|
||||
return dataLength;
|
||||
}
|
||||
|
||||
public List<DatanodeDetails> getFailedServers() {
|
||||
return failedServers;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public XceiverClientSpi getXceiverClient() {
|
||||
return xceiverClient;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public long getTotalDataFlushedLength() {
|
||||
return totalDataFlushedLength;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public BufferPool getBufferPool() {
|
||||
return bufferPool;
|
||||
}
|
||||
|
||||
public IOException getIoException() {
|
||||
return ioException.get();
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<Long, List<ByteBuffer>> getCommitIndex2flushedDataMap() {
|
||||
return commitIndex2flushedDataMap;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(int b) throws IOException {
|
||||
checkOpen();
|
||||
|
@ -202,6 +242,7 @@ public class BlockOutputStream extends OutputStream {
|
|||
|
||||
@Override
|
||||
public void write(byte[] b, int off, int len) throws IOException {
|
||||
checkOpen();
|
||||
if (b == null) {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
|
@ -212,54 +253,45 @@ public class BlockOutputStream extends OutputStream {
|
|||
if (len == 0) {
|
||||
return;
|
||||
}
|
||||
|
||||
while (len > 0) {
|
||||
checkOpen();
|
||||
int writeLen;
|
||||
allocateBuffer();
|
||||
ByteBuffer currentBuffer = getCurrentBuffer();
|
||||
// Allocate a buffer if needed. The buffer will be allocated only
|
||||
// once as needed and will be reused again for multiple blockOutputStream
|
||||
// entries.
|
||||
ByteBuffer currentBuffer = bufferPool.allocateBufferIfNeeded();
|
||||
int pos = currentBuffer.position();
|
||||
writeLen =
|
||||
Math.min(chunkSize - currentBuffer.position() % chunkSize, len);
|
||||
Math.min(chunkSize - pos % chunkSize, len);
|
||||
currentBuffer.put(b, off, writeLen);
|
||||
if (currentBuffer.position() % chunkSize == 0) {
|
||||
int pos = currentBuffer.position() - chunkSize;
|
||||
int limit = currentBuffer.position();
|
||||
writeChunk(pos, limit);
|
||||
if (!currentBuffer.hasRemaining()) {
|
||||
writeChunk(currentBuffer);
|
||||
}
|
||||
off += writeLen;
|
||||
len -= writeLen;
|
||||
writtenDataLength += writeLen;
|
||||
if (currentBuffer.position() == streamBufferFlushSize) {
|
||||
totalDataFlushedLength += streamBufferFlushSize;
|
||||
handlePartialFlush();
|
||||
if (shouldFlush()) {
|
||||
updateFlushLength();
|
||||
executePutBlock();
|
||||
}
|
||||
long bufferedData = computeBufferData();
|
||||
// Data in the bufferList can not exceed streamBufferMaxSize
|
||||
if (bufferedData == streamBufferMaxSize) {
|
||||
// Data in the bufferPool can not exceed streamBufferMaxSize
|
||||
if (isBufferPoolFull()) {
|
||||
handleFullBuffer();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private ByteBuffer getCurrentBuffer() {
|
||||
ByteBuffer buffer = bufferList.get(currentBufferIndex);
|
||||
if (!buffer.hasRemaining()) {
|
||||
currentBufferIndex =
|
||||
currentBufferIndex < getMaxNumBuffers() - 1 ? ++currentBufferIndex :
|
||||
0;
|
||||
}
|
||||
return bufferList.get(currentBufferIndex);
|
||||
private boolean shouldFlush() {
|
||||
return bufferPool.computeBufferData() % streamBufferFlushSize == 0;
|
||||
}
|
||||
|
||||
private int getMaxNumBuffers() {
|
||||
return (int)(streamBufferMaxSize/streamBufferFlushSize);
|
||||
private void updateFlushLength() {
|
||||
totalDataFlushedLength += writtenDataLength - totalDataFlushedLength;
|
||||
}
|
||||
|
||||
private void allocateBuffer() {
|
||||
for (int i = bufferList.size(); i < getMaxNumBuffers(); i++) {
|
||||
bufferList.add(ByteBuffer.allocate((int)streamBufferFlushSize));
|
||||
private boolean isBufferPoolFull() {
|
||||
return bufferPool.computeBufferData() == streamBufferMaxSize;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Will be called on the retryPath in case closedContainerException/
|
||||
* TimeoutException.
|
||||
|
@ -272,51 +304,59 @@ public class BlockOutputStream extends OutputStream {
|
|||
if (len == 0) {
|
||||
return;
|
||||
}
|
||||
int off = 0;
|
||||
int pos = off;
|
||||
int count = 0;
|
||||
Preconditions.checkArgument(len <= streamBufferMaxSize);
|
||||
while (len > 0) {
|
||||
long writeLen;
|
||||
writeLen = Math.min(chunkSize, len);
|
||||
if (writeLen == chunkSize) {
|
||||
int limit = pos + chunkSize;
|
||||
writeChunk(pos, limit);
|
||||
writeChunk(bufferPool.getBuffer(count));
|
||||
}
|
||||
off += writeLen;
|
||||
len -= writeLen;
|
||||
count++;
|
||||
writtenDataLength += writeLen;
|
||||
if (off % streamBufferFlushSize == 0) {
|
||||
// reset the position to zero as now we wll readng thhe next buffer in
|
||||
// the list
|
||||
pos = 0;
|
||||
totalDataFlushedLength += streamBufferFlushSize;
|
||||
handlePartialFlush();
|
||||
// we should not call isBufferFull/shouldFlush here.
|
||||
// The buffer might already be full as whole data is already cached in
|
||||
// the buffer. We should just validate
|
||||
// if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
|
||||
// call for handling full buffer/flush buffer condition.
|
||||
if (writtenDataLength % streamBufferFlushSize == 0) {
|
||||
// reset the position to zero as now we will be reading the
|
||||
// next buffer in the list
|
||||
updateFlushLength();
|
||||
executePutBlock();
|
||||
}
|
||||
if (computeBufferData() % streamBufferMaxSize == 0) {
|
||||
if (writtenDataLength == streamBufferMaxSize) {
|
||||
handleFullBuffer();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* just update the totalAckDataLength. Since we have allocated
|
||||
* the currentBuffer more than the streamBufferMaxSize, we can keep on writing
|
||||
* to the currentBuffer. In case of failure, we will read the data starting
|
||||
* from totalAckDataLength.
|
||||
* just update the totalAckDataLength. In case of failure,
|
||||
* we will read the data starting from totalAckDataLength.
|
||||
*/
|
||||
private void updateFlushIndex(long index) {
|
||||
if (!commitIndex2flushedDataMap.isEmpty()) {
|
||||
private void updateFlushIndex(List<Long> indexes) {
|
||||
Preconditions.checkArgument(!commitIndex2flushedDataMap.isEmpty());
|
||||
for (long index : indexes) {
|
||||
Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
|
||||
totalAckDataLength = commitIndex2flushedDataMap.remove(index);
|
||||
List<ByteBuffer> buffers = commitIndex2flushedDataMap.remove(index);
|
||||
long length = buffers.stream().mapToLong(value -> {
|
||||
int pos = value.position();
|
||||
Preconditions.checkArgument(pos <= chunkSize);
|
||||
return pos;
|
||||
}).sum();
|
||||
// totalAckDataLength replicated yet should always be incremented
|
||||
// with the current length being returned from commitIndex2flushedDataMap.
|
||||
totalAckDataLength += length;
|
||||
LOG.debug("Total data successfully replicated: " + totalAckDataLength);
|
||||
futureMap.remove(totalAckDataLength);
|
||||
// Flush has been committed to required servers successful.
|
||||
// just swap the bufferList head and tail after clearing.
|
||||
ByteBuffer currentBuffer = bufferList.remove(0);
|
||||
currentBuffer.clear();
|
||||
if (currentBufferIndex != 0) {
|
||||
currentBufferIndex--;
|
||||
// just release the current buffer from the buffer pool corresponding
|
||||
// to the buffers that have been committed with the putBlock call.
|
||||
for (ByteBuffer byteBuffer : buffers) {
|
||||
bufferPool.releaseBuffer(byteBuffer);
|
||||
}
|
||||
bufferList.add(currentBuffer);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -333,9 +373,9 @@ public class BlockOutputStream extends OutputStream {
|
|||
waitOnFlushFutures();
|
||||
}
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
setIoException(e);
|
||||
adjustBuffersOnException();
|
||||
throw new IOException(
|
||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||
throw getIoException();
|
||||
}
|
||||
if (!commitIndex2flushedDataMap.isEmpty()) {
|
||||
watchForCommit(
|
||||
|
@ -345,13 +385,13 @@ public class BlockOutputStream extends OutputStream {
|
|||
}
|
||||
|
||||
private void adjustBuffers(long commitIndex) {
|
||||
commitIndex2flushedDataMap.keySet().stream().forEach(index -> {
|
||||
if (index <= commitIndex) {
|
||||
updateFlushIndex(index);
|
||||
} else {
|
||||
List<Long> keyList = commitIndex2flushedDataMap.keySet().stream()
|
||||
.filter(p -> p <= commitIndex).collect(Collectors.toList());
|
||||
if (keyList.isEmpty()) {
|
||||
return;
|
||||
} else {
|
||||
updateFlushIndex(keyList);
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
// It may happen that once the exception is encountered , we still might
|
||||
|
@ -390,17 +430,21 @@ public class BlockOutputStream extends OutputStream {
|
|||
adjustBuffers(index);
|
||||
} catch (TimeoutException | InterruptedException | ExecutionException e) {
|
||||
LOG.warn("watchForCommit failed for index " + commitIndex, e);
|
||||
setIoException(e);
|
||||
adjustBuffersOnException();
|
||||
throw new IOException(
|
||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||
throw getIoException();
|
||||
}
|
||||
}
|
||||
|
||||
private CompletableFuture<ContainerProtos.
|
||||
ContainerCommandResponseProto> handlePartialFlush()
|
||||
ContainerCommandResponseProto> executePutBlock()
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
long flushPos = totalDataFlushedLength;
|
||||
Preconditions.checkNotNull(bufferList);
|
||||
List<ByteBuffer> byteBufferList = bufferList;
|
||||
bufferList = null;
|
||||
Preconditions.checkNotNull(byteBufferList);
|
||||
String requestId =
|
||||
traceID + ContainerProtos.Type.PutBlock + chunkIndex + blockID;
|
||||
CompletableFuture<ContainerProtos.
|
||||
|
@ -417,18 +461,23 @@ public class BlockOutputStream extends OutputStream {
|
|||
throw new CompletionException(sce);
|
||||
}
|
||||
// if the ioException is not set, putBlock is successful
|
||||
if (ioException == null) {
|
||||
LOG.debug(
|
||||
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
|
||||
+ commitIndex2flushedDataMap.size());
|
||||
if (getIoException() == null) {
|
||||
BlockID responseBlockID = BlockID.getFromProtobuf(
|
||||
e.getPutBlock().getCommittedBlockLength().getBlockID());
|
||||
Preconditions.checkState(blockID.getContainerBlockID()
|
||||
.equals(responseBlockID.getContainerBlockID()));
|
||||
// updates the bcsId of the block
|
||||
blockID = responseBlockID;
|
||||
LOG.debug(
|
||||
"Adding index " + asyncReply.getLogIndex() + " commitMap size "
|
||||
+ commitIndex2flushedDataMap.size() + " flushLength "
|
||||
+ flushPos + " numBuffers " + byteBufferList.size()
|
||||
+ " blockID " + blockID + " bufferPool size" + bufferPool
|
||||
.getSize() + " currentBufferIndex " + bufferPool
|
||||
.getCurrentBufferIndex());
|
||||
// for standalone protocol, logIndex will always be 0.
|
||||
commitIndex2flushedDataMap.put(asyncReply.getLogIndex(), flushPos);
|
||||
commitIndex2flushedDataMap
|
||||
.put(asyncReply.getLogIndex(), byteBufferList);
|
||||
}
|
||||
return e;
|
||||
}, responseExecutor).exceptionally(e -> {
|
||||
|
@ -450,91 +499,101 @@ public class BlockOutputStream extends OutputStream {
|
|||
@Override
|
||||
public void flush() throws IOException {
|
||||
if (xceiverClientManager != null && xceiverClient != null
|
||||
&& bufferList != null) {
|
||||
checkOpen();
|
||||
int bufferSize = bufferList.size();
|
||||
if (bufferSize > 0) {
|
||||
&& bufferPool != null && bufferPool.getSize() > 0) {
|
||||
try {
|
||||
// flush the last chunk data residing on the currentBuffer
|
||||
if (totalDataFlushedLength < writtenDataLength) {
|
||||
ByteBuffer currentBuffer = getCurrentBuffer();
|
||||
int pos = currentBuffer.position() - (currentBuffer.position()
|
||||
% chunkSize);
|
||||
int limit = currentBuffer.position() - pos;
|
||||
writeChunk(pos, currentBuffer.position());
|
||||
totalDataFlushedLength += limit;
|
||||
handlePartialFlush();
|
||||
}
|
||||
waitOnFlushFutures();
|
||||
// just check again if the exception is hit while waiting for the
|
||||
// futures to ensure flush has indeed succeeded
|
||||
checkOpen();
|
||||
handleFlush();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
// just set the exception here as well in order to maintain sanctity of
|
||||
// ioException field
|
||||
setIoException(e);
|
||||
adjustBuffersOnException();
|
||||
throw new IOException(
|
||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||
}
|
||||
throw getIoException();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void writeChunk(int pos, int limit) throws IOException {
|
||||
|
||||
private void writeChunk(ByteBuffer buffer)
|
||||
throws IOException {
|
||||
// This data in the buffer will be pushed to datanode and a reference will
|
||||
// be added to the bufferList. Once putBlock gets executed, this list will
|
||||
// be marked null. Hence, during first writeChunk call after every putBlock
|
||||
// call or during the first call to writeChunk here, the list will be null.
|
||||
|
||||
if (bufferList == null) {
|
||||
bufferList = new ArrayList<>();
|
||||
}
|
||||
bufferList.add(buffer);
|
||||
// Please note : We are not flipping the slice when we write since
|
||||
// the slices are pointing the currentBuffer start and end as needed for
|
||||
// the chunk write. Also please note, Duplicate does not create a
|
||||
// copy of data, it only creates metadata that points to the data
|
||||
// stream.
|
||||
ByteBuffer chunk = bufferList.get(currentBufferIndex).duplicate();
|
||||
chunk.position(pos);
|
||||
chunk.limit(limit);
|
||||
ByteBuffer chunk = buffer.duplicate();
|
||||
chunk.position(0);
|
||||
chunk.limit(buffer.position());
|
||||
writeChunkToContainer(chunk);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (xceiverClientManager != null && xceiverClient != null
|
||||
&& bufferList != null) {
|
||||
int bufferSize = bufferList.size();
|
||||
if (bufferSize > 0) {
|
||||
try {
|
||||
private void handleFlush()
|
||||
throws IOException, InterruptedException, ExecutionException {
|
||||
checkOpen();
|
||||
// flush the last chunk data residing on the currentBuffer
|
||||
if (totalDataFlushedLength < writtenDataLength) {
|
||||
ByteBuffer currentBuffer = getCurrentBuffer();
|
||||
int pos = currentBuffer.position() - (currentBuffer.position()
|
||||
% chunkSize);
|
||||
int limit = currentBuffer.position() - pos;
|
||||
writeChunk(pos, currentBuffer.position());
|
||||
totalDataFlushedLength += limit;
|
||||
handlePartialFlush();
|
||||
ByteBuffer currentBuffer = bufferPool.getCurrentBuffer();
|
||||
Preconditions.checkArgument(currentBuffer.position() > 0);
|
||||
if (currentBuffer.position() != chunkSize) {
|
||||
writeChunk(currentBuffer);
|
||||
}
|
||||
// This can be a partially filled chunk. Since we are flushing the buffer
|
||||
// here, we just limit this buffer to the current position. So that next
|
||||
// write will happen in new buffer
|
||||
updateFlushLength();
|
||||
executePutBlock();
|
||||
}
|
||||
waitOnFlushFutures();
|
||||
// irrespective of whether the commitIndex2flushedDataMap is empty
|
||||
// or not, ensure there is no exception set
|
||||
checkOpen();
|
||||
if (!commitIndex2flushedDataMap.isEmpty()) {
|
||||
// wait for the last commit index in the commitIndex2flushedDataMap
|
||||
// to get committed to all or majority of nodes in case timeout
|
||||
// happens.
|
||||
long lastIndex =
|
||||
commitIndex2flushedDataMap.keySet().stream()
|
||||
.mapToLong(v -> v).max().getAsLong();
|
||||
commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v)
|
||||
.max().getAsLong();
|
||||
LOG.debug(
|
||||
"waiting for last flush Index " + lastIndex + " to catch up");
|
||||
watchForCommit(lastIndex);
|
||||
}
|
||||
|
||||
// just check again if the exception is hit while waiting for the
|
||||
// futures to ensure flush has indeed succeeded
|
||||
|
||||
// irrespective of whether the commitIndex2flushedDataMap is empty
|
||||
// or not, ensure there is no exception set
|
||||
checkOpen();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (xceiverClientManager != null && xceiverClient != null
|
||||
&& bufferPool != null && bufferPool.getSize() > 0) {
|
||||
try {
|
||||
handleFlush();
|
||||
} catch (InterruptedException | ExecutionException e) {
|
||||
setIoException(e);
|
||||
adjustBuffersOnException();
|
||||
throw new IOException(
|
||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||
throw getIoException();
|
||||
} finally {
|
||||
cleanup(false);
|
||||
}
|
||||
}
|
||||
// clear the currentBuffer
|
||||
bufferList.stream().forEach(ByteBuffer::clear);
|
||||
// TODO: Turn the below buffer empty check on whne Standalone pipeline
|
||||
// is removed in the write path in tests
|
||||
// Preconditions.checkArgument(buffer.position() == 0);
|
||||
// bufferPool.checkBufferPoolEmpty();
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private void waitOnFlushFutures()
|
||||
throws InterruptedException, ExecutionException {
|
||||
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
|
||||
|
@ -550,8 +609,9 @@ public class BlockOutputStream extends OutputStream {
|
|||
// if the ioException is already set, it means a prev request has failed
|
||||
// just throw the exception. The current operation will fail with the
|
||||
// original error
|
||||
if (ioException != null) {
|
||||
throw ioException;
|
||||
IOException exception = getIoException();
|
||||
if (exception != null) {
|
||||
throw exception;
|
||||
}
|
||||
ContainerProtocolCalls.validateContainerResponse(responseProto);
|
||||
} catch (StorageContainerException sce) {
|
||||
|
@ -561,10 +621,12 @@ public class BlockOutputStream extends OutputStream {
|
|||
}
|
||||
}
|
||||
|
||||
|
||||
private void setIoException(Exception e) {
|
||||
if (ioException != null) {
|
||||
ioException = new IOException(
|
||||
if (getIoException() == null) {
|
||||
IOException exception = new IOException(
|
||||
"Unexpected Storage Container Exception: " + e.toString(), e);
|
||||
ioException.compareAndSet(null, exception);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -578,6 +640,10 @@ public class BlockOutputStream extends OutputStream {
|
|||
futureMap.clear();
|
||||
}
|
||||
futureMap = null;
|
||||
if (bufferList != null) {
|
||||
bufferList.clear();
|
||||
}
|
||||
bufferList = null;
|
||||
if (commitIndex2flushedDataMap != null) {
|
||||
commitIndex2flushedDataMap.clear();
|
||||
}
|
||||
|
@ -594,9 +660,9 @@ public class BlockOutputStream extends OutputStream {
|
|||
private void checkOpen() throws IOException {
|
||||
if (xceiverClient == null) {
|
||||
throw new IOException("BlockOutputStream has been closed.");
|
||||
} else if (ioException != null) {
|
||||
} else if (getIoException() != null) {
|
||||
adjustBuffersOnException();
|
||||
throw ioException;
|
||||
throw getIoException();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,114 @@
|
|||
/*
|
||||
* 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.storage;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This class creates and manages pool of n buffers.
|
||||
*/
|
||||
public class BufferPool {
|
||||
|
||||
private List<ByteBuffer> bufferList;
|
||||
private int currentBufferIndex;
|
||||
private final int bufferSize;
|
||||
private final int capacity;
|
||||
|
||||
public BufferPool(int bufferSize, int capacity) {
|
||||
this.capacity = capacity;
|
||||
this.bufferSize = bufferSize;
|
||||
bufferList = new ArrayList<>(capacity);
|
||||
currentBufferIndex = -1;
|
||||
}
|
||||
|
||||
public ByteBuffer getCurrentBuffer() {
|
||||
return currentBufferIndex == -1 ? null : bufferList.get(currentBufferIndex);
|
||||
}
|
||||
|
||||
/**
|
||||
* If the currentBufferIndex is less than the buffer size - 1,
|
||||
* it means, the next buffer in the list has been freed up for
|
||||
* rewriting. Reuse the next available buffer in such cases.
|
||||
*
|
||||
* In case, the currentBufferIndex == buffer.size and buffer size is still
|
||||
* less than the capacity to be allocated, just allocate a buffer of size
|
||||
* chunk size.
|
||||
*
|
||||
*/
|
||||
public ByteBuffer allocateBufferIfNeeded() {
|
||||
ByteBuffer buffer = getCurrentBuffer();
|
||||
if (buffer != null && buffer.hasRemaining()) {
|
||||
return buffer;
|
||||
}
|
||||
if (currentBufferIndex < bufferList.size() - 1) {
|
||||
buffer = getBuffer(currentBufferIndex + 1);
|
||||
} else {
|
||||
buffer = ByteBuffer.allocate(bufferSize);
|
||||
bufferList.add(buffer);
|
||||
}
|
||||
Preconditions.checkArgument(bufferList.size() <= capacity);
|
||||
currentBufferIndex++;
|
||||
// TODO: Turn the below precondition check on when Standalone pipeline
|
||||
// is removed in the write path in tests
|
||||
// Preconditions.checkArgument(buffer.position() == 0);
|
||||
return buffer;
|
||||
}
|
||||
|
||||
public void releaseBuffer(ByteBuffer byteBuffer) {
|
||||
// always remove from head of the list and append at last
|
||||
ByteBuffer buffer = bufferList.remove(0);
|
||||
// Ensure the buffer to be removed is always at the head of the list.
|
||||
Preconditions.checkArgument(buffer.equals(byteBuffer));
|
||||
buffer.clear();
|
||||
bufferList.add(buffer);
|
||||
Preconditions.checkArgument(currentBufferIndex >= 0);
|
||||
currentBufferIndex--;
|
||||
}
|
||||
|
||||
public void clearBufferPool() {
|
||||
bufferList.clear();
|
||||
currentBufferIndex = -1;
|
||||
}
|
||||
|
||||
public void checkBufferPoolEmpty() {
|
||||
Preconditions.checkArgument(computeBufferData() == 0);
|
||||
}
|
||||
|
||||
public long computeBufferData() {
|
||||
return bufferList.stream().mapToInt(value -> value.position())
|
||||
.sum();
|
||||
}
|
||||
|
||||
public int getSize() {
|
||||
return bufferList.size();
|
||||
}
|
||||
|
||||
public ByteBuffer getBuffer(int index) {
|
||||
return bufferList.get(index);
|
||||
}
|
||||
|
||||
int getCurrentBufferIndex() {
|
||||
return currentBufferIndex;
|
||||
}
|
||||
|
||||
}
|
|
@ -227,4 +227,24 @@ public final class HddsConfigKeys {
|
|||
public static final String HDDS_SECURITY_CLIENT_SCM_CERTIFICATE_PROTOCOL_ACL =
|
||||
"hdds.security.client.scm.certificate.protocol.acl";
|
||||
|
||||
public static final String HDDS_DATANODE_HTTP_ENABLED_KEY =
|
||||
"hdds.datanode.http.enabled";
|
||||
public static final String HDDS_DATANODE_HTTP_BIND_HOST_KEY =
|
||||
"hdds.datanode.http-bind-host";
|
||||
public static final String HDDS_DATANODE_HTTPS_BIND_HOST_KEY =
|
||||
"hdds.datanode.https-bind-host";
|
||||
public static final String HDDS_DATANODE_HTTP_ADDRESS_KEY =
|
||||
"hdds.datanode.http-address";
|
||||
public static final String HDDS_DATANODE_HTTPS_ADDRESS_KEY =
|
||||
"hdds.datanode.https-address";
|
||||
|
||||
public static final String HDDS_DATANODE_HTTP_BIND_HOST_DEFAULT = "0.0.0.0";
|
||||
public static final int HDDS_DATANODE_HTTP_BIND_PORT_DEFAULT = 9882;
|
||||
public static final int HDDS_DATANODE_HTTPS_BIND_PORT_DEFAULT = 9883;
|
||||
public static final String
|
||||
HDDS_DATANODE_HTTP_KERBEROS_PRINCIPAL_KEY =
|
||||
"hdds.datanode.http.kerberos.principal";
|
||||
public static final String
|
||||
HDDS_DATANODE_HTTP_KERBEROS_KEYTAB_FILE_KEY =
|
||||
"hdds.datanode.http.kerberos.keytab";
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hdds;
|
||||
|
||||
import javax.management.ObjectName;
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.net.InetSocketAddress;
|
||||
|
@ -36,7 +37,15 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolPB;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
|
||||
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
||||
import org.apache.hadoop.ipc.Client;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.metrics2.util.MBeans;
|
||||
import org.apache.hadoop.net.DNS;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
|
@ -48,6 +57,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DNS_NAMESERVER_K
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_HOST_NAME_KEY;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_ENABLED_DEFAULT;
|
||||
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -161,6 +172,29 @@ public final class HddsUtils {
|
|||
.orElse(ScmConfigKeys.OZONE_SCM_BLOCK_CLIENT_PORT_DEFAULT));
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a scm security client.
|
||||
* @param conf - Ozone configuration.
|
||||
* @param address - inet socket address of scm.
|
||||
*
|
||||
* @return {@link SCMSecurityProtocol}
|
||||
* @throws IOException
|
||||
*/
|
||||
public static SCMSecurityProtocol getScmSecurityClient(
|
||||
OzoneConfiguration conf, InetSocketAddress address) throws IOException {
|
||||
RPC.setProtocolEngine(conf, SCMSecurityProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
long scmVersion =
|
||||
RPC.getProtocolVersion(ScmBlockLocationProtocolPB.class);
|
||||
SCMSecurityProtocolClientSideTranslatorPB scmSecurityClient =
|
||||
new SCMSecurityProtocolClientSideTranslatorPB(
|
||||
RPC.getProxy(SCMSecurityProtocolPB.class, scmVersion,
|
||||
address, UserGroupInformation.getCurrentUser(),
|
||||
conf, NetUtils.getDefaultSocketFactory(conf),
|
||||
Client.getRpcTimeout(conf)));
|
||||
return scmSecurityClient;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieve the hostname, trying the supplied config keys in order.
|
||||
* Each config value may be absent, or if present in the format
|
||||
|
|
|
@ -74,8 +74,8 @@ public class BlockID {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
return new StringBuffer().append(getContainerBlockID().toString())
|
||||
.append(" bcId: ")
|
||||
return new StringBuilder().append(getContainerBlockID().toString())
|
||||
.append(" bcsId: ")
|
||||
.append(blockCommitSequenceId)
|
||||
.toString();
|
||||
}
|
||||
|
|
|
@ -30,7 +30,9 @@ import javax.xml.bind.annotation.XmlElement;
|
|||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Enumeration;
|
||||
import java.util.List;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Configuration for ozone.
|
||||
|
@ -161,4 +163,31 @@ public class OzoneConfiguration extends Configuration {
|
|||
Configuration.addDefaultResource("ozone-default.xml");
|
||||
Configuration.addDefaultResource("ozone-site.xml");
|
||||
}
|
||||
|
||||
/**
|
||||
* The super class method getAllPropertiesByTag
|
||||
* does not override values of properties
|
||||
* if there is no tag present in the configs of
|
||||
* newly added resources.
|
||||
* @param tag
|
||||
* @return Properties that belong to the tag
|
||||
*/
|
||||
@Override
|
||||
public Properties getAllPropertiesByTag(String tag) {
|
||||
// Call getProps first to load the newly added resources
|
||||
// before calling super.getAllPropertiesByTag
|
||||
Properties updatedProps = getProps();
|
||||
Properties propertiesByTag = super.getAllPropertiesByTag(tag);
|
||||
Properties props = new Properties();
|
||||
Enumeration properties = propertiesByTag.propertyNames();
|
||||
while (properties.hasMoreElements()) {
|
||||
Object propertyName = properties.nextElement();
|
||||
// get the current value of the property
|
||||
Object value = updatedProps.getProperty(propertyName.toString());
|
||||
if (value != null) {
|
||||
props.put(propertyName, value);
|
||||
}
|
||||
}
|
||||
return props;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -45,6 +45,7 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
private String ipAddress;
|
||||
private String hostName;
|
||||
private List<Port> ports;
|
||||
private String certSerialId;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -54,13 +55,15 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
* @param ipAddress IP Address of this DataNode
|
||||
* @param hostName DataNode's hostname
|
||||
* @param ports Ports used by the DataNode
|
||||
* @param certSerialId serial id from SCM issued certificate.
|
||||
*/
|
||||
private DatanodeDetails(String uuid, String ipAddress, String hostName,
|
||||
List<Port> ports) {
|
||||
List<Port> ports, String certSerialId) {
|
||||
this.uuid = UUID.fromString(uuid);
|
||||
this.ipAddress = ipAddress;
|
||||
this.hostName = hostName;
|
||||
this.ports = ports;
|
||||
this.certSerialId = certSerialId;
|
||||
}
|
||||
|
||||
protected DatanodeDetails(DatanodeDetails datanodeDetails) {
|
||||
|
@ -177,6 +180,9 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
if (datanodeDetailsProto.hasHostName()) {
|
||||
builder.setHostName(datanodeDetailsProto.getHostName());
|
||||
}
|
||||
if (datanodeDetailsProto.hasCertSerialId()) {
|
||||
builder.setCertSerialId(datanodeDetailsProto.getCertSerialId());
|
||||
}
|
||||
for (HddsProtos.Port port : datanodeDetailsProto.getPortsList()) {
|
||||
builder.addPort(newPort(
|
||||
Port.Name.valueOf(port.getName().toUpperCase()), port.getValue()));
|
||||
|
@ -198,6 +204,9 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
if (hostName != null) {
|
||||
builder.setHostName(hostName);
|
||||
}
|
||||
if (certSerialId != null) {
|
||||
builder.setCertSerialId(certSerialId);
|
||||
}
|
||||
for (Port port : ports) {
|
||||
builder.addPorts(HddsProtos.Port.newBuilder()
|
||||
.setName(port.getName().toString())
|
||||
|
@ -214,6 +223,7 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
ipAddress +
|
||||
", host: " +
|
||||
hostName +
|
||||
", certSerialId: " + certSerialId +
|
||||
"}";
|
||||
}
|
||||
|
||||
|
@ -250,6 +260,7 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
private String ipAddress;
|
||||
private String hostName;
|
||||
private List<Port> ports;
|
||||
private String certSerialId;
|
||||
|
||||
/**
|
||||
* Default private constructor. To create Builder instance use
|
||||
|
@ -304,6 +315,18 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds certificate serial id.
|
||||
*
|
||||
* @param certId Serial id of SCM issued certificate.
|
||||
*
|
||||
* @return DatanodeDetails.Builder
|
||||
*/
|
||||
public Builder setCertSerialId(String certId) {
|
||||
this.certSerialId = certId;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds and returns DatanodeDetails instance.
|
||||
*
|
||||
|
@ -311,7 +334,7 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
*/
|
||||
public DatanodeDetails build() {
|
||||
Preconditions.checkNotNull(id);
|
||||
return new DatanodeDetails(id, ipAddress, hostName, ports);
|
||||
return new DatanodeDetails(id, ipAddress, hostName, ports, certSerialId);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -398,4 +421,21 @@ public class DatanodeDetails implements Comparable<DatanodeDetails> {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns serial id of SCM issued certificate.
|
||||
*
|
||||
* @return certificate serial id
|
||||
*/
|
||||
public String getCertSerialId() {
|
||||
return certSerialId;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set certificate serial id of SCM issued certificate.
|
||||
*
|
||||
*/
|
||||
public void setCertSerialId(String certSerialId) {
|
||||
this.certSerialId = certSerialId;
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -121,12 +121,12 @@ public final class ScmConfigKeys {
|
|||
TimeDuration.valueOf(3000, TimeUnit.MILLISECONDS);
|
||||
public static final String DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY =
|
||||
"dfs.ratis.client.request.max.retries";
|
||||
public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT = 180;
|
||||
public static final int DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_DEFAULT = 20;
|
||||
public static final String DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY =
|
||||
"dfs.ratis.client.request.retry.interval";
|
||||
public static final TimeDuration
|
||||
DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_DEFAULT =
|
||||
TimeDuration.valueOf(100, TimeUnit.MILLISECONDS);
|
||||
TimeDuration.valueOf(500, TimeUnit.MILLISECONDS);
|
||||
public static final String DFS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY =
|
||||
"dfs.ratis.server.retry-cache.timeout.duration";
|
||||
public static final TimeDuration
|
||||
|
@ -353,7 +353,8 @@ public final class ScmConfigKeys {
|
|||
"hdds.scm.http.kerberos.principal";
|
||||
public static final String
|
||||
HDDS_SCM_HTTP_KERBEROS_KEYTAB_FILE_KEY =
|
||||
"hdds.scm.http.kerberos.keytab";
|
||||
"hdds.scm.http.kerberos.keytab.file";
|
||||
|
||||
/**
|
||||
* Never constructed.
|
||||
*/
|
||||
|
|
|
@ -23,7 +23,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
|||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||
|
||||
import java.util.*;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.Collection;
|
||||
|
||||
/**
|
||||
* This class contains set of dns and containers which ozone client provides
|
||||
|
@ -73,12 +75,12 @@ public class ExcludeList {
|
|||
public HddsProtos.ExcludeListProto getProtoBuf() {
|
||||
HddsProtos.ExcludeListProto.Builder builder =
|
||||
HddsProtos.ExcludeListProto.newBuilder();
|
||||
containerIds.parallelStream()
|
||||
containerIds
|
||||
.forEach(id -> builder.addContainerIds(id.getId()));
|
||||
datanodes.parallelStream().forEach(dn -> {
|
||||
datanodes.forEach(dn -> {
|
||||
builder.addDatanodes(dn.getUuidString());
|
||||
});
|
||||
pipelineIds.parallelStream().forEach(pipelineID -> {
|
||||
pipelineIds.forEach(pipelineID -> {
|
||||
builder.addPipelineIds(pipelineID.getProtobuf());
|
||||
});
|
||||
return builder.build();
|
||||
|
@ -87,7 +89,7 @@ public class ExcludeList {
|
|||
public static ExcludeList getFromProtoBuf(
|
||||
HddsProtos.ExcludeListProto excludeListProto) {
|
||||
ExcludeList excludeList = new ExcludeList();
|
||||
excludeListProto.getContainerIdsList().parallelStream().forEach(id -> {
|
||||
excludeListProto.getContainerIdsList().forEach(id -> {
|
||||
excludeList.addConatinerId(ContainerID.valueof(id));
|
||||
});
|
||||
DatanodeDetails.Builder builder = DatanodeDetails.newBuilder();
|
||||
|
|
|
@ -48,14 +48,17 @@ public interface ScmBlockLocationProtocol extends Closeable {
|
|||
* Asks SCM where a block should be allocated. SCM responds with the
|
||||
* set of datanodes that should be used creating this block.
|
||||
* @param size - size of the block.
|
||||
* @param numBlocks - number of blocks.
|
||||
* @param type - replication type of the blocks.
|
||||
* @param factor - replication factor of the blocks.
|
||||
* @param excludeList List of datanodes/containers to exclude during block
|
||||
* allocation.
|
||||
* @return allocated block accessing info (key, pipeline).
|
||||
* @throws IOException
|
||||
*/
|
||||
AllocatedBlock allocateBlock(long size, ReplicationType type,
|
||||
ReplicationFactor factor, String owner, ExcludeList excludeList)
|
||||
throws IOException;
|
||||
List<AllocatedBlock> allocateBlock(long size, int numBlocks,
|
||||
ReplicationType type, ReplicationFactor factor, String owner,
|
||||
ExcludeList excludeList) throws IOException;
|
||||
|
||||
/**
|
||||
* Delete blocks for a set of object keys.
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.stream.Collectors;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdds.client.ContainerBlockID;
|
||||
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
|
||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateBlockResponse;
|
||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.AllocateScmBlockResponseProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos.DeleteScmKeyBlocksRequestProto;
|
||||
|
@ -75,11 +76,15 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
|
|||
* Asks SCM where a block should be allocated. SCM responds with the
|
||||
* set of datanodes that should be used creating this block.
|
||||
* @param size - size of the block.
|
||||
* @param num - number of blocks.
|
||||
* @param type - replication type of the blocks.
|
||||
* @param factor - replication factor of the blocks.
|
||||
* @param excludeList - exclude list while allocating blocks.
|
||||
* @return allocated block accessing info (key, pipeline).
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public AllocatedBlock allocateBlock(long size,
|
||||
public List<AllocatedBlock> allocateBlock(long size, int num,
|
||||
HddsProtos.ReplicationType type, HddsProtos.ReplicationFactor factor,
|
||||
String owner, ExcludeList excludeList) throws IOException {
|
||||
Preconditions.checkArgument(size > 0, "block size must be greater than 0");
|
||||
|
@ -87,6 +92,7 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
|
|||
AllocateScmBlockRequestProto request =
|
||||
AllocateScmBlockRequestProto.newBuilder()
|
||||
.setSize(size)
|
||||
.setNumBlocks(num)
|
||||
.setType(type)
|
||||
.setFactor(factor)
|
||||
.setOwner(owner)
|
||||
|
@ -104,11 +110,17 @@ public final class ScmBlockLocationProtocolClientSideTranslatorPB
|
|||
throw new IOException(response.hasErrorMessage() ?
|
||||
response.getErrorMessage() : "Allocate block failed.");
|
||||
}
|
||||
|
||||
List<AllocatedBlock> blocks = new ArrayList<>(response.getBlocksCount());
|
||||
for (AllocateBlockResponse resp : response.getBlocksList()) {
|
||||
AllocatedBlock.Builder builder = new AllocatedBlock.Builder()
|
||||
.setContainerBlockID(
|
||||
ContainerBlockID.getFromProtobuf(response.getContainerBlockID()))
|
||||
.setPipeline(Pipeline.getFromProtobuf(response.getPipeline()));
|
||||
return builder.build();
|
||||
ContainerBlockID.getFromProtobuf(resp.getContainerBlockID()))
|
||||
.setPipeline(Pipeline.getFromProtobuf(resp.getPipeline()));
|
||||
blocks.add(builder.build());
|
||||
}
|
||||
|
||||
return blocks;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -60,9 +60,9 @@ public class BlockTokenVerifier implements TokenVerifier {
|
|||
if (conf.isBlockTokenEnabled()) {
|
||||
// TODO: add audit logs.
|
||||
|
||||
if (Strings.isNullOrEmpty(tokenStr) || isTestStub()) {
|
||||
if (Strings.isNullOrEmpty(tokenStr)) {
|
||||
throw new BlockTokenException("Fail to find any token (empty or " +
|
||||
"null.");
|
||||
"null.)");
|
||||
}
|
||||
final Token<OzoneBlockTokenIdentifier> token = new Token();
|
||||
OzoneBlockTokenIdentifier tokenId = new OzoneBlockTokenIdentifier();
|
||||
|
@ -78,29 +78,26 @@ public class BlockTokenVerifier implements TokenVerifier {
|
|||
throw new BlockTokenException("Failed to decode token : " + tokenStr);
|
||||
}
|
||||
|
||||
// TODO: revisit this when caClient is ready, skip signature check now.
|
||||
/**
|
||||
* the final code should like
|
||||
* if (caClient == null) {
|
||||
* throw new SCMSecurityException("Certificate client not available to
|
||||
* validate token");
|
||||
* }
|
||||
*/
|
||||
if (caClient != null) {
|
||||
X509Certificate singerCert = caClient.queryCertificate(
|
||||
"certId=" + tokenId.getOmCertSerialId());
|
||||
if (caClient == null) {
|
||||
throw new SCMSecurityException("Certificate client not available " +
|
||||
"to validate token");
|
||||
}
|
||||
|
||||
X509Certificate singerCert;
|
||||
singerCert = caClient.getCertificate(tokenId.getOmCertSerialId());
|
||||
|
||||
if (singerCert == null) {
|
||||
throw new BlockTokenException("Can't find signer certificate " +
|
||||
"(OmCertSerialId: " + tokenId.getOmCertSerialId() +
|
||||
") of the block token for user: " + tokenId.getUser());
|
||||
}
|
||||
Boolean validToken = caClient.verifySignature(tokenId.getBytes(),
|
||||
boolean validToken = caClient.verifySignature(tokenId.getBytes(),
|
||||
token.getPassword(), singerCert);
|
||||
if (!validToken) {
|
||||
throw new BlockTokenException("Invalid block token for user: " +
|
||||
tokenId.getUser());
|
||||
}
|
||||
}
|
||||
|
||||
// check expiration
|
||||
if (isExpired(tokenId.getExpiryDate())) {
|
||||
UserGroupInformation tokenUser = tokenId.getUser();
|
||||
|
|
|
@ -21,6 +21,7 @@ package org.apache.hadoop.hdds.security.x509;
|
|||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslProvider;
|
||||
import org.bouncycastle.jce.provider.BouncyCastleProvider;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -32,6 +33,7 @@ import java.nio.file.Paths;
|
|||
import java.security.Provider;
|
||||
import java.security.Security;
|
||||
import java.time.Duration;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_ALGORITHM;
|
||||
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_DEFAULT_KEY_LEN;
|
||||
|
@ -459,4 +461,14 @@ public class SecurityConfig {
|
|||
throw new SecurityException("Unknown security provider:" + provider);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns max date for which S3 tokens will be valid.
|
||||
* */
|
||||
public long getS3TokenMaxDate() {
|
||||
return getConfiguration().getTimeDuration(
|
||||
OzoneConfigKeys.OZONE_S3_TOKEN_MAX_LIFETIME_KEY,
|
||||
OzoneConfigKeys.OZONE_S3_TOKEN_MAX_LIFETIME_KEY_DEFAULT,
|
||||
TimeUnit.MICROSECONDS);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -60,17 +60,22 @@ public interface CertificateApprover {
|
|||
* @param validFrom - Begin Date
|
||||
* @param validTill - End Date
|
||||
* @param certificationRequest - Certification Request.
|
||||
* @param scmId - SCM id.
|
||||
* @param clusterId - Cluster id.
|
||||
* @return Signed Certificate.
|
||||
* @throws IOException - On Error
|
||||
* @throws OperatorCreationException - on Error.
|
||||
*/
|
||||
@SuppressWarnings("ParameterNumber")
|
||||
X509CertificateHolder sign(
|
||||
SecurityConfig config,
|
||||
PrivateKey caPrivate,
|
||||
X509CertificateHolder caCertificate,
|
||||
Date validFrom,
|
||||
Date validTill,
|
||||
PKCS10CertificationRequest certificationRequest)
|
||||
PKCS10CertificationRequest certificationRequest,
|
||||
String scmId,
|
||||
String clusterId)
|
||||
throws IOException, OperatorCreationException;
|
||||
|
||||
|
||||
|
|
|
@ -22,7 +22,10 @@ package org.apache.hadoop.hdds.security.x509.certificate.authority;
|
|||
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.authority.PKIProfiles.PKIProfile;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.SecurityUtil;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.bouncycastle.asn1.x500.X500Name;
|
||||
import org.bouncycastle.asn1.x500.style.BCStyle;
|
||||
import org.bouncycastle.asn1.x509.AlgorithmIdentifier;
|
||||
import org.bouncycastle.asn1.x509.SubjectPublicKeyInfo;
|
||||
import org.bouncycastle.cert.X509CertificateHolder;
|
||||
|
@ -67,18 +70,22 @@ public class DefaultApprover extends BaseApprover {
|
|||
* @param validFrom - Begin Da te
|
||||
* @param validTill - End Date
|
||||
* @param certificationRequest - Certification Request.
|
||||
* @param scmId - SCM id.
|
||||
* @param clusterId - Cluster id.
|
||||
* @return Signed Certificate.
|
||||
* @throws IOException - On Error
|
||||
* @throws OperatorCreationException - on Error.
|
||||
*/
|
||||
@SuppressWarnings("ParameterNumber")
|
||||
public X509CertificateHolder sign(
|
||||
SecurityConfig config,
|
||||
PrivateKey caPrivate,
|
||||
X509CertificateHolder caCertificate,
|
||||
Date validFrom,
|
||||
Date validTill,
|
||||
PKCS10CertificationRequest certificationRequest)
|
||||
throws IOException, OperatorCreationException {
|
||||
PKCS10CertificationRequest certificationRequest,
|
||||
String scmId,
|
||||
String clusterId) throws IOException, OperatorCreationException {
|
||||
|
||||
AlgorithmIdentifier sigAlgId = new
|
||||
DefaultSignatureAlgorithmIdentifierFinder().find(
|
||||
|
@ -91,6 +98,29 @@ public class DefaultApprover extends BaseApprover {
|
|||
SubjectPublicKeyInfo keyInfo =
|
||||
certificationRequest.getSubjectPublicKeyInfo();
|
||||
|
||||
// Get scmId and cluster Id from subject name.
|
||||
X500Name x500Name = certificationRequest.getSubject();
|
||||
String csrScmId = x500Name.getRDNs(BCStyle.OU)[0].getFirst().getValue().
|
||||
toASN1Primitive().toString();
|
||||
String csrClusterId = x500Name.getRDNs(BCStyle.O)[0].getFirst().getValue().
|
||||
toASN1Primitive().toString();
|
||||
|
||||
if (!scmId.equals(csrScmId) || !clusterId.equals(csrClusterId)) {
|
||||
if (csrScmId.equalsIgnoreCase("null") &&
|
||||
csrClusterId.equalsIgnoreCase("null")) {
|
||||
// Special case to handle DN certificate generation as DN might not know
|
||||
// scmId and clusterId before registration. In secure mode registration
|
||||
// will succeed only after datanode has a valid certificate.
|
||||
String cn = x500Name.getRDNs(BCStyle.CN)[0].getFirst().getValue()
|
||||
.toASN1Primitive().toString();
|
||||
x500Name = SecurityUtil.getDistinguishedName(cn, scmId, clusterId);
|
||||
} else {
|
||||
// Throw exception if scmId and clusterId doesn't match.
|
||||
throw new SCMSecurityException("ScmId and ClusterId in CSR subject" +
|
||||
" are incorrect.");
|
||||
}
|
||||
}
|
||||
|
||||
RSAKeyParameters rsa =
|
||||
(RSAKeyParameters) PublicKeyFactory.createKey(keyInfo);
|
||||
if (rsa.getModulus().bitLength() < config.getSize()) {
|
||||
|
@ -104,7 +134,7 @@ public class DefaultApprover extends BaseApprover {
|
|||
BigInteger.valueOf(Time.monotonicNowNanos()),
|
||||
validFrom,
|
||||
validTill,
|
||||
certificationRequest.getSubject(), keyInfo);
|
||||
x500Name, keyInfo);
|
||||
|
||||
ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId)
|
||||
.build(asymmetricKP);
|
||||
|
|
|
@ -224,12 +224,16 @@ public class DefaultCAServer implements CertificateServer {
|
|||
break;
|
||||
case KERBEROS_TRUSTED:
|
||||
case TESTING_AUTOMATIC:
|
||||
X509CertificateHolder xcert = approver.sign(config,
|
||||
getCAKeys().getPrivate(),
|
||||
getCACertificate(), java.sql.Date.valueOf(beginDate),
|
||||
java.sql.Date.valueOf(endDate), csr);
|
||||
store.storeValidCertificate(xcert.getSerialNumber(),
|
||||
CertificateCodec.getX509Certificate(xcert));
|
||||
X509CertificateHolder xcert;
|
||||
try {
|
||||
xcert = signAndStoreCertificate(beginDate, endDate, csr);
|
||||
} catch (SCMSecurityException e) {
|
||||
// Certificate with conflicting serial id, retry again may resolve
|
||||
// this issue.
|
||||
LOG.error("Certificate storage failed, retrying one more time.", e);
|
||||
xcert = signAndStoreCertificate(beginDate, endDate, csr);
|
||||
}
|
||||
|
||||
xcertHolder.complete(xcert);
|
||||
break;
|
||||
default:
|
||||
|
@ -242,6 +246,18 @@ public class DefaultCAServer implements CertificateServer {
|
|||
return xcertHolder;
|
||||
}
|
||||
|
||||
private X509CertificateHolder signAndStoreCertificate(LocalDate beginDate,
|
||||
LocalDate endDate, PKCS10CertificationRequest csr) throws IOException,
|
||||
OperatorCreationException, CertificateException {
|
||||
X509CertificateHolder xcert = approver.sign(config,
|
||||
getCAKeys().getPrivate(),
|
||||
getCACertificate(), java.sql.Date.valueOf(beginDate),
|
||||
java.sql.Date.valueOf(endDate), csr, scmID, clusterID);
|
||||
store.storeValidCertificate(xcert.getSerialNumber(),
|
||||
CertificateCodec.getX509Certificate(xcert));
|
||||
return xcert;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Future<X509CertificateHolder> requestCertificate(String csr,
|
||||
CertificateApprover.ApprovalType type) throws IOException {
|
||||
|
|
|
@ -54,8 +54,17 @@ public interface CertificateClient {
|
|||
/**
|
||||
* Returns the certificate of the specified component if it exists on the
|
||||
* local system.
|
||||
* @param certSerialId
|
||||
*
|
||||
* @return certificate or Null if there is no data.
|
||||
*/
|
||||
X509Certificate getCertificate(String certSerialId)
|
||||
throws CertificateException;
|
||||
|
||||
/**
|
||||
* Returns the certificate of the specified component if it exists on the
|
||||
* local system.
|
||||
*
|
||||
* @return certificate or Null if there is no data.
|
||||
*/
|
||||
X509Certificate getCertificate();
|
||||
|
@ -121,13 +130,15 @@ public interface CertificateClient {
|
|||
X509Certificate queryCertificate(String query);
|
||||
|
||||
/**
|
||||
* Stores the Certificate.
|
||||
* Stores the Certificate for this client. Don't use this api to add
|
||||
* trusted certificates of others.
|
||||
*
|
||||
* @param certificate - X509 Certificate
|
||||
|
||||
* @param pemEncodedCert - pem encoded X509 Certificate
|
||||
* @param force - override any existing file
|
||||
* @throws CertificateException - on Error.
|
||||
*
|
||||
*/
|
||||
void storeCertificate(X509Certificate certificate)
|
||||
void storeCertificate(String pemEncodedCert, boolean force)
|
||||
throws CertificateException;
|
||||
|
||||
/**
|
||||
|
|
|
@ -32,8 +32,13 @@ public class DNCertificateClient extends DefaultCertificateClient {
|
|||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(DNCertificateClient.class);
|
||||
public DNCertificateClient(SecurityConfig securityConfig,
|
||||
String certSerialId) {
|
||||
super(securityConfig, LOG, certSerialId);
|
||||
}
|
||||
|
||||
public DNCertificateClient(SecurityConfig securityConfig) {
|
||||
super(securityConfig, LOG);
|
||||
super(securityConfig, LOG, null);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -28,13 +28,26 @@ import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRe
|
|||
import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
|
||||
import org.apache.hadoop.hdds.HddsUtils;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
|
||||
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB;
|
||||
import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolPB;
|
||||
import org.apache.hadoop.hdds.scm.protocolPB.ScmBlockLocationProtocolPB;
|
||||
import org.apache.hadoop.ipc.Client;
|
||||
import org.apache.hadoop.ipc.ProtobufRpcEngine;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.ozone.OzoneSecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.bouncycastle.cert.X509CertificateHolder;
|
||||
import org.slf4j.Logger;
|
||||
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Path;
|
||||
|
@ -47,11 +60,12 @@ import java.security.PublicKey;
|
|||
import java.security.Signature;
|
||||
import java.security.SignatureException;
|
||||
import java.security.cert.CertStore;
|
||||
import java.security.cert.CertificateEncodingException;
|
||||
import java.security.cert.X509Certificate;
|
||||
import java.security.spec.InvalidKeySpecException;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
|
||||
import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
|
||||
import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.GETCERT;
|
||||
|
@ -65,24 +79,75 @@ import static org.apache.hadoop.hdds.security.x509.exceptions.CertificateExcepti
|
|||
*/
|
||||
public abstract class DefaultCertificateClient implements CertificateClient {
|
||||
|
||||
private static final String CERT_FILE_NAME_FORMAT = "%s.crt";
|
||||
private final Logger logger;
|
||||
private final SecurityConfig securityConfig;
|
||||
private final KeyCodec keyCodec;
|
||||
private PrivateKey privateKey;
|
||||
private PublicKey publicKey;
|
||||
private X509Certificate x509Certificate;
|
||||
private Map<String, X509Certificate> certificateMap;
|
||||
private String certSerialId;
|
||||
|
||||
|
||||
DefaultCertificateClient(SecurityConfig securityConfig, Logger log) {
|
||||
DefaultCertificateClient(SecurityConfig securityConfig, Logger log,
|
||||
String certSerialId) {
|
||||
Objects.requireNonNull(securityConfig);
|
||||
this.securityConfig = securityConfig;
|
||||
keyCodec = new KeyCodec(securityConfig);
|
||||
this.logger = log;
|
||||
this.certificateMap = new ConcurrentHashMap<>();
|
||||
this.certSerialId = certSerialId;
|
||||
|
||||
loadAllCertificates();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the private key of the specified component if it exists on the
|
||||
* local system.
|
||||
* Load all certificates from configured location.
|
||||
* */
|
||||
private void loadAllCertificates() {
|
||||
// See if certs directory exists in file system.
|
||||
Path certPath = securityConfig.getCertificateLocation();
|
||||
if (Files.exists(certPath) && Files.isDirectory(certPath)) {
|
||||
getLogger().info("Loading certificate from location:{}.",
|
||||
certPath);
|
||||
File[] certFiles = certPath.toFile().listFiles();
|
||||
|
||||
if (certFiles != null) {
|
||||
CertificateCodec certificateCodec =
|
||||
new CertificateCodec(securityConfig);
|
||||
for (File file : certFiles) {
|
||||
if (file.isFile()) {
|
||||
try {
|
||||
X509CertificateHolder x509CertificateHolder = certificateCodec
|
||||
.readCertificate(certPath, file.getName());
|
||||
X509Certificate cert =
|
||||
CertificateCodec.getX509Certificate(x509CertificateHolder);
|
||||
if (cert != null && cert.getSerialNumber() != null) {
|
||||
if (cert.getSerialNumber().toString().equals(certSerialId)) {
|
||||
x509Certificate = cert;
|
||||
}
|
||||
certificateMap.putIfAbsent(cert.getSerialNumber().toString(),
|
||||
cert);
|
||||
getLogger().info("Added certificate from file:{}.",
|
||||
file.getAbsolutePath());
|
||||
} else {
|
||||
getLogger().error("Error reading certificate from file:{}",
|
||||
file);
|
||||
}
|
||||
} catch (java.security.cert.CertificateException | IOException e) {
|
||||
getLogger().error("Error reading certificate from file:{}.",
|
||||
file.getAbsolutePath(), e);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the private key of the specified if it exists on the local
|
||||
* system.
|
||||
*
|
||||
* @return private key or Null if there is no data.
|
||||
*/
|
||||
|
@ -106,8 +171,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the public key of the specified component if it exists on the
|
||||
* local system.
|
||||
* Returns the public key of the specified if it exists on the local system.
|
||||
*
|
||||
* @return public key or Null if there is no data.
|
||||
*/
|
||||
|
@ -131,8 +195,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Returns the certificate of the specified component if it exists on the
|
||||
* local system.
|
||||
* Returns the default certificate of given client if it exists.
|
||||
*
|
||||
* @return certificate or Null if there is no data.
|
||||
*/
|
||||
|
@ -142,23 +205,62 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
return x509Certificate;
|
||||
}
|
||||
|
||||
Path certPath = securityConfig.getCertificateLocation();
|
||||
if (OzoneSecurityUtil.checkIfFileExist(certPath,
|
||||
securityConfig.getCertificateFileName())) {
|
||||
CertificateCodec certificateCodec =
|
||||
new CertificateCodec(securityConfig);
|
||||
try {
|
||||
X509CertificateHolder x509CertificateHolder =
|
||||
certificateCodec.readCertificate();
|
||||
x509Certificate =
|
||||
CertificateCodec.getX509Certificate(x509CertificateHolder);
|
||||
} catch (java.security.cert.CertificateException | IOException e) {
|
||||
getLogger().error("Error reading certificate.", e);
|
||||
if (certSerialId == null) {
|
||||
getLogger().error("Default certificate serial id is not set. Can't " +
|
||||
"locate the default certificate for this client.");
|
||||
return null;
|
||||
}
|
||||
// Refresh the cache from file system.
|
||||
loadAllCertificates();
|
||||
if (certificateMap.containsKey(certSerialId)) {
|
||||
x509Certificate = certificateMap.get(certSerialId);
|
||||
}
|
||||
return x509Certificate;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the certificate with the specified certificate serial id if it
|
||||
* exists else try to get it from SCM.
|
||||
* @param certId
|
||||
*
|
||||
* @return certificate or Null if there is no data.
|
||||
*/
|
||||
@Override
|
||||
public X509Certificate getCertificate(String certId)
|
||||
throws CertificateException {
|
||||
// Check if it is in cache.
|
||||
if (certificateMap.containsKey(certId)) {
|
||||
return certificateMap.get(certId);
|
||||
}
|
||||
// Try to get it from SCM.
|
||||
return this.getCertificateFromScm(certId);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get certificate from SCM and store it in local file system.
|
||||
* @param certId
|
||||
* @return certificate
|
||||
*/
|
||||
private X509Certificate getCertificateFromScm(String certId)
|
||||
throws CertificateException {
|
||||
|
||||
getLogger().info("Getting certificate with certSerialId:{}.",
|
||||
certId);
|
||||
try {
|
||||
SCMSecurityProtocol scmSecurityProtocolClient = getScmSecurityClient(
|
||||
(OzoneConfiguration) securityConfig.getConfiguration());
|
||||
String pemEncodedCert =
|
||||
scmSecurityProtocolClient.getCertificate(certId);
|
||||
this.storeCertificate(pemEncodedCert, true);
|
||||
return CertificateCodec.getX509Certificate(pemEncodedCert);
|
||||
} catch (Exception e) {
|
||||
getLogger().error("Error while getting Certificate with " +
|
||||
"certSerialId:{} from scm.", certId, e);
|
||||
throw new CertificateException("Error while getting certificate for " +
|
||||
"certSerialId:" + certId, e, CERTIFICATE_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Verifies if this certificate is part of a trusted chain.
|
||||
*
|
||||
|
@ -171,8 +273,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates digital signature over the data stream using the components
|
||||
* private key.
|
||||
* Creates digital signature over the data stream using the s private key.
|
||||
*
|
||||
* @param stream - Data stream to sign.
|
||||
* @throws CertificateException - on Error.
|
||||
|
@ -200,8 +301,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Creates digital signature over the data stream using the components
|
||||
* private key.
|
||||
* Creates digital signature over the data stream using the s private key.
|
||||
*
|
||||
* @param data - Data to sign.
|
||||
* @throws CertificateException - on Error.
|
||||
|
@ -349,27 +449,37 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
}
|
||||
|
||||
/**
|
||||
* Stores the Certificate for this client. Don't use this api to add
|
||||
* trusted certificates of other components.
|
||||
* Stores the Certificate for this client. Don't use this api to add trusted
|
||||
* certificates of others.
|
||||
*
|
||||
* @param certificate - X509 Certificate
|
||||
* @param pemEncodedCert - pem encoded X509 Certificate
|
||||
* @param force - override any existing file
|
||||
* @throws CertificateException - on Error.
|
||||
*
|
||||
*/
|
||||
@Override
|
||||
public void storeCertificate(X509Certificate certificate)
|
||||
public void storeCertificate(String pemEncodedCert, boolean force)
|
||||
throws CertificateException {
|
||||
CertificateCodec certificateCodec = new CertificateCodec(securityConfig);
|
||||
try {
|
||||
certificateCodec.writeCertificate(
|
||||
new X509CertificateHolder(certificate.getEncoded()));
|
||||
} catch (IOException | CertificateEncodingException e) {
|
||||
Path basePath = securityConfig.getCertificateLocation();
|
||||
|
||||
X509Certificate cert =
|
||||
CertificateCodec.getX509Certificate(pemEncodedCert);
|
||||
String certName = String.format(CERT_FILE_NAME_FORMAT,
|
||||
cert.getSerialNumber().toString());
|
||||
|
||||
certificateCodec.writeCertificate(basePath, certName,
|
||||
pemEncodedCert, force);
|
||||
certificateMap.putIfAbsent(cert.getSerialNumber().toString(), cert);
|
||||
} catch (IOException | java.security.cert.CertificateException e) {
|
||||
throw new CertificateException("Error while storing certificate.", e,
|
||||
CERTIFICATE_ERROR);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Stores the trusted chain of certificates for a specific component.
|
||||
* Stores the trusted chain of certificates for a specific .
|
||||
*
|
||||
* @param ks - Key Store.
|
||||
* @throws CertificateException - on Error.
|
||||
|
@ -382,7 +492,7 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
|
||||
|
||||
/**
|
||||
* Stores the trusted chain of certificates for a specific component.
|
||||
* Stores the trusted chain of certificates for a specific .
|
||||
*
|
||||
* @param certificates - List of Certificates.
|
||||
* @throws CertificateException - on Error.
|
||||
|
@ -640,4 +750,26 @@ public abstract class DefaultCertificateClient implements CertificateClient {
|
|||
public Logger getLogger() {
|
||||
return logger;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a scm security client, used to get SCM signed certificate.
|
||||
*
|
||||
* @return {@link SCMSecurityProtocol}
|
||||
*/
|
||||
private static SCMSecurityProtocol getScmSecurityClient(
|
||||
OzoneConfiguration conf) throws IOException {
|
||||
RPC.setProtocolEngine(conf, SCMSecurityProtocolPB.class,
|
||||
ProtobufRpcEngine.class);
|
||||
long scmVersion =
|
||||
RPC.getProtocolVersion(ScmBlockLocationProtocolPB.class);
|
||||
InetSocketAddress scmSecurityProtoAdd =
|
||||
HddsUtils.getScmAddressForSecurityProtocol(conf);
|
||||
SCMSecurityProtocolClientSideTranslatorPB scmSecurityClient =
|
||||
new SCMSecurityProtocolClientSideTranslatorPB(
|
||||
RPC.getProxy(SCMSecurityProtocolPB.class, scmVersion,
|
||||
scmSecurityProtoAdd, UserGroupInformation.getCurrentUser(),
|
||||
conf, NetUtils.getDefaultSocketFactory(conf),
|
||||
Client.getRpcTimeout(conf)));
|
||||
return scmSecurityClient;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,8 +39,13 @@ public class OMCertificateClient extends DefaultCertificateClient {
|
|||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(OMCertificateClient.class);
|
||||
|
||||
public OMCertificateClient(SecurityConfig securityConfig,
|
||||
String certSerialId) {
|
||||
super(securityConfig, LOG, certSerialId);
|
||||
}
|
||||
|
||||
public OMCertificateClient(SecurityConfig securityConfig) {
|
||||
super(securityConfig, LOG);
|
||||
super(securityConfig, LOG, null);
|
||||
}
|
||||
|
||||
protected InitResponse handleCase(InitCase init) throws
|
||||
|
|
|
@ -269,10 +269,6 @@ public final class CertificateSignRequest {
|
|||
Preconditions.checkNotNull(key, "KeyPair cannot be null");
|
||||
Preconditions.checkArgument(Strings.isNotBlank(subject), "Subject " +
|
||||
"cannot be blank");
|
||||
Preconditions.checkArgument(Strings.isNotBlank(clusterID), "Cluster ID " +
|
||||
"cannot be blank");
|
||||
Preconditions.checkArgument(Strings.isNotBlank(scmID), "SCM ID cannot " +
|
||||
"be blank");
|
||||
|
||||
try {
|
||||
CertificateSignRequest csr = new CertificateSignRequest(subject, scmID,
|
||||
|
|
|
@ -83,6 +83,7 @@ public class CertificateException extends SCMSecurityException {
|
|||
CERTIFICATE_ERROR,
|
||||
BOOTSTRAP_ERROR,
|
||||
CSR_ERROR,
|
||||
CRYPTO_SIGNATURE_VERIFICATION_ERROR
|
||||
CRYPTO_SIGNATURE_VERIFICATION_ERROR,
|
||||
CERTIFICATE_NOT_FOUND_ERROR
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,20 +25,27 @@ import io.jaegertracing.internal.exceptions.MalformedTracerStateStringException;
|
|||
import io.jaegertracing.internal.exceptions.TraceIdOutOfBoundException;
|
||||
import io.jaegertracing.spi.Codec;
|
||||
import io.opentracing.propagation.Format;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* A jaeger codec to save the current tracing context t a string.
|
||||
* A jaeger codec to save the current tracing context as a string.
|
||||
*/
|
||||
public class StringCodec implements Codec<StringBuilder> {
|
||||
|
||||
public static final Logger LOG = LoggerFactory.getLogger(StringCodec.class);
|
||||
public static final StringFormat FORMAT = new StringFormat();
|
||||
|
||||
@Override
|
||||
public JaegerSpanContext extract(StringBuilder s) {
|
||||
if (s == null) {
|
||||
throw new EmptyTracerStateStringException();
|
||||
}
|
||||
String value = s.toString();
|
||||
if (value != null && !value.equals("")) {
|
||||
String[] parts = value.split(":");
|
||||
if (parts.length != 4) {
|
||||
LOG.trace("MalformedTracerStateString: {}", value);
|
||||
throw new MalformedTracerStateStringException(value);
|
||||
} else {
|
||||
String traceId = parts[0];
|
||||
|
|
|
@ -133,6 +133,11 @@ public final class OzoneConfigKeys {
|
|||
public static final String OZONE_CLIENT_WATCH_REQUEST_TIMEOUT_DEFAULT =
|
||||
"30s";
|
||||
|
||||
public static final String OZONE_CLIENT_MAX_RETRIES =
|
||||
"ozone.client.max.retries";
|
||||
public static final int OZONE_CLIENT_MAX_RETRIES_DEFAULT = 5;
|
||||
|
||||
|
||||
// This defines the overall connection limit for the connection pool used in
|
||||
// RestClient.
|
||||
public static final String OZONE_REST_CLIENT_HTTP_CONNECTION_MAX =
|
||||
|
@ -198,10 +203,10 @@ public final class OzoneConfigKeys {
|
|||
public static final String OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT
|
||||
= "300s"; // 300s for default
|
||||
|
||||
public static final String OZONE_KEY_PREALLOCATION_MAXSIZE =
|
||||
"ozone.key.preallocation.maxsize";
|
||||
public static final long OZONE_KEY_PREALLOCATION_MAXSIZE_DEFAULT
|
||||
= 128 * OzoneConsts.MB;
|
||||
public static final String OZONE_KEY_PREALLOCATION_BLOCKS_MAX =
|
||||
"ozone.key.preallocation.max.blocks";
|
||||
public static final int OZONE_KEY_PREALLOCATION_BLOCKS_MAX_DEFAULT
|
||||
= 64;
|
||||
|
||||
public static final String OZONE_BLOCK_DELETING_LIMIT_PER_CONTAINER =
|
||||
"ozone.block.deleting.limit.per.task";
|
||||
|
@ -373,12 +378,31 @@ public final class OzoneConfigKeys {
|
|||
"ozone.acl.enabled";
|
||||
public static final boolean OZONE_ACL_ENABLED_DEFAULT =
|
||||
false;
|
||||
|
||||
public static final String OZONE_S3_TOKEN_MAX_LIFETIME_KEY =
|
||||
"ozone.s3.token.max.lifetime";
|
||||
public static final String OZONE_S3_TOKEN_MAX_LIFETIME_KEY_DEFAULT = "3m";
|
||||
//For technical reasons this is unused and hardcoded to the
|
||||
// OzoneFileSystem.initialize.
|
||||
public static final String OZONE_FS_ISOLATED_CLASSLOADER =
|
||||
"ozone.fs.isolated-classloader";
|
||||
|
||||
// Ozone Client Retry and Failover configurations
|
||||
public static final String OZONE_CLIENT_RETRY_MAX_ATTEMPTS_KEY =
|
||||
"ozone.client.retry.max.attempts";
|
||||
public static final int OZONE_CLIENT_RETRY_MAX_ATTEMPTS_DEFAULT =
|
||||
10;
|
||||
public static final String OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_KEY =
|
||||
"ozone.client.failover.max.attempts";
|
||||
public static final int OZONE_CLIENT_FAILOVER_MAX_ATTEMPTS_DEFAULT =
|
||||
15;
|
||||
public static final String OZONE_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_KEY =
|
||||
"ozone.client.failover.sleep.base.millis";
|
||||
public static final int OZONE_CLIENT_FAILOVER_SLEEP_BASE_MILLIS_DEFAULT =
|
||||
500;
|
||||
public static final String OZONE_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_KEY =
|
||||
"ozone.client.failover.sleep.max.millis";
|
||||
public static final int OZONE_CLIENT_FAILOVER_SLEEP_MAX_MILLIS_DEFAULT =
|
||||
15000;
|
||||
|
||||
public static final String OZONE_FREON_HTTP_ENABLED_KEY =
|
||||
"ozone.freon.http.enabled";
|
||||
|
|
|
@ -276,4 +276,7 @@ public final class OzoneConsts {
|
|||
|
||||
// Default OMServiceID for OM Ratis servers to use as RaftGroupId
|
||||
public static final String OM_SERVICE_ID_DEFAULT = "omServiceIdDefault";
|
||||
|
||||
// Dummy OMNodeID for OM Clients to use for a non-HA OM setup
|
||||
public static final String OM_NODE_ID_DUMMY = "omNodeIdDummy";
|
||||
}
|
||||
|
|
|
@ -245,5 +245,17 @@ public abstract class Storage {
|
|||
storageInfo.writeTo(getVersionFile());
|
||||
}
|
||||
|
||||
/**
|
||||
* Persists current StorageInfo to file system..
|
||||
* @throws IOException
|
||||
*/
|
||||
public void persistCurrentState() throws IOException {
|
||||
if (!getCurrentDir().exists()) {
|
||||
throw new IOException("Metadata dir doesn't exist, dir: " +
|
||||
getCurrentDir());
|
||||
}
|
||||
storageInfo.writeTo(getVersionFile());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,8 @@ import com.google.protobuf.ServiceException;
|
|||
import io.opentracing.Scope;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.hdds.protocol.proto.ScmBlockLocationProtocolProtos
|
||||
.AllocateBlockResponse;
|
||||
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.ExcludeList;
|
||||
|
@ -76,22 +78,30 @@ public final class ScmBlockLocationProtocolServerSideTranslatorPB
|
|||
try (Scope scope = TracingUtil
|
||||
.importAndCreateScope("ScmBlockLocationProtocol.allocateBlock",
|
||||
request.getTraceID())) {
|
||||
AllocatedBlock allocatedBlock =
|
||||
impl.allocateBlock(request.getSize(), request.getType(),
|
||||
List<AllocatedBlock> allocatedBlocks =
|
||||
impl.allocateBlock(request.getSize(),
|
||||
request.getNumBlocks(), request.getType(),
|
||||
request.getFactor(), request.getOwner(),
|
||||
ExcludeList.getFromProtoBuf(request.getExcludeList()));
|
||||
if (allocatedBlock != null) {
|
||||
return
|
||||
AllocateScmBlockResponseProto.newBuilder()
|
||||
.setContainerBlockID(allocatedBlock.getBlockID().getProtobuf())
|
||||
.setPipeline(allocatedBlock.getPipeline().getProtobufMessage())
|
||||
.setErrorCode(AllocateScmBlockResponseProto.Error.success)
|
||||
.build();
|
||||
} else {
|
||||
return AllocateScmBlockResponseProto.newBuilder()
|
||||
|
||||
AllocateScmBlockResponseProto.Builder builder =
|
||||
AllocateScmBlockResponseProto.newBuilder();
|
||||
|
||||
if (allocatedBlocks.size() < request.getNumBlocks()) {
|
||||
return builder
|
||||
.setErrorCode(AllocateScmBlockResponseProto.Error.unknownFailure)
|
||||
.build();
|
||||
}
|
||||
|
||||
for (AllocatedBlock block : allocatedBlocks) {
|
||||
builder.addBlocks(AllocateBlockResponse.newBuilder()
|
||||
.setContainerBlockID(block.getBlockID().getProtobuf())
|
||||
.setPipeline(block.getPipeline().getProtobufMessage()));
|
||||
}
|
||||
|
||||
return builder
|
||||
.setErrorCode(AllocateScmBlockResponseProto.Error.success)
|
||||
.build();
|
||||
} catch (IOException e) {
|
||||
throw new ServiceException(e);
|
||||
}
|
||||
|
|
|
@ -87,15 +87,18 @@ public class Scheduler {
|
|||
* yet executed are also cancelled. For the executing tasks the scheduler
|
||||
* waits 60 seconds for completion.
|
||||
*/
|
||||
public void close() {
|
||||
public synchronized void close() {
|
||||
isClosed = true;
|
||||
if (scheduler != null) {
|
||||
scheduler.shutdownNow();
|
||||
try {
|
||||
scheduler.awaitTermination(60, TimeUnit.SECONDS);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info(threadName + " interrupted while waiting for task completion {}",
|
||||
LOG.info(
|
||||
threadName + " interrupted while waiting for task completion {}",
|
||||
e);
|
||||
}
|
||||
}
|
||||
scheduler = null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ public class CodecRegistry {
|
|||
public CodecRegistry() {
|
||||
valueCodecs = new HashMap<>();
|
||||
valueCodecs.put(String.class, new StringCodec());
|
||||
valueCodecs.put(Long.class, new LongCodec());
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*
|
||||
*/
|
||||
package org.apache.hadoop.utils.db;
|
||||
|
||||
import com.google.common.primitives.Longs;
|
||||
|
||||
|
||||
/**
|
||||
* Codec to convert Long to/from byte array.
|
||||
*/
|
||||
public class LongCodec implements Codec<Long> {
|
||||
|
||||
@Override
|
||||
public byte[] toPersistedFormat(Long object) {
|
||||
if (object != null) {
|
||||
return Longs.toByteArray(object);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Long fromPersistedFormat(byte[] rawData) {
|
||||
if (rawData != null) {
|
||||
return Longs.fromByteArray(rawData);
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
|
@ -36,6 +36,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
|
||||
import org.apache.ratis.client.RaftClient;
|
||||
import org.apache.ratis.client.RaftClientConfigKeys;
|
||||
import org.apache.ratis.conf.RaftProperties;
|
||||
import org.apache.ratis.grpc.GrpcConfigKeys;
|
||||
import org.apache.ratis.grpc.GrpcFactory;
|
||||
|
@ -134,33 +135,51 @@ public interface RatisHelper {
|
|||
|
||||
static RaftClient newRaftClient(RpcType rpcType, Pipeline pipeline,
|
||||
RetryPolicy retryPolicy, int maxOutStandingRequest,
|
||||
GrpcTlsConfig tlsConfig) throws IOException {
|
||||
GrpcTlsConfig tlsConfig, TimeDuration timeout) throws IOException {
|
||||
return newRaftClient(rpcType, toRaftPeerId(pipeline.getFirstNode()),
|
||||
newRaftGroup(RaftGroupId.valueOf(pipeline.getId().getId()),
|
||||
pipeline.getNodes()), retryPolicy, maxOutStandingRequest, tlsConfig);
|
||||
pipeline.getNodes()), retryPolicy, maxOutStandingRequest, tlsConfig,
|
||||
timeout);
|
||||
}
|
||||
|
||||
static TimeDuration getClientRequestTimeout(Configuration conf) {
|
||||
// Set the client requestTimeout
|
||||
final TimeUnit timeUnit =
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
|
||||
.getUnit();
|
||||
final long duration = conf.getTimeDuration(
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
|
||||
.getDuration(), timeUnit);
|
||||
final TimeDuration clientRequestTimeout =
|
||||
TimeDuration.valueOf(duration, timeUnit);
|
||||
return clientRequestTimeout;
|
||||
}
|
||||
|
||||
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
|
||||
RetryPolicy retryPolicy, int maxOutstandingRequests,
|
||||
GrpcTlsConfig tlsConfig) {
|
||||
GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
|
||||
return newRaftClient(rpcType, leader.getId(),
|
||||
newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
|
||||
maxOutstandingRequests, tlsConfig);
|
||||
maxOutstandingRequests, tlsConfig, clientRequestTimeout);
|
||||
}
|
||||
|
||||
static RaftClient newRaftClient(RpcType rpcType, RaftPeer leader,
|
||||
RetryPolicy retryPolicy, int maxOutstandingRequests) {
|
||||
RetryPolicy retryPolicy, int maxOutstandingRequests,
|
||||
TimeDuration clientRequestTimeout) {
|
||||
return newRaftClient(rpcType, leader.getId(),
|
||||
newRaftGroup(new ArrayList<>(Arrays.asList(leader))), retryPolicy,
|
||||
maxOutstandingRequests, null);
|
||||
maxOutstandingRequests, null, clientRequestTimeout);
|
||||
}
|
||||
|
||||
static RaftClient newRaftClient(RpcType rpcType, RaftPeerId leader,
|
||||
RaftGroup group, RetryPolicy retryPolicy, int maxOutStandingRequest,
|
||||
GrpcTlsConfig tlsConfig) {
|
||||
GrpcTlsConfig tlsConfig, TimeDuration clientRequestTimeout) {
|
||||
LOG.trace("newRaftClient: {}, leader={}, group={}", rpcType, leader, group);
|
||||
final RaftProperties properties = new RaftProperties();
|
||||
RaftConfigKeys.Rpc.setType(properties, rpcType);
|
||||
RaftClientConfigKeys.Rpc
|
||||
.setRequestTimeout(properties, clientRequestTimeout);
|
||||
|
||||
GrpcConfigKeys.setMessageSizeMax(properties,
|
||||
SizeInBytes.valueOf(OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE));
|
||||
|
|
|
@ -142,6 +142,8 @@ enum Result {
|
|||
UNKNOWN_BCSID = 37;
|
||||
BCSID_MISMATCH = 38;
|
||||
CONTAINER_NOT_OPEN = 39;
|
||||
CONTAINER_MISSING = 40;
|
||||
BLOCK_TOKEN_VERIFICATION_FAILED = 41;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -245,6 +247,10 @@ message ContainerDataProto {
|
|||
optional ContainerType containerType = 10 [default = KeyValueContainer];
|
||||
}
|
||||
|
||||
message ContainerIdSetProto {
|
||||
repeated int64 containerId = 1;
|
||||
}
|
||||
|
||||
enum ContainerType {
|
||||
KeyValueContainer = 1;
|
||||
}
|
||||
|
|
|
@ -38,12 +38,12 @@ import "hdds.proto";
|
|||
*/
|
||||
message AllocateScmBlockRequestProto {
|
||||
required uint64 size = 1;
|
||||
required ReplicationType type = 2;
|
||||
required hadoop.hdds.ReplicationFactor factor = 3;
|
||||
required string owner = 4;
|
||||
optional string traceID = 5;
|
||||
optional ExcludeListProto excludeList = 6;
|
||||
|
||||
required uint32 numBlocks = 2;
|
||||
required ReplicationType type = 3;
|
||||
required hadoop.hdds.ReplicationFactor factor = 4;
|
||||
required string owner = 5;
|
||||
optional string traceID = 6;
|
||||
optional ExcludeListProto excludeList = 7;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -96,6 +96,11 @@ message DeleteScmBlockResult {
|
|||
required BlockID blockID = 2;
|
||||
}
|
||||
|
||||
message AllocateBlockResponse {
|
||||
optional ContainerBlockID containerBlockID = 1;
|
||||
optional hadoop.hdds.Pipeline pipeline = 2;
|
||||
}
|
||||
|
||||
/**
|
||||
* Reply from SCM indicating that the container.
|
||||
*/
|
||||
|
@ -107,9 +112,8 @@ message AllocateScmBlockResponseProto {
|
|||
unknownFailure = 4;
|
||||
}
|
||||
required Error errorCode = 1;
|
||||
optional ContainerBlockID containerBlockID = 2;
|
||||
optional hadoop.hdds.Pipeline pipeline = 3;
|
||||
optional string errorMessage = 4;
|
||||
optional string errorMessage = 2;
|
||||
repeated AllocateBlockResponse blocks = 3;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -33,6 +33,7 @@ message DatanodeDetailsProto {
|
|||
required string ipAddress = 2; // IP address
|
||||
required string hostName = 3; // hostname
|
||||
repeated Port ports = 4;
|
||||
optional string certSerialId = 5; // Certificate serial id.
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -231,17 +231,19 @@
|
|||
<name>dfs.ratis.client.request.timeout.duration</name>
|
||||
<value>3s</value>
|
||||
<tag>OZONE, RATIS, MANAGEMENT</tag>
|
||||
<description>The timeout duration for ratis client request.</description>
|
||||
<description>The timeout duration for ratis client request.It should be
|
||||
set greater than leader election timeout in Ratis.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.ratis.client.request.max.retries</name>
|
||||
<value>180</value>
|
||||
<value>20</value>
|
||||
<tag>OZONE, RATIS, MANAGEMENT</tag>
|
||||
<description>Number of retries for ratis client request.</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>dfs.ratis.client.request.retry.interval</name>
|
||||
<value>100ms</value>
|
||||
<value>500ms</value>
|
||||
<tag>OZONE, RATIS, MANAGEMENT</tag>
|
||||
<description>Interval between successive retries for a ratis client request.
|
||||
</description>
|
||||
|
@ -417,6 +419,14 @@
|
|||
a particular request getting replayed to all servers.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.max.retries</name>
|
||||
<value>5</value>
|
||||
<tag>OZONE, CLIENT</tag>
|
||||
<description>Maximum number of retries by Ozone Client on encountering
|
||||
exception while writing a key.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.protocol</name>
|
||||
<value>org.apache.hadoop.ozone.client.rpc.RpcClient</value>
|
||||
|
@ -1089,13 +1099,14 @@
|
|||
</property>
|
||||
|
||||
<property>
|
||||
<name>ozone.key.preallocation.maxsize</name>
|
||||
<value>134217728</value>
|
||||
<name>ozone.key.preallocation.max.blocks</name>
|
||||
<value>64</value>
|
||||
<tag>OZONE, OM, PERFORMANCE</tag>
|
||||
<description>
|
||||
When a new key write request is sent to OM, if a size is requested, at most
|
||||
128MB of size is allocated at request time. If client needs more space for the
|
||||
write, separate block allocation requests will be made.
|
||||
While allocating blocks from OM, this configuration limits the maximum
|
||||
number of blocks being allocated. This configuration ensures that the
|
||||
allocated block response do not exceed rpc payload limit. If client needs
|
||||
more space for the write, separate block allocation requests will be made.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
@ -1697,7 +1708,7 @@
|
|||
<value>HTTP/_HOST@EXAMPLE.COM</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.scm.http.kerberos.keytab</name>
|
||||
<name>hdds.scm.http.kerberos.keytab.file</name>
|
||||
<value>/etc/security/keytabs/HTTP.keytab</value>
|
||||
</property>
|
||||
|
||||
|
@ -1907,7 +1918,7 @@
|
|||
an isolated classloader.
|
||||
|
||||
Default depends from the used jar. true for ozone-filesystem-lib-legacy
|
||||
jar and false for the ozone-filesystem-lib.jar
|
||||
jar and false for the ozone-filesystem-lib-current.jar
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
|
@ -2029,4 +2040,109 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>hdds.datanode.http.kerberos.principal</name>
|
||||
<value>HTTP/_HOST@EXAMPLE.COM</value>
|
||||
<tag>HDDS, SECURITY, MANAGEMENT</tag>
|
||||
<description>
|
||||
The kerberos principal for the datanode http server.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.http.kerberos.keytab</name>
|
||||
<value>/etc/security/keytabs/HTTP.keytab</value>
|
||||
<tag>HDDS, SECURITY, MANAGEMENT</tag>
|
||||
<description>
|
||||
The kerberos keytab file for datanode http server
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.http-address</name>
|
||||
<value>0.0.0.0:9882</value>
|
||||
<tag>HDDS, MANAGEMENT</tag>
|
||||
<description>
|
||||
The address and the base port where the Datanode web ui will listen on.
|
||||
If the port is 0 then the server will start on a free port.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.http-bind-host</name>
|
||||
<value>0.0.0.0</value>
|
||||
<tag>HDDS, MANAGEMENT</tag>
|
||||
<description>
|
||||
The actual address the Datanode web server will bind to. If this
|
||||
optional address is set, it overrides only the hostname portion of
|
||||
hdds.datanode.http-address.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.http.enabled</name>
|
||||
<value>true</value>
|
||||
<tag>HDDS, MANAGEMENT</tag>
|
||||
<description>
|
||||
Property to enable or disable Datanode web ui.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.https-address</name>
|
||||
<value>0.0.0.0:9883</value>
|
||||
<tag>HDDS, MANAGEMENT, SECURITY</tag>
|
||||
<description>
|
||||
The address and the base port where the Datanode web UI will listen
|
||||
on using HTTPS.
|
||||
|
||||
If the port is 0 then the server will start on a free port.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>hdds.datanode.https-bind-host</name>
|
||||
<value>0.0.0.0</value>
|
||||
<tag>HDDS, MANAGEMENT, SECURITY</tag>
|
||||
<description>
|
||||
The actual address the Datanode web server will bind to using HTTPS.
|
||||
If this optional address is set, it overrides only the hostname portion of
|
||||
hdds.datanode.http-address.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.retry.max.attempts</name>
|
||||
<value>10</value>
|
||||
<description>
|
||||
Max retry attempts for Ozone RpcClient talking to OzoneManagers.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.failover.max.attempts</name>
|
||||
<value>15</value>
|
||||
<description>
|
||||
Expert only. The number of client failover attempts that should be
|
||||
made before the failover is considered failed.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.failover.sleep.base.millis</name>
|
||||
<value>500</value>
|
||||
<description>
|
||||
Expert only. The time to wait, in milliseconds, between failover
|
||||
attempts increases exponentially as a function of the number of
|
||||
attempts made so far, with a random factor of +/- 50%. This option
|
||||
specifies the base value used in the failover calculation. The
|
||||
first failover will retry immediately. The 2nd failover attempt
|
||||
will delay at least ozone.client.failover.sleep.base.millis
|
||||
milliseconds. And so on.
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>ozone.client.failover.sleep.max.millis</name>
|
||||
<value>15000</value>
|
||||
<description>
|
||||
Expert only. The time to wait, in milliseconds, between failover
|
||||
attempts increases exponentially as a function of the number of
|
||||
attempts made so far, with a random factor of +/- 50%. This option
|
||||
specifies the maximum value to wait between failovers.
|
||||
Specifically, the time between two failover attempts will not
|
||||
exceed +/- 50% of ozone.client.failover.sleep.max.millis
|
||||
milliseconds.
|
||||
</description>
|
||||
</property>
|
||||
</configuration>
|
||||
|
|
|
@ -0,0 +1,143 @@
|
|||
/**
|
||||
* 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.conf;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.junit.Assert;
|
||||
import org.junit.rules.TemporaryFolder;
|
||||
|
||||
import java.io.BufferedWriter;
|
||||
import java.io.File;
|
||||
import java.io.FileWriter;
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* Test class for OzoneConfiguration.
|
||||
*/
|
||||
public class TestOzoneConfiguration {
|
||||
|
||||
private Configuration conf;
|
||||
|
||||
@Rule
|
||||
public TemporaryFolder tempConfigs = new TemporaryFolder();
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
conf = new OzoneConfiguration();
|
||||
}
|
||||
|
||||
private void startConfig(BufferedWriter out) throws IOException {
|
||||
out.write("<?xml version=\"1.0\"?>\n");
|
||||
out.write("<configuration>\n");
|
||||
}
|
||||
|
||||
private void endConfig(BufferedWriter out) throws IOException {
|
||||
out.write("</configuration>\n");
|
||||
out.flush();
|
||||
out.close();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetAllPropertiesByTags() throws Exception {
|
||||
File coreDefault = tempConfigs.newFile("core-default-test.xml");
|
||||
File coreSite = tempConfigs.newFile("core-site-test.xml");
|
||||
try (BufferedWriter out = new BufferedWriter(new FileWriter(coreDefault))) {
|
||||
startConfig(out);
|
||||
appendProperty(out, "hadoop.tags.system", "YARN,HDFS,NAMENODE");
|
||||
appendProperty(out, "hadoop.tags.custom", "MYCUSTOMTAG");
|
||||
appendPropertyByTag(out, "dfs.cblock.trace.io", "false", "YARN");
|
||||
appendPropertyByTag(out, "dfs.replication", "1", "HDFS");
|
||||
appendPropertyByTag(out, "dfs.namenode.logging.level", "INFO",
|
||||
"NAMENODE");
|
||||
appendPropertyByTag(out, "dfs.random.key", "XYZ", "MYCUSTOMTAG");
|
||||
endConfig(out);
|
||||
|
||||
Path fileResource = new Path(coreDefault.getAbsolutePath());
|
||||
conf.addResource(fileResource);
|
||||
Assert.assertEquals(conf.getAllPropertiesByTag("MYCUSTOMTAG")
|
||||
.getProperty("dfs.random.key"), "XYZ");
|
||||
}
|
||||
|
||||
try (BufferedWriter out = new BufferedWriter(new FileWriter(coreSite))) {
|
||||
startConfig(out);
|
||||
appendProperty(out, "dfs.random.key", "ABC");
|
||||
appendProperty(out, "dfs.replication", "3");
|
||||
appendProperty(out, "dfs.cblock.trace.io", "true");
|
||||
endConfig(out);
|
||||
|
||||
Path fileResource = new Path(coreSite.getAbsolutePath());
|
||||
conf.addResource(fileResource);
|
||||
}
|
||||
|
||||
// Test if values are getting overridden even without tags being present
|
||||
Assert.assertEquals("3", conf.getAllPropertiesByTag("HDFS")
|
||||
.getProperty("dfs.replication"));
|
||||
Assert.assertEquals("ABC", conf.getAllPropertiesByTag("MYCUSTOMTAG")
|
||||
.getProperty("dfs.random.key"));
|
||||
Assert.assertEquals("true", conf.getAllPropertiesByTag("YARN")
|
||||
.getProperty("dfs.cblock.trace.io"));
|
||||
}
|
||||
|
||||
private void appendProperty(BufferedWriter out, String name, String val)
|
||||
throws IOException {
|
||||
this.appendProperty(out, name, val, false);
|
||||
}
|
||||
|
||||
private void appendProperty(BufferedWriter out, String name, String val,
|
||||
boolean isFinal) throws IOException {
|
||||
out.write("<property>");
|
||||
out.write("<name>");
|
||||
out.write(name);
|
||||
out.write("</name>");
|
||||
out.write("<value>");
|
||||
out.write(val);
|
||||
out.write("</value>");
|
||||
if (isFinal) {
|
||||
out.write("<final>true</final>");
|
||||
}
|
||||
out.write("</property>\n");
|
||||
}
|
||||
|
||||
private void appendPropertyByTag(BufferedWriter out, String name, String val,
|
||||
String tags) throws IOException {
|
||||
this.appendPropertyByTag(out, name, val, false, tags);
|
||||
}
|
||||
|
||||
private void appendPropertyByTag(BufferedWriter out, String name, String val,
|
||||
boolean isFinal,
|
||||
String tag) throws IOException {
|
||||
out.write("<property>");
|
||||
out.write("<name>");
|
||||
out.write(name);
|
||||
out.write("</name>");
|
||||
out.write("<value>");
|
||||
out.write(val);
|
||||
out.write("</value>");
|
||||
if (isFinal) {
|
||||
out.write("<final>true</final>");
|
||||
}
|
||||
out.write("<tag>");
|
||||
out.write(tag);
|
||||
out.write("</tag>");
|
||||
out.write("</property>\n");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,22 @@
|
|||
/*
|
||||
* 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.
|
||||
*/
|
||||
|
||||
/**
|
||||
* This package contains the OzoneConfiguration related tests.
|
||||
*/
|
||||
package org.apache.hadoop.hdds.conf;
|
|
@ -49,7 +49,8 @@ public class MockApprover extends BaseApprover {
|
|||
public X509CertificateHolder sign(SecurityConfig config, PrivateKey caPrivate,
|
||||
X509CertificateHolder caCertificate,
|
||||
Date validFrom, Date validTill,
|
||||
PKCS10CertificationRequest request)
|
||||
PKCS10CertificationRequest request,
|
||||
String scmId, String clusterId)
|
||||
throws IOException, OperatorCreationException {
|
||||
return null;
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
|
|||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.HDDSKeyGenerator;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
import org.bouncycastle.cert.X509CertificateHolder;
|
||||
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||
import org.junit.Before;
|
||||
|
@ -139,14 +140,57 @@ public class TestDefaultCAServer {
|
|||
public void testRequestCertificate() throws IOException,
|
||||
ExecutionException, InterruptedException,
|
||||
NoSuchProviderException, NoSuchAlgorithmException {
|
||||
String scmId = RandomStringUtils.randomAlphabetic(4);
|
||||
String clusterId = RandomStringUtils.randomAlphabetic(4);
|
||||
KeyPair keyPair =
|
||||
new HDDSKeyGenerator(conf).generateKey();
|
||||
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||
.addDnsName("hadoop.apache.org")
|
||||
.addIpAddress("8.8.8.8")
|
||||
.setCA(false)
|
||||
.setClusterID(clusterId)
|
||||
.setScmID(scmId)
|
||||
.setSubject("Ozone Cluster")
|
||||
.setConfiguration(conf)
|
||||
.setKey(keyPair)
|
||||
.build();
|
||||
|
||||
// Let us convert this to a string to mimic the common use case.
|
||||
String csrString = CertificateSignRequest.getEncodedString(csr);
|
||||
|
||||
CertificateServer testCA = new DefaultCAServer("testCA",
|
||||
clusterId, scmId, caStore);
|
||||
testCA.init(new SecurityConfig(conf),
|
||||
CertificateServer.CAType.SELF_SIGNED_CA);
|
||||
|
||||
Future<X509CertificateHolder> holder = testCA.requestCertificate(csrString,
|
||||
CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
|
||||
// Right now our calls are synchronous. Eventually this will have to wait.
|
||||
assertTrue(holder.isDone());
|
||||
assertNotNull(holder.get());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests that we are able
|
||||
* to create a Test CA, creates it own self-Signed CA and then issue a
|
||||
* certificate based on a CSR when scmId and clusterId are not set in
|
||||
* csr subject.
|
||||
* @throws SCMSecurityException - on ERROR.
|
||||
* @throws ExecutionException - on ERROR.
|
||||
* @throws InterruptedException - on ERROR.
|
||||
* @throws NoSuchProviderException - on ERROR.
|
||||
* @throws NoSuchAlgorithmException - on ERROR.
|
||||
*/
|
||||
@Test
|
||||
public void testRequestCertificateWithInvalidSubject() throws IOException,
|
||||
ExecutionException, InterruptedException,
|
||||
NoSuchProviderException, NoSuchAlgorithmException {
|
||||
KeyPair keyPair =
|
||||
new HDDSKeyGenerator(conf).generateKey();
|
||||
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||
.addDnsName("hadoop.apache.org")
|
||||
.addIpAddress("8.8.8.8")
|
||||
.setCA(false)
|
||||
.setClusterID("ClusterID")
|
||||
.setScmID("SCMID")
|
||||
.setSubject("Ozone Cluster")
|
||||
.setConfiguration(conf)
|
||||
.setKey(keyPair)
|
||||
|
@ -168,4 +212,40 @@ public class TestDefaultCAServer {
|
|||
assertNotNull(holder.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRequestCertificateWithInvalidSubjectFailure()
|
||||
throws Exception {
|
||||
KeyPair keyPair =
|
||||
new HDDSKeyGenerator(conf).generateKey();
|
||||
PKCS10CertificationRequest csr = new CertificateSignRequest.Builder()
|
||||
.addDnsName("hadoop.apache.org")
|
||||
.addIpAddress("8.8.8.8")
|
||||
.setCA(false)
|
||||
.setScmID("wrong one")
|
||||
.setClusterID("223432rf")
|
||||
.setSubject("Ozone Cluster")
|
||||
.setConfiguration(conf)
|
||||
.setKey(keyPair)
|
||||
.build();
|
||||
|
||||
// Let us convert this to a string to mimic the common use case.
|
||||
String csrString = CertificateSignRequest.getEncodedString(csr);
|
||||
|
||||
CertificateServer testCA = new DefaultCAServer("testCA",
|
||||
RandomStringUtils.randomAlphabetic(4),
|
||||
RandomStringUtils.randomAlphabetic(4), caStore);
|
||||
testCA.init(new SecurityConfig(conf),
|
||||
CertificateServer.CAType.SELF_SIGNED_CA);
|
||||
|
||||
LambdaTestUtils.intercept(ExecutionException.class, "ScmId and " +
|
||||
"ClusterId in CSR subject are incorrect",
|
||||
() -> {
|
||||
Future<X509CertificateHolder> holder =
|
||||
testCA.requestCertificate(csrString,
|
||||
CertificateApprover.ApprovalType.TESTING_AUTOMATIC);
|
||||
holder.isDone();
|
||||
holder.get();
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -59,12 +59,15 @@ import static org.junit.Assert.assertTrue;
|
|||
@SuppressWarnings("visibilitymodifier")
|
||||
public class TestCertificateClientInit {
|
||||
|
||||
private KeyPair keyPair;
|
||||
private String certSerialId = "3284792342234";
|
||||
private CertificateClient dnCertificateClient;
|
||||
private CertificateClient omCertificateClient;
|
||||
private HDDSKeyGenerator keyGenerator;
|
||||
private Path metaDirPath;
|
||||
private SecurityConfig securityConfig;
|
||||
private KeyCodec keyCodec;
|
||||
private X509Certificate x509Certificate;
|
||||
|
||||
@Parameter
|
||||
public boolean pvtKeyPresent;
|
||||
|
@ -96,10 +99,16 @@ public class TestCertificateClientInit {
|
|||
metaDirPath = Paths.get(path, "test");
|
||||
config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
|
||||
securityConfig = new SecurityConfig(config);
|
||||
dnCertificateClient = new DNCertificateClient(securityConfig);
|
||||
omCertificateClient = new OMCertificateClient(securityConfig);
|
||||
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
||||
keyPair = keyGenerator.generateKey();
|
||||
x509Certificate = getX509Certificate();
|
||||
certSerialId = x509Certificate.getSerialNumber().toString();
|
||||
dnCertificateClient = new DNCertificateClient(securityConfig,
|
||||
certSerialId);
|
||||
omCertificateClient = new OMCertificateClient(securityConfig,
|
||||
certSerialId);
|
||||
keyCodec = new KeyCodec(securityConfig);
|
||||
|
||||
Files.createDirectories(securityConfig.getKeyLocation());
|
||||
}
|
||||
|
||||
|
@ -113,7 +122,6 @@ public class TestCertificateClientInit {
|
|||
|
||||
@Test
|
||||
public void testInitDatanode() throws Exception {
|
||||
KeyPair keyPair = keyGenerator.generateKey();
|
||||
if (pvtKeyPresent) {
|
||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
} else {
|
||||
|
@ -131,9 +139,6 @@ public class TestCertificateClientInit {
|
|||
}
|
||||
|
||||
if (certPresent) {
|
||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
||||
|
||||
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||
codec.writeCertificate(new X509CertificateHolder(
|
||||
x509Certificate.getEncoded()));
|
||||
|
@ -157,7 +162,6 @@ public class TestCertificateClientInit {
|
|||
|
||||
@Test
|
||||
public void testInitOzoneManager() throws Exception {
|
||||
KeyPair keyPair = keyGenerator.generateKey();
|
||||
if (pvtKeyPresent) {
|
||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
} else {
|
||||
|
@ -175,9 +179,6 @@ public class TestCertificateClientInit {
|
|||
}
|
||||
|
||||
if (certPresent) {
|
||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
||||
|
||||
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||
codec.writeCertificate(new X509CertificateHolder(
|
||||
x509Certificate.getEncoded()));
|
||||
|
@ -202,4 +203,9 @@ public class TestCertificateClientInit {
|
|||
securityConfig.getPublicKeyFileName()));
|
||||
}
|
||||
}
|
||||
|
||||
private X509Certificate getX509Certificate() throws Exception {
|
||||
return KeyStoreTestUtil.generateCertificate(
|
||||
"CN=Test", keyPair, 10, securityConfig.getSignatureAlgo());
|
||||
}
|
||||
}
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.hdds.security.x509.certificate.client;
|
||||
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
||||
import org.apache.hadoop.hdds.security.x509.exceptions.CertificateException;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
|
||||
import org.bouncycastle.cert.X509CertificateHolder;
|
||||
import org.junit.After;
|
||||
|
@ -49,8 +50,11 @@ import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
|
|||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.*;
|
||||
import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_METADATA_DIR_NAME;
|
||||
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_NAMES;
|
||||
import static org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient.InitResponse.FAILURE;
|
||||
import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec.getPEMEncodedString;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
|
@ -62,37 +66,60 @@ import static org.junit.Assert.assertTrue;
|
|||
*/
|
||||
public class TestDefaultCertificateClient {
|
||||
|
||||
private String certSerialId;
|
||||
private X509Certificate x509Certificate;
|
||||
private OMCertificateClient omCertClient;
|
||||
private DNCertificateClient dnCertClient;
|
||||
private HDDSKeyGenerator keyGenerator;
|
||||
private Path metaDirPath;
|
||||
private SecurityConfig securityConfig;
|
||||
private Path omMetaDirPath;
|
||||
private Path dnMetaDirPath;
|
||||
private SecurityConfig omSecurityConfig;
|
||||
private SecurityConfig dnSecurityConfig;
|
||||
private final static String UTF = "UTF-8";
|
||||
private KeyCodec keyCodec;
|
||||
private KeyCodec omKeyCodec;
|
||||
private KeyCodec dnKeyCodec;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
OzoneConfiguration config = new OzoneConfiguration();
|
||||
final String path = GenericTestUtils
|
||||
config.setStrings(OZONE_SCM_NAMES, "localhost");
|
||||
config.setInt(IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 2);
|
||||
final String omPath = GenericTestUtils
|
||||
.getTempPath(UUID.randomUUID().toString());
|
||||
metaDirPath = Paths.get(path, "test");
|
||||
config.set(HDDS_METADATA_DIR_NAME, metaDirPath.toString());
|
||||
securityConfig = new SecurityConfig(config);
|
||||
final String dnPath = GenericTestUtils
|
||||
.getTempPath(UUID.randomUUID().toString());
|
||||
|
||||
omMetaDirPath = Paths.get(omPath, "test");
|
||||
dnMetaDirPath = Paths.get(dnPath, "test");
|
||||
|
||||
config.set(HDDS_METADATA_DIR_NAME, omMetaDirPath.toString());
|
||||
omSecurityConfig = new SecurityConfig(config);
|
||||
config.set(HDDS_METADATA_DIR_NAME, dnMetaDirPath.toString());
|
||||
dnSecurityConfig = new SecurityConfig(config);
|
||||
|
||||
|
||||
keyGenerator = new HDDSKeyGenerator(omSecurityConfig);
|
||||
omKeyCodec = new KeyCodec(omSecurityConfig);
|
||||
dnKeyCodec = new KeyCodec(dnSecurityConfig);
|
||||
|
||||
Files.createDirectories(omSecurityConfig.getKeyLocation());
|
||||
Files.createDirectories(dnSecurityConfig.getKeyLocation());
|
||||
x509Certificate = generateX509Cert(null);
|
||||
certSerialId = x509Certificate.getSerialNumber().toString();
|
||||
getCertClient();
|
||||
keyGenerator = new HDDSKeyGenerator(securityConfig);
|
||||
keyCodec = new KeyCodec(securityConfig);
|
||||
Files.createDirectories(securityConfig.getKeyLocation());
|
||||
}
|
||||
|
||||
private void getCertClient() {
|
||||
omCertClient = new OMCertificateClient(securityConfig);
|
||||
dnCertClient = new DNCertificateClient(securityConfig);
|
||||
omCertClient = new OMCertificateClient(omSecurityConfig, certSerialId);
|
||||
dnCertClient = new DNCertificateClient(dnSecurityConfig, certSerialId);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() {
|
||||
omCertClient = null;
|
||||
FileUtils.deleteQuietly(metaDirPath.toFile());
|
||||
dnCertClient = null;
|
||||
FileUtils.deleteQuietly(omMetaDirPath.toFile());
|
||||
FileUtils.deleteQuietly(dnMetaDirPath.toFile());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -101,6 +128,7 @@ public class TestDefaultCertificateClient {
|
|||
*/
|
||||
@Test
|
||||
public void testKeyOperations() throws Exception {
|
||||
cleanupOldKeyPair();
|
||||
PrivateKey pvtKey = omCertClient.getPrivateKey();
|
||||
PublicKey publicKey = omCertClient.getPublicKey();
|
||||
assertNull(publicKey);
|
||||
|
@ -111,18 +139,33 @@ public class TestDefaultCertificateClient {
|
|||
assertNotNull(pvtKey);
|
||||
assertEquals(pvtKey, keyPair.getPrivate());
|
||||
|
||||
publicKey = omCertClient.getPublicKey();
|
||||
publicKey = dnCertClient.getPublicKey();
|
||||
assertNotNull(publicKey);
|
||||
assertEquals(publicKey, keyPair.getPublic());
|
||||
}
|
||||
|
||||
private KeyPair generateKeyPairFiles() throws Exception {
|
||||
cleanupOldKeyPair();
|
||||
KeyPair keyPair = keyGenerator.generateKey();
|
||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
keyCodec.writePublicKey(keyPair.getPublic());
|
||||
omKeyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
omKeyCodec.writePublicKey(keyPair.getPublic());
|
||||
|
||||
dnKeyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
dnKeyCodec.writePublicKey(keyPair.getPublic());
|
||||
return keyPair;
|
||||
}
|
||||
|
||||
private void cleanupOldKeyPair() {
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
}
|
||||
|
||||
/**
|
||||
* Tests: 1. storeCertificate 2. getCertificate 3. verifyCertificate
|
||||
*/
|
||||
|
@ -130,11 +173,11 @@ public class TestDefaultCertificateClient {
|
|||
public void testCertificateOps() throws Exception {
|
||||
X509Certificate cert = omCertClient.getCertificate();
|
||||
assertNull(cert);
|
||||
omCertClient.storeCertificate(getPEMEncodedString(x509Certificate),
|
||||
true);
|
||||
|
||||
X509Certificate x509Certificate = generateX509Cert(null);
|
||||
omCertClient.storeCertificate(x509Certificate);
|
||||
|
||||
cert = omCertClient.getCertificate();
|
||||
cert = omCertClient.getCertificate(
|
||||
x509Certificate.getSerialNumber().toString());
|
||||
assertNotNull(cert);
|
||||
assertTrue(cert.getEncoded().length > 0);
|
||||
assertEquals(cert, x509Certificate);
|
||||
|
@ -147,12 +190,17 @@ public class TestDefaultCertificateClient {
|
|||
keyPair = generateKeyPairFiles();
|
||||
}
|
||||
return KeyStoreTestUtil.generateCertificate("CN=Test", keyPair, 30,
|
||||
securityConfig.getSignatureAlgo());
|
||||
omSecurityConfig.getSignatureAlgo());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSignDataStream() throws Exception {
|
||||
String data = RandomStringUtils.random(100);
|
||||
String data = RandomStringUtils.random(100, UTF);
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
|
||||
// Expect error when there is no private key to sign.
|
||||
LambdaTestUtils.intercept(IOException.class, "Error while " +
|
||||
"signing the stream",
|
||||
|
@ -171,8 +219,8 @@ public class TestDefaultCertificateClient {
|
|||
private void validateHash(byte[] hash, byte[] data)
|
||||
throws Exception {
|
||||
Signature rsaSignature =
|
||||
Signature.getInstance(securityConfig.getSignatureAlgo(),
|
||||
securityConfig.getProvider());
|
||||
Signature.getInstance(omSecurityConfig.getSignatureAlgo(),
|
||||
omSecurityConfig.getProvider());
|
||||
rsaSignature.initVerify(omCertClient.getPublicKey());
|
||||
rsaSignature.update(data);
|
||||
Assert.assertTrue(rsaSignature.verify(hash));
|
||||
|
@ -184,8 +232,6 @@ public class TestDefaultCertificateClient {
|
|||
@Test
|
||||
public void verifySignatureStream() throws Exception {
|
||||
String data = RandomStringUtils.random(500, UTF);
|
||||
|
||||
X509Certificate x509Certificate = generateX509Cert(null);
|
||||
byte[] sign = omCertClient.signDataStream(IOUtils.toInputStream(data,
|
||||
UTF));
|
||||
|
||||
|
@ -209,7 +255,6 @@ public class TestDefaultCertificateClient {
|
|||
@Test
|
||||
public void verifySignatureDataArray() throws Exception {
|
||||
String data = RandomStringUtils.random(500, UTF);
|
||||
X509Certificate x509Certificate = generateX509Cert(null);
|
||||
byte[] sign = omCertClient.signData(data.getBytes());
|
||||
|
||||
// Positive tests.
|
||||
|
@ -233,6 +278,67 @@ public class TestDefaultCertificateClient {
|
|||
() -> omCertClient.queryCertificate(""));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCertificateLoadingOnInit() throws Exception {
|
||||
KeyPair keyPair = keyGenerator.generateKey();
|
||||
X509Certificate cert1 = generateX509Cert(keyPair);
|
||||
X509Certificate cert2 = generateX509Cert(keyPair);
|
||||
X509Certificate cert3 = generateX509Cert(keyPair);
|
||||
|
||||
Path certPath = dnSecurityConfig.getCertificateLocation();
|
||||
CertificateCodec codec = new CertificateCodec(dnSecurityConfig);
|
||||
|
||||
// Certificate not found.
|
||||
LambdaTestUtils.intercept(CertificateException.class, "Error while" +
|
||||
" getting certificate",
|
||||
() -> dnCertClient.getCertificate(cert1.getSerialNumber()
|
||||
.toString()));
|
||||
LambdaTestUtils.intercept(CertificateException.class, "Error while" +
|
||||
" getting certificate",
|
||||
() -> dnCertClient.getCertificate(cert2.getSerialNumber()
|
||||
.toString()));
|
||||
LambdaTestUtils.intercept(CertificateException.class, "Error while" +
|
||||
" getting certificate",
|
||||
() -> dnCertClient.getCertificate(cert3.getSerialNumber()
|
||||
.toString()));
|
||||
codec.writeCertificate(certPath, "1.crt",
|
||||
getPEMEncodedString(cert1), true);
|
||||
codec.writeCertificate(certPath, "2.crt",
|
||||
getPEMEncodedString(cert2), true);
|
||||
codec.writeCertificate(certPath, "3.crt",
|
||||
getPEMEncodedString(cert3), true);
|
||||
|
||||
// Re instentiate DN client which will load certificates from filesystem.
|
||||
dnCertClient = new DNCertificateClient(dnSecurityConfig, certSerialId);
|
||||
|
||||
assertNotNull(dnCertClient.getCertificate(cert1.getSerialNumber()
|
||||
.toString()));
|
||||
assertNotNull(dnCertClient.getCertificate(cert2.getSerialNumber()
|
||||
.toString()));
|
||||
assertNotNull(dnCertClient.getCertificate(cert3.getSerialNumber()
|
||||
.toString()));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStoreCertificate() throws Exception {
|
||||
KeyPair keyPair = keyGenerator.generateKey();
|
||||
X509Certificate cert1 = generateX509Cert(keyPair);
|
||||
X509Certificate cert2 = generateX509Cert(keyPair);
|
||||
X509Certificate cert3 = generateX509Cert(keyPair);
|
||||
|
||||
dnCertClient.storeCertificate(getPEMEncodedString(cert1), true);
|
||||
dnCertClient.storeCertificate(getPEMEncodedString(cert2), true);
|
||||
dnCertClient.storeCertificate(getPEMEncodedString(cert3), true);
|
||||
|
||||
assertNotNull(dnCertClient.getCertificate(cert1.getSerialNumber()
|
||||
.toString()));
|
||||
assertNotNull(dnCertClient.getCertificate(cert2.getSerialNumber()
|
||||
.toString()));
|
||||
assertNotNull(dnCertClient.getCertificate(cert3.getSerialNumber()
|
||||
.toString()));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testInitCertAndKeypairValidationFailures() throws Exception {
|
||||
|
||||
|
@ -246,13 +352,23 @@ public class TestDefaultCertificateClient {
|
|||
omClientLog.clearOutput();
|
||||
|
||||
// Case 1. Expect failure when keypair validation fails.
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||
keyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
|
||||
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
|
||||
omKeyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
omKeyCodec.writePublicKey(keyPair2.getPublic());
|
||||
|
||||
dnKeyCodec.writePrivateKey(keyPair.getPrivate());
|
||||
dnKeyCodec.writePublicKey(keyPair2.getPublic());
|
||||
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||
keyCodec.writePublicKey(keyPair2.getPublic());
|
||||
|
||||
// Check for DN.
|
||||
assertEquals(dnCertClient.init(), FAILURE);
|
||||
|
@ -271,15 +387,18 @@ public class TestDefaultCertificateClient {
|
|||
// Case 2. Expect failure when certificate is generated from different
|
||||
// private key and keypair validation fails.
|
||||
getCertClient();
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getCertificateFileName()).toFile());
|
||||
X509Certificate x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||
"CN=Test", keyGenerator.generateKey(), 10,
|
||||
securityConfig.getSignatureAlgo());
|
||||
CertificateCodec codec = new CertificateCodec(securityConfig);
|
||||
codec.writeCertificate(new X509CertificateHolder(
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getCertificateFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getCertificateFileName()).toFile());
|
||||
|
||||
CertificateCodec omCertCodec = new CertificateCodec(omSecurityConfig);
|
||||
omCertCodec.writeCertificate(new X509CertificateHolder(
|
||||
x509Certificate.getEncoded()));
|
||||
|
||||
CertificateCodec dnCertCodec = new CertificateCodec(dnSecurityConfig);
|
||||
dnCertCodec.writeCertificate(new X509CertificateHolder(
|
||||
x509Certificate.getEncoded()));
|
||||
// Check for DN.
|
||||
assertEquals(dnCertClient.init(), FAILURE);
|
||||
assertTrue(dnClientLog.getOutput().contains("Keypair validation " +
|
||||
|
@ -297,10 +416,13 @@ public class TestDefaultCertificateClient {
|
|||
// private key and certificate validation fails.
|
||||
|
||||
// Re write the correct public key.
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
getCertClient();
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||
keyCodec.writePublicKey(keyPair.getPublic());
|
||||
omKeyCodec.writePublicKey(keyPair.getPublic());
|
||||
dnKeyCodec.writePublicKey(keyPair.getPublic());
|
||||
|
||||
// Check for DN.
|
||||
assertEquals(dnCertClient.init(), FAILURE);
|
||||
|
@ -318,8 +440,10 @@ public class TestDefaultCertificateClient {
|
|||
|
||||
// Case 4. Failure when public key recovery fails.
|
||||
getCertClient();
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(omSecurityConfig.getKeyLocation()
|
||||
.toString(), omSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(dnSecurityConfig.getKeyLocation()
|
||||
.toString(), dnSecurityConfig.getPublicKeyFileName()).toFile());
|
||||
|
||||
// Check for DN.
|
||||
assertEquals(dnCertClient.init(), FAILURE);
|
||||
|
|
|
@ -213,24 +213,6 @@ public class TestCertificateSignRequest {
|
|||
builder.setSubject(subject);
|
||||
}
|
||||
|
||||
// Now try with blank/null SCM ID
|
||||
try {
|
||||
builder.setScmID(null);
|
||||
builder.build();
|
||||
Assert.fail("Null/Blank SCM ID should have thrown.");
|
||||
} catch (IllegalArgumentException e) {
|
||||
builder.setScmID(scmID);
|
||||
}
|
||||
|
||||
// Now try with blank/null SCM ID
|
||||
try {
|
||||
builder.setClusterID(null);
|
||||
builder.build();
|
||||
Assert.fail("Null/Blank Cluster ID should have thrown.");
|
||||
} catch (IllegalArgumentException e) {
|
||||
builder.setClusterID(clusterID);
|
||||
}
|
||||
|
||||
// Now try with invalid IP address
|
||||
try {
|
||||
builder.addIpAddress("255.255.255.*");
|
||||
|
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* 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.tracing;
|
||||
|
||||
import io.jaegertracing.internal.JaegerSpanContext;
|
||||
import io.jaegertracing.internal.exceptions.EmptyTracerStateStringException;
|
||||
import io.jaegertracing.internal.exceptions.MalformedTracerStateStringException;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
class TestStringCodec {
|
||||
|
||||
@Test
|
||||
void testExtract() throws Exception {
|
||||
StringCodec codec = new StringCodec();
|
||||
|
||||
LambdaTestUtils.intercept(EmptyTracerStateStringException.class,
|
||||
() -> codec.extract(null));
|
||||
|
||||
StringBuilder sb = new StringBuilder().append("123");
|
||||
LambdaTestUtils.intercept(MalformedTracerStateStringException.class,
|
||||
"String does not match tracer state format",
|
||||
() -> codec.extract(sb));
|
||||
|
||||
sb.append(":456:789");
|
||||
LambdaTestUtils.intercept(MalformedTracerStateStringException.class,
|
||||
"String does not match tracer state format",
|
||||
() -> codec.extract(sb));
|
||||
sb.append(":66");
|
||||
JaegerSpanContext context = codec.extract(sb);
|
||||
String expectedContextString = new String("123:456:789:66");
|
||||
assertTrue(context.getTraceId().equals("123"));
|
||||
assertTrue(context.toString().equals(expectedContextString));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,21 @@
|
|||
/**
|
||||
* 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.tracing;
|
||||
/**
|
||||
Test cases for ozone tracing.
|
||||
*/
|
|
@ -0,0 +1,86 @@
|
|||
/**
|
||||
* 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.ozone;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||
import org.apache.hadoop.hdds.server.BaseHttpServer;
|
||||
|
||||
/**
|
||||
* Simple http server to provide basic monitoring for hdds datanode.
|
||||
* <p>
|
||||
* This server is used to access default /conf /prom /prof endpoints.
|
||||
*/
|
||||
public class HddsDatanodeHttpServer extends BaseHttpServer {
|
||||
|
||||
public HddsDatanodeHttpServer(Configuration conf) throws IOException {
|
||||
super(conf, "hddsDatanode");
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getHttpAddressKey() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_ADDRESS_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getHttpBindHostKey() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_HOST_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getHttpsAddressKey() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTPS_ADDRESS_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getHttpsBindHostKey() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTPS_BIND_HOST_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getBindHostDefault() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_HOST_DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getHttpBindPortDefault() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_BIND_PORT_DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected int getHttpsBindPortDefault() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTPS_BIND_PORT_DEFAULT;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getKeytabFile() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_KERBEROS_KEYTAB_FILE_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getSpnegoPrincipal() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_KERBEROS_PRINCIPAL_KEY;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getEnabledKey() {
|
||||
return HddsConfigKeys.HDDS_DATANODE_HTTP_ENABLED_KEY;
|
||||
}
|
||||
}
|
|
@ -26,17 +26,23 @@ import org.apache.hadoop.hdds.cli.GenericCli;
|
|||
import org.apache.hadoop.hdds.cli.HddsVersionProvider;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol;
|
||||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.DNCertificateClient;
|
||||
import org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest;
|
||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||
import org.apache.hadoop.ozone.container.common.statemachine.DatanodeStateMachine;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||
import org.apache.hadoop.util.ServicePlugin;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import picocli.CommandLine.Command;
|
||||
|
@ -44,9 +50,13 @@ import picocli.CommandLine.Command;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.security.KeyPair;
|
||||
import java.security.cert.CertificateException;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
|
||||
import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec.getX509Certificate;
|
||||
import static org.apache.hadoop.hdds.security.x509.certificates.utils.CertificateSignRequest.getEncodedString;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY;
|
||||
import static org.apache.hadoop.util.ExitUtil.terminate;
|
||||
|
||||
|
@ -67,6 +77,9 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
private DatanodeDetails datanodeDetails;
|
||||
private DatanodeStateMachine datanodeStateMachine;
|
||||
private List<ServicePlugin> plugins;
|
||||
private CertificateClient dnCertClient;
|
||||
private String component;
|
||||
private HddsDatanodeHttpServer httpServer;
|
||||
|
||||
/**
|
||||
* Default constructor.
|
||||
|
@ -133,6 +146,10 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
}
|
||||
}
|
||||
|
||||
public static Logger getLogger() {
|
||||
return LOG;
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts HddsDatanode services.
|
||||
*
|
||||
|
@ -158,13 +175,16 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
.substring(0, 8));
|
||||
LOG.info("HddsDatanodeService host:{} ip:{}", hostname, ip);
|
||||
// Authenticate Hdds Datanode service if security is enabled
|
||||
if (conf.getBoolean(OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY,
|
||||
true)) {
|
||||
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
||||
component = "dn-" + datanodeDetails.getUuidString();
|
||||
|
||||
dnCertClient = new DNCertificateClient(new SecurityConfig(conf),
|
||||
datanodeDetails.getCertSerialId());
|
||||
|
||||
if (SecurityUtil.getAuthenticationMethod(conf).equals(
|
||||
UserGroupInformation.AuthenticationMethod.KERBEROS)) {
|
||||
LOG.debug("Ozone security is enabled. Attempting login for Hdds " +
|
||||
"Datanode user. "
|
||||
+ "Principal: {},keytab: {}", conf.get(
|
||||
LOG.info("Ozone security is enabled. Attempting login for Hdds " +
|
||||
"Datanode user. Principal: {},keytab: {}", conf.get(
|
||||
DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY),
|
||||
conf.get(DFSConfigKeys.DFS_DATANODE_KEYTAB_FILE_KEY));
|
||||
|
||||
|
@ -179,7 +199,17 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
}
|
||||
LOG.info("Hdds Datanode login successful.");
|
||||
}
|
||||
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf);
|
||||
if (OzoneSecurityUtil.isSecurityEnabled(conf)) {
|
||||
initializeCertificateClient(conf);
|
||||
}
|
||||
datanodeStateMachine = new DatanodeStateMachine(datanodeDetails, conf,
|
||||
dnCertClient);
|
||||
try {
|
||||
httpServer = new HddsDatanodeHttpServer(conf);
|
||||
httpServer.start();
|
||||
} catch (Exception ex) {
|
||||
LOG.error("HttpServer failed to start.", ex);
|
||||
}
|
||||
startPlugins();
|
||||
// Starting HDDS Daemons
|
||||
datanodeStateMachine.startDaemon();
|
||||
|
@ -192,6 +222,87 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Initializes secure Datanode.
|
||||
* */
|
||||
@VisibleForTesting
|
||||
public void initializeCertificateClient(OzoneConfiguration config)
|
||||
throws IOException {
|
||||
LOG.info("Initializing secure Datanode.");
|
||||
|
||||
CertificateClient.InitResponse response = dnCertClient.init();
|
||||
LOG.info("Init response: {}", response);
|
||||
switch (response) {
|
||||
case SUCCESS:
|
||||
LOG.info("Initialization successful, case:{}.", response);
|
||||
break;
|
||||
case GETCERT:
|
||||
getSCMSignedCert(config);
|
||||
LOG.info("Successfully stored SCM signed certificate, case:{}.",
|
||||
response);
|
||||
break;
|
||||
case FAILURE:
|
||||
LOG.error("DN security initialization failed, case:{}.", response);
|
||||
throw new RuntimeException("DN security initialization failed.");
|
||||
case RECOVER:
|
||||
LOG.error("DN security initialization failed, case:{}. OM certificate " +
|
||||
"is missing.", response);
|
||||
throw new RuntimeException("DN security initialization failed.");
|
||||
default:
|
||||
LOG.error("DN security initialization failed. Init response: {}",
|
||||
response);
|
||||
throw new RuntimeException("DN security initialization failed.");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get SCM signed certificate and store it using certificate client.
|
||||
* @param config
|
||||
* */
|
||||
private void getSCMSignedCert(OzoneConfiguration config) {
|
||||
try {
|
||||
PKCS10CertificationRequest csr = getCSR(config);
|
||||
// TODO: For SCM CA we should fetch certificate from multiple SCMs.
|
||||
SCMSecurityProtocol secureScmClient =
|
||||
HddsUtils.getScmSecurityClient(config,
|
||||
HddsUtils.getScmAddressForSecurityProtocol(config));
|
||||
|
||||
String pemEncodedCert = secureScmClient.getDataNodeCertificate(
|
||||
datanodeDetails.getProtoBufMessage(), getEncodedString(csr));
|
||||
dnCertClient.storeCertificate(pemEncodedCert, true);
|
||||
datanodeDetails.setCertSerialId(getX509Certificate(pemEncodedCert).
|
||||
getSerialNumber().toString());
|
||||
persistDatanodeDetails(datanodeDetails);
|
||||
} catch (IOException | CertificateException e) {
|
||||
LOG.error("Error while storing SCM signed certificate.", e);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates CSR for DN.
|
||||
* @param config
|
||||
* */
|
||||
@VisibleForTesting
|
||||
public PKCS10CertificationRequest getCSR(Configuration config)
|
||||
throws IOException {
|
||||
CertificateSignRequest.Builder builder = dnCertClient.getCSRBuilder();
|
||||
KeyPair keyPair = new KeyPair(dnCertClient.getPublicKey(),
|
||||
dnCertClient.getPrivateKey());
|
||||
|
||||
String hostname = InetAddress.getLocalHost().getCanonicalHostName();
|
||||
String subject = UserGroupInformation.getCurrentUser()
|
||||
.getShortUserName() + "@" + hostname;
|
||||
|
||||
builder.setCA(false)
|
||||
.setKey(keyPair)
|
||||
.setConfiguration(config)
|
||||
.setSubject(subject);
|
||||
|
||||
LOG.info("Creating csr for DN-> subject:{}", subject);
|
||||
return builder.build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns DatanodeDetails or null in case of Error.
|
||||
*
|
||||
|
@ -221,6 +332,29 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Persist DatanodeDetails to file system.
|
||||
* @param dnDetails
|
||||
*
|
||||
* @return DatanodeDetails
|
||||
*/
|
||||
private void persistDatanodeDetails(DatanodeDetails dnDetails)
|
||||
throws IOException {
|
||||
String idFilePath = HddsUtils.getDatanodeIdFilePath(conf);
|
||||
if (idFilePath == null || idFilePath.isEmpty()) {
|
||||
LOG.error("A valid file path is needed for config setting {}",
|
||||
ScmConfigKeys.OZONE_SCM_DATANODE_ID);
|
||||
throw new IllegalArgumentException(ScmConfigKeys.OZONE_SCM_DATANODE_ID +
|
||||
" must be defined. See" +
|
||||
" https://wiki.apache.org/hadoop/Ozone#Configuration" +
|
||||
" for details on configuring Ozone.");
|
||||
}
|
||||
|
||||
Preconditions.checkNotNull(idFilePath);
|
||||
File idFile = new File(idFilePath);
|
||||
ContainerUtils.writeDatanodeDetailsTo(dnDetails, idFile);
|
||||
}
|
||||
|
||||
/**
|
||||
* Starts all the service plugins which are configured using
|
||||
* OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY.
|
||||
|
@ -294,6 +428,14 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
if (datanodeStateMachine != null) {
|
||||
datanodeStateMachine.stopDaemon();
|
||||
}
|
||||
if (httpServer != null) {
|
||||
try {
|
||||
httpServer.stop();
|
||||
} catch (Exception e) {
|
||||
LOG.error("Stopping HttpServer is failed.", e);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -308,4 +450,18 @@ public class HddsDatanodeService extends GenericCli implements ServicePlugin {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public String getComponent() {
|
||||
return component;
|
||||
}
|
||||
|
||||
public CertificateClient getCertificateClient() {
|
||||
return dnCertClient;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public void setCertificateClient(CertificateClient client) {
|
||||
dnCertClient = client;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,9 +35,11 @@ import org.slf4j.LoggerFactory;
|
|||
import java.io.IOException;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ConcurrentNavigableMap;
|
||||
import java.util.concurrent.ConcurrentSkipListMap;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
|
||||
|
@ -50,7 +52,8 @@ public class ContainerSet {
|
|||
|
||||
private final ConcurrentSkipListMap<Long, Container> containerMap = new
|
||||
ConcurrentSkipListMap<>();
|
||||
|
||||
private final ConcurrentSkipListSet<Long> missingContainerSet =
|
||||
new ConcurrentSkipListSet<>();
|
||||
/**
|
||||
* Add Container to container map.
|
||||
* @param container
|
||||
|
@ -128,6 +131,7 @@ public class ContainerSet {
|
|||
* @return containerMap Iterator
|
||||
*/
|
||||
public Iterator<Map.Entry<Long, Container>> getContainerMapIterator() {
|
||||
containerMap.keySet().stream().collect(Collectors.toSet());
|
||||
return containerMap.entrySet().iterator();
|
||||
}
|
||||
|
||||
|
@ -218,4 +222,20 @@ public class ContainerSet {
|
|||
return deletionPolicy
|
||||
.chooseContainerForBlockDeletion(count, containerDataMap);
|
||||
}
|
||||
|
||||
public Set<Long> getMissingContainerSet() {
|
||||
return missingContainerSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builds the missing container set by taking a diff total no containers
|
||||
* actually found and number of containers which actually got created.
|
||||
* This will only be called during the initialization of Datanode Service
|
||||
* when it still not a part of any write Pipeline.
|
||||
* @param createdContainerSet ContainerId set persisted in the Ratis snapshot
|
||||
*/
|
||||
public void buildMissingContainerSet(Set<Long> createdContainerSet) {
|
||||
missingContainerSet.addAll(createdContainerSet);
|
||||
missingContainerSet.removeAll(containerMap.keySet());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -66,6 +66,7 @@ import org.slf4j.LoggerFactory;
|
|||
|
||||
import java.util.Map;
|
||||
import java.util.Optional;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Ozone Container dispatcher takes a call from the netty server and routes it
|
||||
|
@ -101,7 +102,6 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|||
this.containerCloseThreshold = conf.getFloat(
|
||||
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
|
||||
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -132,6 +132,12 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void buildMissingContainerSet(Set<Long> createdContainerSet) {
|
||||
containerSet.buildMissingContainerSet(createdContainerSet);
|
||||
}
|
||||
|
||||
@SuppressWarnings("methodlength")
|
||||
@Override
|
||||
public ContainerCommandResponseProto dispatch(
|
||||
ContainerCommandRequestProto msg, DispatcherContext dispatcherContext) {
|
||||
|
@ -145,18 +151,61 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|||
Map<String, String> params =
|
||||
ContainerCommandRequestPBHelper.getAuditParams(msg);
|
||||
|
||||
Container container = null;
|
||||
ContainerType containerType = null;
|
||||
Container container;
|
||||
ContainerType containerType;
|
||||
ContainerCommandResponseProto responseProto = null;
|
||||
long startTime = System.nanoTime();
|
||||
ContainerProtos.Type cmdType = msg.getCmdType();
|
||||
long containerID = msg.getContainerID();
|
||||
metrics.incContainerOpsMetrics(cmdType);
|
||||
container = getContainer(containerID);
|
||||
boolean isWriteStage =
|
||||
(cmdType == ContainerProtos.Type.WriteChunk && dispatcherContext != null
|
||||
&& dispatcherContext.getStage()
|
||||
== DispatcherContext.WriteChunkStage.WRITE_DATA);
|
||||
boolean isWriteCommitStage =
|
||||
(cmdType == ContainerProtos.Type.WriteChunk && dispatcherContext != null
|
||||
&& dispatcherContext.getStage()
|
||||
== DispatcherContext.WriteChunkStage.COMMIT_DATA);
|
||||
|
||||
// if the command gets executed other than Ratis, the default wroite stage
|
||||
// is WriteChunkStage.COMBINED
|
||||
boolean isCombinedStage =
|
||||
cmdType == ContainerProtos.Type.WriteChunk && (dispatcherContext == null
|
||||
|| dispatcherContext.getStage()
|
||||
== DispatcherContext.WriteChunkStage.COMBINED);
|
||||
Set<Long> containerIdSet = null;
|
||||
if (dispatcherContext != null) {
|
||||
containerIdSet = dispatcherContext.getCreateContainerSet();
|
||||
}
|
||||
if (isWriteCommitStage) {
|
||||
// check if the container Id exist in the loaded snapshot file. if
|
||||
// it does not , it infers that , this is a restart of dn where
|
||||
// the we are reapplying the transaction which was not captured in the
|
||||
// snapshot.
|
||||
// just add it to the list, and remove it from missing container set
|
||||
// as it might have been added in the list during "init".
|
||||
Preconditions.checkNotNull(containerIdSet);
|
||||
if (!containerIdSet.contains(containerID)) {
|
||||
containerIdSet.add(containerID);
|
||||
containerSet.getMissingContainerSet().remove(containerID);
|
||||
}
|
||||
}
|
||||
if (getMissingContainerSet().contains(containerID)) {
|
||||
StorageContainerException sce = new StorageContainerException(
|
||||
"ContainerID " + containerID
|
||||
+ " has been lost and and cannot be recreated on this DataNode",
|
||||
ContainerProtos.Result.CONTAINER_MISSING);
|
||||
audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
||||
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
||||
}
|
||||
|
||||
if (cmdType != ContainerProtos.Type.CreateContainer) {
|
||||
container = getContainer(containerID);
|
||||
|
||||
if (container == null && (cmdType == ContainerProtos.Type.WriteChunk
|
||||
/**
|
||||
* Create Container should happen only as part of Write_Data phase of
|
||||
* writeChunk.
|
||||
*/
|
||||
if (container == null && ((isWriteStage || isCombinedStage)
|
||||
|| cmdType == ContainerProtos.Type.PutSmallFile)) {
|
||||
// If container does not exist, create one for WriteChunk and
|
||||
// PutSmallFile request
|
||||
|
@ -168,7 +217,12 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|||
audit(action, eventType, params, AuditEventStatus.FAILURE, sce);
|
||||
return ContainerUtils.logAndReturnError(LOG, sce, msg);
|
||||
}
|
||||
|
||||
Preconditions.checkArgument(isWriteStage && containerIdSet != null
|
||||
|| dispatcherContext == null);
|
||||
if (containerIdSet != null) {
|
||||
// adds this container to list of containers created in the pipeline
|
||||
containerIdSet.add(containerID);
|
||||
}
|
||||
container = getContainer(containerID);
|
||||
}
|
||||
|
||||
|
@ -406,6 +460,11 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
|
|||
return containerSet.getContainer(containerID);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Set<Long> getMissingContainerSet() {
|
||||
return containerSet.getMissingContainerSet();
|
||||
}
|
||||
|
||||
private ContainerType getContainerType(Container container) {
|
||||
return container.getContainerType();
|
||||
}
|
||||
|
|
|
@ -26,6 +26,8 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
|||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||
import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Dispatcher acts as the bridge between the transport layer and
|
||||
* the actual container layer. This layer is capable of transforming
|
||||
|
@ -58,6 +60,12 @@ public interface ContainerDispatcher {
|
|||
*/
|
||||
void init();
|
||||
|
||||
/**
|
||||
* finds and builds the missing containers in case of a lost disk etc
|
||||
* in the ContainerSet.
|
||||
*/
|
||||
void buildMissingContainerSet(Set<Long> createdContainers);
|
||||
|
||||
/**
|
||||
* Shutdown Dispatcher services.
|
||||
*/
|
||||
|
|
|
@ -33,6 +33,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
|||
.StorageContainerDatanodeProtocolProtos.ContainerReportsProto;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.NodeReportProto;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.ozone.container.common.report.ReportManager;
|
||||
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler
|
||||
.CloseContainerCommandHandler;
|
||||
|
@ -82,15 +83,17 @@ public class DatanodeStateMachine implements Closeable {
|
|||
private final ReplicationSupervisor supervisor;
|
||||
|
||||
private JvmPauseMonitor jvmPauseMonitor;
|
||||
private CertificateClient dnCertClient;
|
||||
|
||||
/**
|
||||
* Constructs a a datanode state machine.
|
||||
*
|
||||
* @param datanodeDetails - DatanodeDetails used to identify a datanode
|
||||
* @param conf - Configuration.
|
||||
* @param certClient - Datanode Certificate client, required if security is
|
||||
* enabled
|
||||
*/
|
||||
public DatanodeStateMachine(DatanodeDetails datanodeDetails,
|
||||
Configuration conf) throws IOException {
|
||||
Configuration conf, CertificateClient certClient) throws IOException {
|
||||
this.conf = conf;
|
||||
this.datanodeDetails = datanodeDetails;
|
||||
executorService = HadoopExecutors.newCachedThreadPool(
|
||||
|
@ -99,7 +102,8 @@ public class DatanodeStateMachine implements Closeable {
|
|||
connectionManager = new SCMConnectionManager(conf);
|
||||
context = new StateContext(this.conf, DatanodeStates.getInitState(), this);
|
||||
container = new OzoneContainer(this.datanodeDetails,
|
||||
new OzoneConfiguration(conf), context);
|
||||
new OzoneConfiguration(conf), context, certClient);
|
||||
dnCertClient = certClient;
|
||||
nextHB = new AtomicLong(Time.monotonicNow());
|
||||
|
||||
ContainerReplicator replicator =
|
||||
|
|
|
@ -32,6 +32,8 @@ import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient
|
|||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
import static org.apache.hadoop.hdds.security.exception.SCMSecurityException.ErrorCode.MISSING_BLOCK_TOKEN;
|
||||
|
||||
/**
|
||||
* A server endpoint that acts as the communication layer for Ozone containers.
|
||||
*/
|
||||
|
@ -39,10 +41,12 @@ public abstract class XceiverServer implements XceiverServerSpi {
|
|||
|
||||
private final SecurityConfig secConfig;
|
||||
private final TokenVerifier tokenVerifier;
|
||||
private final CertificateClient caClient;
|
||||
|
||||
public XceiverServer(Configuration conf) {
|
||||
public XceiverServer(Configuration conf, CertificateClient client) {
|
||||
Preconditions.checkNotNull(conf);
|
||||
this.secConfig = new SecurityConfig(conf);
|
||||
this.caClient = client;
|
||||
tokenVerifier = new BlockTokenVerifier(secConfig, getCaClient());
|
||||
}
|
||||
|
||||
|
@ -59,17 +63,15 @@ public abstract class XceiverServer implements XceiverServerSpi {
|
|||
String encodedToken = request.getEncodedToken();
|
||||
if (encodedToken == null) {
|
||||
throw new SCMSecurityException("Security is enabled but client " +
|
||||
"request is missing block token.",
|
||||
SCMSecurityException.ErrorCode.MISSING_BLOCK_TOKEN);
|
||||
"request is missing block token.", MISSING_BLOCK_TOKEN);
|
||||
}
|
||||
tokenVerifier.verify(encodedToken, "");
|
||||
tokenVerifier.verify(encodedToken, encodedToken);
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
protected CertificateClient getCaClient() {
|
||||
// TODO: instantiate CertificateClient
|
||||
return null;
|
||||
return caClient;
|
||||
}
|
||||
|
||||
protected SecurityConfig getSecurityConfig() {
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
|||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.
|
||||
StorageContainerException;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.hdds.tracing.GrpcServerInterceptor;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
|
@ -73,8 +74,9 @@ public final class XceiverServerGrpc extends XceiverServer {
|
|||
* @param conf - Configuration
|
||||
*/
|
||||
public XceiverServerGrpc(DatanodeDetails datanodeDetails, Configuration conf,
|
||||
ContainerDispatcher dispatcher, BindableService... additionalServices) {
|
||||
super(conf);
|
||||
ContainerDispatcher dispatcher, CertificateClient caClient,
|
||||
BindableService... additionalServices) {
|
||||
super(conf, caClient);
|
||||
Preconditions.checkNotNull(conf);
|
||||
|
||||
this.id = datanodeDetails.getUuid();
|
||||
|
|
|
@ -21,10 +21,13 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
|||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.cache.Cache;
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.hdds.HddsUtils;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
|
||||
|
||||
import io.opentracing.Scope;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerUtils;
|
||||
import org.apache.ratis.proto.RaftProtos.RaftPeerRole;
|
||||
import org.apache.ratis.protocol.RaftGroup;
|
||||
import org.apache.ratis.protocol.RaftGroupId;
|
||||
|
@ -36,6 +39,8 @@ import org.apache.ratis.statemachine.impl.SingleFileSnapshotInfo;
|
|||
import org.apache.ratis.thirdparty.com.google.protobuf
|
||||
.InvalidProtocolBufferException;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.
|
||||
ContainerIdSetProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
.ContainerCommandRequestProto;
|
||||
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
|
||||
|
@ -73,10 +78,15 @@ import java.util.concurrent.CompletableFuture;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ThreadPoolExecutor;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentSkipListSet;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.OutputStream;
|
||||
|
||||
|
||||
/** A {@link org.apache.ratis.statemachine.StateMachine} for containers.
|
||||
*
|
||||
|
@ -126,10 +136,12 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
private final XceiverServerRatis ratisServer;
|
||||
private final ConcurrentHashMap<Long, CompletableFuture<Message>>
|
||||
writeChunkFutureMap;
|
||||
|
||||
// keeps track of the containers created per pipeline
|
||||
private final Set<Long> createContainerSet;
|
||||
private ExecutorService[] executors;
|
||||
private final int numExecutors;
|
||||
private final Map<Long, Long> applyTransactionCompletionMap;
|
||||
private long lastIndex;
|
||||
private final Cache<Long, ByteString> stateMachineDataCache;
|
||||
private final boolean isBlockTokenEnabled;
|
||||
private final TokenVerifier tokenVerifier;
|
||||
|
@ -152,7 +164,6 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
this.executors = executors.toArray(new ExecutorService[numExecutors]);
|
||||
this.writeChunkFutureMap = new ConcurrentHashMap<>();
|
||||
applyTransactionCompletionMap = new ConcurrentHashMap<>();
|
||||
this.lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
||||
stateMachineDataCache = CacheBuilder.newBuilder()
|
||||
.expireAfterAccess(expiryInterval, TimeUnit.MILLISECONDS)
|
||||
// set the limit on no of cached entries equal to no of max threads
|
||||
|
@ -160,6 +171,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
.maximumSize(chunkExecutor.getCorePoolSize()).build();
|
||||
this.isBlockTokenEnabled = isBlockTokenEnabled;
|
||||
this.tokenVerifier = tokenVerifier;
|
||||
this.createContainerSet = new ConcurrentSkipListSet<>();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -181,38 +193,64 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
loadSnapshot(storage.getLatestSnapshot());
|
||||
}
|
||||
|
||||
private long loadSnapshot(SingleFileSnapshotInfo snapshot) {
|
||||
private long loadSnapshot(SingleFileSnapshotInfo snapshot)
|
||||
throws IOException {
|
||||
if (snapshot == null) {
|
||||
TermIndex empty = TermIndex.newTermIndex(0,
|
||||
RaftServerConstants.INVALID_LOG_INDEX);
|
||||
LOG.info("The snapshot info is null." +
|
||||
"Setting the last applied index to:" + empty);
|
||||
TermIndex empty =
|
||||
TermIndex.newTermIndex(0, RaftServerConstants.INVALID_LOG_INDEX);
|
||||
LOG.info(
|
||||
"The snapshot info is null." + "Setting the last applied index to:"
|
||||
+ empty);
|
||||
setLastAppliedTermIndex(empty);
|
||||
lastIndex = RaftServerConstants.INVALID_LOG_INDEX;
|
||||
return RaftServerConstants.INVALID_LOG_INDEX;
|
||||
}
|
||||
|
||||
final File snapshotFile = snapshot.getFile().getPath().toFile();
|
||||
final TermIndex last =
|
||||
SimpleStateMachineStorage.getTermIndexFromSnapshotFile(
|
||||
snapshot.getFile().getPath().toFile());
|
||||
SimpleStateMachineStorage.getTermIndexFromSnapshotFile(snapshotFile);
|
||||
LOG.info("Setting the last applied index to " + last);
|
||||
setLastAppliedTermIndex(last);
|
||||
lastIndex = last.getIndex();
|
||||
|
||||
// initialize the dispatcher with snapshot so that it build the missing
|
||||
// container list
|
||||
try (FileInputStream fin = new FileInputStream(snapshotFile)) {
|
||||
byte[] containerIds = IOUtils.toByteArray(fin);
|
||||
ContainerProtos.ContainerIdSetProto proto =
|
||||
ContainerProtos.ContainerIdSetProto.parseFrom(containerIds);
|
||||
// read the created containers list from the snapshot file and add it to
|
||||
// the createContainerSet here.
|
||||
// createContainerSet will further grow as and when containers get created
|
||||
createContainerSet.addAll(proto.getContainerIdList());
|
||||
dispatcher.buildMissingContainerSet(createContainerSet);
|
||||
}
|
||||
return last.getIndex();
|
||||
}
|
||||
|
||||
/**
|
||||
* As a part of taking snapshot with Ratis StateMachine, it will persist
|
||||
* the existing container set in the snapshotFile.
|
||||
* @param out OutputStream mapped to the Ratis snapshot file
|
||||
* @throws IOException
|
||||
*/
|
||||
public void persistContainerSet(OutputStream out) throws IOException {
|
||||
ContainerIdSetProto.Builder builder = ContainerIdSetProto.newBuilder();
|
||||
builder.addAllContainerId(createContainerSet);
|
||||
// TODO : while snapshot is being taken, deleteContainer call should not
|
||||
// should not happen. Lock protection will be required if delete
|
||||
// container happens outside of Ratis.
|
||||
IOUtils.write(builder.build().toByteArray(), out);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long takeSnapshot() throws IOException {
|
||||
TermIndex ti = getLastAppliedTermIndex();
|
||||
LOG.info("Taking snapshot at termIndex:" + ti);
|
||||
if (ti != null) {
|
||||
if (ti != null && ti.getIndex() != RaftServerConstants.INVALID_LOG_INDEX) {
|
||||
final File snapshotFile =
|
||||
storage.getSnapshotFile(ti.getTerm(), ti.getIndex());
|
||||
LOG.info("Taking a snapshot to file {}", snapshotFile);
|
||||
try {
|
||||
//TODO: For now, just create the file to save the term index,
|
||||
//persist open container info to snapshot later.
|
||||
snapshotFile.createNewFile();
|
||||
try (FileOutputStream fos = new FileOutputStream(snapshotFile)) {
|
||||
persistContainerSet(fos);
|
||||
} catch (IOException ioe) {
|
||||
LOG.warn("Failed to write snapshot file \"" + snapshotFile
|
||||
+ "\", last applied index=" + ti);
|
||||
|
@ -227,7 +265,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
public TransactionContext startTransaction(RaftClientRequest request)
|
||||
throws IOException {
|
||||
final ContainerCommandRequestProto proto =
|
||||
getRequestProto(request.getMessage().getContent());
|
||||
getContainerCommandRequestProto(request.getMessage().getContent());
|
||||
Preconditions.checkArgument(request.getRaftGroupId().equals(gid));
|
||||
try (Scope scope = TracingUtil
|
||||
.importAndCreateScope(proto.getCmdType().name(), proto.getTraceID())) {
|
||||
|
@ -244,17 +282,6 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
}
|
||||
if (proto.getCmdType() == Type.WriteChunk) {
|
||||
final WriteChunkRequestProto write = proto.getWriteChunk();
|
||||
// create the state machine data proto
|
||||
final WriteChunkRequestProto dataWriteChunkProto =
|
||||
WriteChunkRequestProto
|
||||
.newBuilder(write)
|
||||
.build();
|
||||
ContainerCommandRequestProto dataContainerCommandProto =
|
||||
ContainerCommandRequestProto
|
||||
.newBuilder(proto)
|
||||
.setWriteChunk(dataWriteChunkProto)
|
||||
.build();
|
||||
|
||||
// create the log entry proto
|
||||
final WriteChunkRequestProto commitWriteChunkProto =
|
||||
WriteChunkRequestProto.newBuilder()
|
||||
|
@ -273,7 +300,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
.setClientRequest(request)
|
||||
.setStateMachine(this)
|
||||
.setServerRole(RaftPeerRole.LEADER)
|
||||
.setStateMachineData(dataContainerCommandProto.toByteString())
|
||||
.setStateMachineData(write.getData())
|
||||
.setLogData(commitContainerCommandProto.toByteString())
|
||||
.build();
|
||||
} else {
|
||||
|
@ -291,8 +318,8 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
return entryProto.getStateMachineEntry().getStateMachineData();
|
||||
}
|
||||
|
||||
private ContainerCommandRequestProto getRequestProto(ByteString request)
|
||||
throws InvalidProtocolBufferException {
|
||||
private ContainerCommandRequestProto getContainerCommandRequestProto(
|
||||
ByteString request) throws InvalidProtocolBufferException {
|
||||
// TODO: We can avoid creating new builder and set pipeline Id if
|
||||
// the client is already sending the pipeline id, then we just have to
|
||||
// validate the pipeline Id.
|
||||
|
@ -302,13 +329,22 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
}
|
||||
|
||||
private ContainerCommandResponseProto dispatchCommand(
|
||||
ContainerCommandRequestProto requestProto,
|
||||
DispatcherContext context) throws IOException {
|
||||
LOG.trace("dispatch {}", requestProto);
|
||||
ContainerCommandRequestProto requestProto, DispatcherContext context) {
|
||||
LOG.trace("dispatch {} containerID={} pipelineID={} traceID={}",
|
||||
requestProto.getCmdType(), requestProto.getContainerID(),
|
||||
requestProto.getPipelineID(), requestProto.getTraceID());
|
||||
if (isBlockTokenEnabled) {
|
||||
try {
|
||||
// ServerInterceptors intercepts incoming request and creates ugi.
|
||||
tokenVerifier.verify(UserGroupInformation.getCurrentUser()
|
||||
.getShortUserName(), requestProto.getEncodedToken());
|
||||
tokenVerifier
|
||||
.verify(UserGroupInformation.getCurrentUser().getShortUserName(),
|
||||
requestProto.getEncodedToken());
|
||||
} catch (IOException ioe) {
|
||||
StorageContainerException sce = new StorageContainerException(
|
||||
"Block token verification failed. " + ioe.getMessage(), ioe,
|
||||
ContainerProtos.Result.BLOCK_TOKEN_VERIFICATION_FAILED);
|
||||
return ContainerUtils.logAndReturnError(LOG, sce, requestProto);
|
||||
}
|
||||
}
|
||||
ContainerCommandResponseProto response =
|
||||
dispatcher.dispatch(requestProto, context);
|
||||
|
@ -317,7 +353,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
}
|
||||
|
||||
private Message runCommand(ContainerCommandRequestProto requestProto,
|
||||
DispatcherContext context) throws IOException {
|
||||
DispatcherContext context) {
|
||||
return dispatchCommand(requestProto, context)::toByteString;
|
||||
}
|
||||
|
||||
|
@ -344,15 +380,12 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
.setTerm(term)
|
||||
.setLogIndex(entryIndex)
|
||||
.setStage(DispatcherContext.WriteChunkStage.WRITE_DATA)
|
||||
.setCreateContainerSet(createContainerSet)
|
||||
.build();
|
||||
CompletableFuture<Message> writeChunkFuture;
|
||||
try {
|
||||
Message msg = runCommand(requestProto, context);
|
||||
writeChunkFuture = CompletableFuture
|
||||
.supplyAsync(() -> msg, chunkExecutor);
|
||||
}catch(IOException ie) {
|
||||
writeChunkFuture = completeExceptionally(ie);
|
||||
}
|
||||
// ensure the write chunk happens asynchronously in writeChunkExecutor pool
|
||||
// thread.
|
||||
CompletableFuture<Message> writeChunkFuture = CompletableFuture
|
||||
.supplyAsync(() -> runCommand(requestProto, context), chunkExecutor);
|
||||
|
||||
writeChunkFutureMap.put(entryIndex, writeChunkFuture);
|
||||
LOG.debug("writeChunk writeStateMachineData : blockId " + write.getBlockID()
|
||||
|
@ -378,8 +411,15 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
public CompletableFuture<Message> writeStateMachineData(LogEntryProto entry) {
|
||||
try {
|
||||
metrics.incNumWriteStateMachineOps();
|
||||
final ContainerCommandRequestProto requestProto =
|
||||
getRequestProto(getStateMachineData(entry.getStateMachineLogEntry()));
|
||||
ContainerCommandRequestProto requestProto =
|
||||
getContainerCommandRequestProto(
|
||||
entry.getStateMachineLogEntry().getLogData());
|
||||
WriteChunkRequestProto writeChunk =
|
||||
WriteChunkRequestProto.newBuilder(requestProto.getWriteChunk())
|
||||
.setData(getStateMachineData(entry.getStateMachineLogEntry()))
|
||||
.build();
|
||||
requestProto = ContainerCommandRequestProto.newBuilder(requestProto)
|
||||
.setWriteChunk(writeChunk).build();
|
||||
Type cmdType = requestProto.getCmdType();
|
||||
|
||||
// For only writeChunk, there will be writeStateMachineData call.
|
||||
|
@ -403,7 +443,7 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
try {
|
||||
metrics.incNumReadStateMachineOps();
|
||||
final ContainerCommandRequestProto requestProto =
|
||||
getRequestProto(request.getContent());
|
||||
getContainerCommandRequestProto(request.getContent());
|
||||
return CompletableFuture.completedFuture(runCommand(requestProto, null));
|
||||
} catch (IOException e) {
|
||||
metrics.incNumReadStateMachineFails();
|
||||
|
@ -453,34 +493,8 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
*/
|
||||
private ByteString getCachedStateMachineData(Long logIndex, long term,
|
||||
ContainerCommandRequestProto requestProto) throws ExecutionException {
|
||||
try {
|
||||
return reconstructWriteChunkRequest(
|
||||
stateMachineDataCache.get(logIndex, new Callable<ByteString>() {
|
||||
@Override
|
||||
public ByteString call() throws Exception {
|
||||
return readStateMachineData(requestProto, term, logIndex);
|
||||
}
|
||||
}), requestProto);
|
||||
} catch (ExecutionException e) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
private ByteString reconstructWriteChunkRequest(ByteString data,
|
||||
ContainerCommandRequestProto requestProto) {
|
||||
WriteChunkRequestProto writeChunkRequestProto =
|
||||
requestProto.getWriteChunk();
|
||||
// reconstruct the write chunk request
|
||||
final WriteChunkRequestProto.Builder dataWriteChunkProto =
|
||||
WriteChunkRequestProto.newBuilder(writeChunkRequestProto)
|
||||
// adding the state machine data
|
||||
.setData(data);
|
||||
|
||||
ContainerCommandRequestProto.Builder newStateMachineProto =
|
||||
ContainerCommandRequestProto.newBuilder(requestProto)
|
||||
.setWriteChunk(dataWriteChunkProto);
|
||||
|
||||
return newStateMachineProto.build().toByteString();
|
||||
return stateMachineDataCache.get(logIndex,
|
||||
() -> readStateMachineData(requestProto, term, logIndex));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -514,20 +528,23 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
}
|
||||
try {
|
||||
final ContainerCommandRequestProto requestProto =
|
||||
getRequestProto(entry.getStateMachineLogEntry().getLogData());
|
||||
getContainerCommandRequestProto(
|
||||
entry.getStateMachineLogEntry().getLogData());
|
||||
// readStateMachineData should only be called for "write" to Ratis.
|
||||
Preconditions.checkArgument(!HddsUtils.isReadOnly(requestProto));
|
||||
if (requestProto.getCmdType() == Type.WriteChunk) {
|
||||
CompletableFuture<ByteString> future = new CompletableFuture<>();
|
||||
return future.supplyAsync(() -> {
|
||||
final CompletableFuture<ByteString> future = new CompletableFuture<>();
|
||||
CompletableFuture.supplyAsync(() -> {
|
||||
try {
|
||||
return getCachedStateMachineData(entry.getIndex(), entry.getTerm(),
|
||||
requestProto);
|
||||
future.complete(
|
||||
getCachedStateMachineData(entry.getIndex(), entry.getTerm(),
|
||||
requestProto));
|
||||
} catch (ExecutionException e) {
|
||||
future.completeExceptionally(e);
|
||||
return null;
|
||||
}
|
||||
return future;
|
||||
}, chunkExecutor);
|
||||
return future;
|
||||
} else {
|
||||
throw new IllegalStateException("Cmd type:" + requestProto.getCmdType()
|
||||
+ " cannot have state machine data");
|
||||
|
@ -554,6 +571,18 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Notifies the state machine about index updates because of entries
|
||||
* which do not cause state machine update, i.e. conf entries, metadata
|
||||
* entries
|
||||
* @param term term of the log entry
|
||||
* @param index index of the log entry
|
||||
*/
|
||||
@Override
|
||||
public void notifyIndexUpdate(long term, long index) {
|
||||
applyTransactionCompletionMap.put(index, term);
|
||||
}
|
||||
|
||||
/*
|
||||
* ApplyTransaction calls in Ratis are sequential.
|
||||
*/
|
||||
|
@ -565,20 +594,12 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
.setTerm(trx.getLogEntry().getTerm())
|
||||
.setLogIndex(index);
|
||||
|
||||
// ApplyTransaction call can come with an entryIndex much greater than
|
||||
// lastIndex updated because in between entries in the raft log can be
|
||||
// appended because raft config persistence. Just add a dummy entry
|
||||
// for those.
|
||||
for (long i = lastIndex + 1; i < index; i++) {
|
||||
LOG.info("Gap in indexes at:{} detected, adding dummy entries ", i);
|
||||
applyTransactionCompletionMap.put(i, trx.getLogEntry().getTerm());
|
||||
}
|
||||
try {
|
||||
metrics.incNumApplyTransactionsOps();
|
||||
ContainerCommandRequestProto requestProto =
|
||||
getRequestProto(trx.getStateMachineLogEntry().getLogData());
|
||||
getContainerCommandRequestProto(
|
||||
trx.getStateMachineLogEntry().getLogData());
|
||||
Type cmdType = requestProto.getCmdType();
|
||||
CompletableFuture<Message> future;
|
||||
// Make sure that in write chunk, the user data is not set
|
||||
if (cmdType == Type.WriteChunk) {
|
||||
Preconditions
|
||||
|
@ -586,15 +607,15 @@ public class ContainerStateMachine extends BaseStateMachine {
|
|||
builder
|
||||
.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA);
|
||||
}
|
||||
try {
|
||||
Message msg = runCommand(requestProto, builder.build());
|
||||
future = CompletableFuture.supplyAsync(() -> msg,
|
||||
getCommandExecutor(requestProto));
|
||||
} catch (IOException ie) {
|
||||
future = completeExceptionally(ie);
|
||||
if (cmdType == Type.WriteChunk || cmdType ==Type.PutSmallFile) {
|
||||
builder.setCreateContainerSet(createContainerSet);
|
||||
}
|
||||
// Ensure the command gets executed in a separate thread than
|
||||
// stateMachineUpdater thread which is calling applyTransaction here.
|
||||
CompletableFuture<Message> future = CompletableFuture
|
||||
.supplyAsync(() -> runCommand(requestProto, builder.build()),
|
||||
getCommandExecutor(requestProto));
|
||||
|
||||
lastIndex = index;
|
||||
future.thenAccept(m -> {
|
||||
final Long previous =
|
||||
applyTransactionCompletionMap
|
||||
|
|
|
@ -20,6 +20,8 @@ package org.apache.hadoop.ozone.container.common.transport.server.ratis;
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* DispatcherContext class holds transport protocol specific context info
|
||||
* required for execution of container commands over the container dispatcher.
|
||||
|
@ -43,12 +45,15 @@ public final class DispatcherContext {
|
|||
// the log index in Ratis log to which the request belongs to
|
||||
private final long logIndex;
|
||||
|
||||
private final Set<Long> createContainerSet;
|
||||
|
||||
private DispatcherContext(long term, long index, WriteChunkStage stage,
|
||||
boolean readFromTmpFile) {
|
||||
boolean readFromTmpFile, Set<Long> containerSet) {
|
||||
this.term = term;
|
||||
this.logIndex = index;
|
||||
this.stage = stage;
|
||||
this.readFromTmpFile = readFromTmpFile;
|
||||
this.createContainerSet = containerSet;
|
||||
}
|
||||
|
||||
public long getLogIndex() {
|
||||
|
@ -67,6 +72,10 @@ public final class DispatcherContext {
|
|||
return stage;
|
||||
}
|
||||
|
||||
public Set<Long> getCreateContainerSet() {
|
||||
return createContainerSet;
|
||||
}
|
||||
|
||||
/**
|
||||
* Builder class for building DispatcherContext.
|
||||
*/
|
||||
|
@ -75,11 +84,12 @@ public final class DispatcherContext {
|
|||
private boolean readFromTmpFile = false;
|
||||
private long term;
|
||||
private long logIndex;
|
||||
private Set<Long> createContainerSet;
|
||||
|
||||
/**
|
||||
* Sets the WriteChunkStage.
|
||||
*
|
||||
* @param stage WriteChunk Stage
|
||||
* @param writeChunkStage WriteChunk Stage
|
||||
* @return DispatcherContext.Builder
|
||||
*/
|
||||
public Builder setStage(WriteChunkStage writeChunkStage) {
|
||||
|
@ -90,7 +100,7 @@ public final class DispatcherContext {
|
|||
/**
|
||||
* Sets the flag for reading from tmp chunk files.
|
||||
*
|
||||
* @param readFromTmpFile whether to read from tmp chunk file or not
|
||||
* @param setReadFromTmpFile whether to read from tmp chunk file or not
|
||||
* @return DispatcherContext.Builder
|
||||
*/
|
||||
public Builder setReadFromTmpFile(boolean setReadFromTmpFile) {
|
||||
|
@ -101,7 +111,7 @@ public final class DispatcherContext {
|
|||
/**
|
||||
* Sets the current term for the container request from Ratis.
|
||||
*
|
||||
* @param term current term
|
||||
* @param currentTerm current term
|
||||
* @return DispatcherContext.Builder
|
||||
*/
|
||||
public Builder setTerm(long currentTerm) {
|
||||
|
@ -112,7 +122,7 @@ public final class DispatcherContext {
|
|||
/**
|
||||
* Sets the logIndex for the container request from Ratis.
|
||||
*
|
||||
* @param logIndex log index
|
||||
* @param index log index
|
||||
* @return DispatcherContext.Builder
|
||||
*/
|
||||
public Builder setLogIndex(long index) {
|
||||
|
@ -120,13 +130,24 @@ public final class DispatcherContext {
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the createContainerSet to contain all the containerIds per
|
||||
* RaftGroup.
|
||||
* @param set createContainerSet
|
||||
* @return Builder
|
||||
*/
|
||||
public Builder setCreateContainerSet(Set<Long> set) {
|
||||
this.createContainerSet = set;
|
||||
return this;
|
||||
}
|
||||
/**
|
||||
* Builds and returns DispatcherContext instance.
|
||||
*
|
||||
* @return DispatcherContext
|
||||
*/
|
||||
public DispatcherContext build() {
|
||||
return new DispatcherContext(term, logIndex, stage, readFromTmpFile);
|
||||
return new DispatcherContext(term, logIndex, stage, readFromTmpFile,
|
||||
createContainerSet);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
|||
import org.apache.hadoop.hdds.scm.HddsServerUtil;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.hdds.tracing.TracingUtil;
|
||||
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
||||
import org.apache.hadoop.ozone.OzoneConsts;
|
||||
|
@ -44,7 +45,6 @@ import org.apache.hadoop.ozone.container.common.transport.server.XceiverServer;
|
|||
import io.opentracing.Scope;
|
||||
import org.apache.ratis.RaftConfigKeys;
|
||||
import org.apache.ratis.RatisHelper;
|
||||
import org.apache.ratis.client.RaftClientConfigKeys;
|
||||
import org.apache.ratis.conf.RaftProperties;
|
||||
import org.apache.ratis.grpc.GrpcConfigKeys;
|
||||
import org.apache.ratis.grpc.GrpcFactory;
|
||||
|
@ -114,9 +114,9 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
|
||||
private XceiverServerRatis(DatanodeDetails dd, int port,
|
||||
ContainerDispatcher dispatcher, Configuration conf, StateContext
|
||||
context, GrpcTlsConfig tlsConfig)
|
||||
context, GrpcTlsConfig tlsConfig, CertificateClient caClient)
|
||||
throws IOException {
|
||||
super(conf);
|
||||
super(conf, caClient);
|
||||
Objects.requireNonNull(dd, "id == null");
|
||||
this.port = port;
|
||||
RaftProperties serverProperties = newRaftProperties(conf);
|
||||
|
@ -176,7 +176,7 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
setRaftSegmentPreallocatedSize(conf, properties);
|
||||
|
||||
// Set max write buffer size, which is the scm chunk size
|
||||
final int maxChunkSize = setMaxWriteBuffer(conf, properties);
|
||||
final int maxChunkSize = setMaxWriteBuffer(properties);
|
||||
TimeUnit timeUnit;
|
||||
long duration;
|
||||
|
||||
|
@ -329,23 +329,10 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
.setRequestTimeout(properties, serverRequestTimeout);
|
||||
}
|
||||
|
||||
private int setMaxWriteBuffer(Configuration conf, RaftProperties properties) {
|
||||
private int setMaxWriteBuffer(RaftProperties properties) {
|
||||
final int maxChunkSize = OzoneConsts.OZONE_SCM_CHUNK_MAX_SIZE;
|
||||
RaftServerConfigKeys.Log.setWriteBufferSize(properties,
|
||||
SizeInBytes.valueOf(maxChunkSize));
|
||||
|
||||
// Set the client requestTimeout
|
||||
TimeUnit timeUnit =
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
|
||||
.getUnit();
|
||||
long duration = conf.getTimeDuration(
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_KEY,
|
||||
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_TIMEOUT_DURATION_DEFAULT
|
||||
.getDuration(), timeUnit);
|
||||
final TimeDuration clientRequestTimeout =
|
||||
TimeDuration.valueOf(duration, timeUnit);
|
||||
RaftClientConfigKeys.Rpc
|
||||
.setRequestTimeout(properties, clientRequestTimeout);
|
||||
return maxChunkSize;
|
||||
}
|
||||
|
||||
|
@ -394,7 +381,8 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
|
||||
public static XceiverServerRatis newXceiverServerRatis(
|
||||
DatanodeDetails datanodeDetails, Configuration ozoneConf,
|
||||
ContainerDispatcher dispatcher, StateContext context) throws IOException {
|
||||
ContainerDispatcher dispatcher, StateContext context,
|
||||
CertificateClient caClient) throws IOException {
|
||||
int localPort = ozoneConf.getInt(
|
||||
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT,
|
||||
OzoneConfigKeys.DFS_CONTAINER_RATIS_IPC_PORT_DEFAULT);
|
||||
|
@ -420,7 +408,7 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
datanodeDetails.setPort(
|
||||
DatanodeDetails.newPort(DatanodeDetails.Port.Name.RATIS, localPort));
|
||||
return new XceiverServerRatis(datanodeDetails, localPort,
|
||||
dispatcher, ozoneConf, context, tlsConfig);
|
||||
dispatcher, ozoneConf, context, tlsConfig, caClient);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -506,7 +494,8 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
RaftClientRequest.Type type) {
|
||||
return new RaftClientRequest(clientId, server.getId(),
|
||||
RaftGroupId.valueOf(PipelineID.getFromProtobuf(pipelineID).getId()),
|
||||
nextCallId(), 0, Message.valueOf(request.toByteString()), type);
|
||||
nextCallId(), Message.valueOf(request.toByteString()), type,
|
||||
null);
|
||||
}
|
||||
|
||||
private void handlePipelineFailure(RaftGroupId groupId,
|
||||
|
@ -582,6 +571,17 @@ public final class XceiverServerRatis extends XceiverServer {
|
|||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public List<PipelineID> getPipelineIds() {
|
||||
Iterable<RaftGroupId> gids = server.getGroupIds();
|
||||
List<PipelineID> pipelineIDs = new ArrayList<>();
|
||||
for (RaftGroupId groupId : gids) {
|
||||
pipelineIDs.add(PipelineID.valueOf(groupId.getUuid()));
|
||||
LOG.info("pipeline id {}", PipelineID.valueOf(groupId.getUuid()));
|
||||
}
|
||||
return pipelineIDs;
|
||||
}
|
||||
|
||||
void handleNodeSlowness(RaftGroup group, RoleInfoProto roleInfoProto) {
|
||||
handlePipelineFailure(group.getGroupId(), roleInfoProto);
|
||||
}
|
||||
|
|
|
@ -618,6 +618,9 @@ public class KeyValueContainer implements Container<KeyValueContainerData> {
|
|||
case CLOSED:
|
||||
state = ContainerReplicaProto.State.CLOSED;
|
||||
break;
|
||||
case UNHEALTHY:
|
||||
state = ContainerReplicaProto.State.UNHEALTHY;
|
||||
break;
|
||||
default:
|
||||
throw new StorageContainerException("Invalid Container state found: " +
|
||||
containerData.getContainerID(), INVALID_CONTAINER_STATE);
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.hdds.protocol.proto
|
|||
.StorageContainerDatanodeProtocolProtos;
|
||||
import org.apache.hadoop.hdds.protocol.proto
|
||||
.StorageContainerDatanodeProtocolProtos.PipelineReportsProto;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
|
||||
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
|
||||
import org.apache.hadoop.ozone.container.common.impl.HddsDispatcher;
|
||||
|
@ -74,11 +75,13 @@ public class OzoneContainer {
|
|||
* Construct OzoneContainer object.
|
||||
* @param datanodeDetails
|
||||
* @param conf
|
||||
* @param certClient
|
||||
* @throws DiskOutOfSpaceException
|
||||
* @throws IOException
|
||||
*/
|
||||
public OzoneContainer(DatanodeDetails datanodeDetails, OzoneConfiguration
|
||||
conf, StateContext context) throws IOException {
|
||||
conf, StateContext context, CertificateClient certClient)
|
||||
throws IOException {
|
||||
this.config = conf;
|
||||
this.volumeSet = new VolumeSet(datanodeDetails.getUuidString(), conf);
|
||||
this.containerSet = new ContainerSet();
|
||||
|
@ -100,9 +103,10 @@ public class OzoneContainer {
|
|||
*/
|
||||
this.controller = new ContainerController(containerSet, handlers);
|
||||
this.writeChannel = XceiverServerRatis.newXceiverServerRatis(
|
||||
datanodeDetails, config, hddsDispatcher, context);
|
||||
datanodeDetails, config, hddsDispatcher, context, certClient);
|
||||
this.readChannel = new XceiverServerGrpc(
|
||||
datanodeDetails, config, hddsDispatcher, createReplicationService());
|
||||
datanodeDetails, config, hddsDispatcher, certClient,
|
||||
createReplicationService());
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -95,13 +95,12 @@ public class DownloadAndImportReplicator implements ContainerReplicator {
|
|||
LOG.error(
|
||||
"Can't import the downloaded container data id=" + containerID,
|
||||
e);
|
||||
} finally {
|
||||
try {
|
||||
Files.delete(tarFilePath);
|
||||
} catch (Exception ex) {
|
||||
LOG.error(
|
||||
"Container import is failed and the downloaded file can't be "
|
||||
+ "deleted: "
|
||||
+ tarFilePath.toAbsolutePath().toString());
|
||||
LOG.error("Got exception while deleting downloaded container file: "
|
||||
+ tarFilePath.toAbsolutePath().toString(), ex);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,17 @@
|
|||
/**
|
||||
* 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.
|
||||
*/
|
|
@ -0,0 +1,271 @@
|
|||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with this
|
||||
* work for additional information regarding copyright ownership. The ASF
|
||||
* licenses this file to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
|
||||
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
|
||||
* License for the specific language governing permissions and limitations under
|
||||
* the License.
|
||||
*/
|
||||
package org.apache.hadoop.ozone;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.hdds.HddsConfigKeys;
|
||||
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
|
||||
import org.apache.hadoop.hdds.security.x509.SecurityConfig;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.client.DNCertificateClient;
|
||||
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
|
||||
import org.apache.hadoop.hdds.security.x509.keys.KeyCodec;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.security.ssl.KeyStoreTestUtil;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
import org.apache.hadoop.util.ServicePlugin;
|
||||
import org.bouncycastle.cert.X509CertificateHolder;
|
||||
import org.bouncycastle.pkcs.PKCS10CertificationRequest;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
import org.junit.Test;
|
||||
|
||||
import java.io.File;
|
||||
import java.nio.file.Paths;
|
||||
import java.security.KeyPair;
|
||||
import java.security.PrivateKey;
|
||||
import java.security.PublicKey;
|
||||
import java.security.cert.X509Certificate;
|
||||
import java.util.concurrent.Callable;
|
||||
|
||||
import static org.apache.hadoop.ozone.HddsDatanodeService.getLogger;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_SECURITY_ENABLED_KEY;
|
||||
|
||||
/**
|
||||
* Test class for {@link HddsDatanodeService}.
|
||||
*/
|
||||
public class TestHddsSecureDatanodeInit {
|
||||
|
||||
private static File testDir;
|
||||
private static OzoneConfiguration conf;
|
||||
private static HddsDatanodeService service;
|
||||
private static String[] args = new String[]{};
|
||||
private static PrivateKey privateKey;
|
||||
private static PublicKey publicKey;
|
||||
private static GenericTestUtils.LogCapturer dnLogs;
|
||||
private static CertificateClient client;
|
||||
private static SecurityConfig securityConfig;
|
||||
private static KeyCodec keyCodec;
|
||||
private static CertificateCodec certCodec;
|
||||
private static X509CertificateHolder certHolder;
|
||||
|
||||
@BeforeClass
|
||||
public static void setUp() throws Exception {
|
||||
testDir = GenericTestUtils.getRandomizedTestDir();
|
||||
conf = new OzoneConfiguration();
|
||||
conf.setBoolean(OzoneConfigKeys.OZONE_ENABLED, true);
|
||||
conf.set(HddsConfigKeys.OZONE_METADATA_DIRS, testDir.getPath());
|
||||
//conf.set(ScmConfigKeys.OZONE_SCM_NAMES, "localhost");
|
||||
String volumeDir = testDir + "/disk1";
|
||||
conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, volumeDir);
|
||||
|
||||
conf.setBoolean(OZONE_SECURITY_ENABLED_KEY, true);
|
||||
conf.setClass(OzoneConfigKeys.HDDS_DATANODE_PLUGINS_KEY,
|
||||
TestHddsDatanodeService.MockService.class,
|
||||
ServicePlugin.class);
|
||||
securityConfig = new SecurityConfig(conf);
|
||||
|
||||
service = HddsDatanodeService.createHddsDatanodeService(args, conf);
|
||||
dnLogs = GenericTestUtils.LogCapturer.captureLogs(getLogger());
|
||||
callQuietly(() -> {
|
||||
service.start(null);
|
||||
return null;
|
||||
});
|
||||
callQuietly(() -> {
|
||||
service.initializeCertificateClient(conf);
|
||||
return null;
|
||||
});
|
||||
certCodec = new CertificateCodec(securityConfig);
|
||||
keyCodec = new KeyCodec(securityConfig);
|
||||
dnLogs.clearOutput();
|
||||
privateKey = service.getCertificateClient().getPrivateKey();
|
||||
publicKey = service.getCertificateClient().getPublicKey();
|
||||
X509Certificate x509Certificate = null;
|
||||
|
||||
x509Certificate = KeyStoreTestUtil.generateCertificate(
|
||||
"CN=Test", new KeyPair(publicKey, privateKey), 10,
|
||||
securityConfig.getSignatureAlgo());
|
||||
certHolder = new X509CertificateHolder(x509Certificate.getEncoded());
|
||||
|
||||
}
|
||||
|
||||
@AfterClass
|
||||
public static void tearDown() {
|
||||
FileUtil.fullyDelete(testDir);
|
||||
}
|
||||
|
||||
@Before
|
||||
public void setUpDNCertClient(){
|
||||
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPrivateKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig.getKeyLocation()
|
||||
.toString(), securityConfig.getPublicKeyFileName()).toFile());
|
||||
FileUtils.deleteQuietly(Paths.get(securityConfig
|
||||
.getCertificateLocation().toString(),
|
||||
securityConfig.getCertificateFileName()).toFile());
|
||||
dnLogs.clearOutput();
|
||||
client = new DNCertificateClient(securityConfig,
|
||||
certHolder.getSerialNumber().toString());
|
||||
service.setCertificateClient(client);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase0() throws Exception {
|
||||
|
||||
// Case 0: When keypair as well as certificate is missing. Initial keypair
|
||||
// boot-up. Get certificate will fail as no SCM is not running.
|
||||
LambdaTestUtils.intercept(Exception.class, "",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
|
||||
Assert.assertNotNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: GETCERT"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase1() throws Exception {
|
||||
// Case 1: When only certificate is present.
|
||||
|
||||
certCodec.writeCertificate(certHolder);
|
||||
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||
" initialization failed",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
Assert.assertNull(client.getPrivateKey());
|
||||
Assert.assertNull(client.getPublicKey());
|
||||
Assert.assertNotNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase2() throws Exception {
|
||||
// Case 2: When private key and certificate is missing.
|
||||
keyCodec.writePublicKey(publicKey);
|
||||
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||
" initialization failed",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
Assert.assertNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase3() throws Exception {
|
||||
// Case 3: When only public key and certificate is present.
|
||||
keyCodec.writePublicKey(publicKey);
|
||||
certCodec.writeCertificate(certHolder);
|
||||
LambdaTestUtils.intercept(RuntimeException.class, "DN security" +
|
||||
" initialization failed",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
Assert.assertNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNotNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase4() throws Exception {
|
||||
// Case 4: When public key as well as certificate is missing.
|
||||
keyCodec.writePrivateKey(privateKey);
|
||||
LambdaTestUtils.intercept(RuntimeException.class, " DN security" +
|
||||
" initialization failed",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
Assert.assertNotNull(client.getPrivateKey());
|
||||
Assert.assertNull(client.getPublicKey());
|
||||
Assert.assertNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: FAILURE"));
|
||||
dnLogs.clearOutput();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase5() throws Exception {
|
||||
// Case 5: If private key and certificate is present.
|
||||
certCodec.writeCertificate(certHolder);
|
||||
keyCodec.writePrivateKey(privateKey);
|
||||
service.initializeCertificateClient(conf);
|
||||
Assert.assertNotNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNotNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: SUCCESS"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase6() throws Exception {
|
||||
// Case 6: If key pair already exist than response should be GETCERT.
|
||||
keyCodec.writePublicKey(publicKey);
|
||||
keyCodec.writePrivateKey(privateKey);
|
||||
LambdaTestUtils.intercept(Exception.class, "",
|
||||
() -> service.initializeCertificateClient(conf));
|
||||
Assert.assertNotNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: GETCERT"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSecureDnStartupCase7() throws Exception {
|
||||
// Case 7 When keypair and certificate is present.
|
||||
keyCodec.writePublicKey(publicKey);
|
||||
keyCodec.writePrivateKey(privateKey);
|
||||
certCodec.writeCertificate(certHolder);
|
||||
|
||||
service.initializeCertificateClient(conf);
|
||||
Assert.assertNotNull(client.getPrivateKey());
|
||||
Assert.assertNotNull(client.getPublicKey());
|
||||
Assert.assertNotNull(client.getCertificate());
|
||||
Assert.assertTrue(dnLogs.getOutput().contains("Init response: SUCCESS"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Invoke a callable; Ignore all exception.
|
||||
* @param closure closure to execute
|
||||
* @return
|
||||
*/
|
||||
public static void callQuietly(Callable closure) {
|
||||
try {
|
||||
closure.call();
|
||||
} catch (Throwable e) {
|
||||
// Ignore all Throwable,
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetCSR() throws Exception {
|
||||
keyCodec.writePublicKey(publicKey);
|
||||
keyCodec.writePrivateKey(privateKey);
|
||||
service.setCertificateClient(client);
|
||||
PKCS10CertificationRequest csr =
|
||||
service.getCSR(conf);
|
||||
Assert.assertNotNull(csr);
|
||||
|
||||
csr = service.getCSR(conf);
|
||||
Assert.assertNotNull(csr);
|
||||
|
||||
csr = service.getCSR(conf);
|
||||
Assert.assertNotNull(csr);
|
||||
|
||||
csr = service.getCSR(conf);
|
||||
Assert.assertNotNull(csr);
|
||||
}
|
||||
|
||||
}
|
|
@ -161,7 +161,7 @@ public class TestDatanodeStateMachine {
|
|||
public void testStartStopDatanodeStateMachine() throws IOException,
|
||||
InterruptedException, TimeoutException {
|
||||
try (DatanodeStateMachine stateMachine =
|
||||
new DatanodeStateMachine(getNewDatanodeDetails(), conf)) {
|
||||
new DatanodeStateMachine(getNewDatanodeDetails(), conf, null)) {
|
||||
stateMachine.startDaemon();
|
||||
SCMConnectionManager connectionManager =
|
||||
stateMachine.getConnectionManager();
|
||||
|
@ -219,7 +219,7 @@ public class TestDatanodeStateMachine {
|
|||
ContainerUtils.writeDatanodeDetailsTo(datanodeDetails, idPath);
|
||||
|
||||
try (DatanodeStateMachine stateMachine =
|
||||
new DatanodeStateMachine(datanodeDetails, conf)) {
|
||||
new DatanodeStateMachine(datanodeDetails, conf, null)) {
|
||||
DatanodeStateMachine.DatanodeStates currentState =
|
||||
stateMachine.getContext().getState();
|
||||
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
|
||||
|
@ -325,7 +325,7 @@ public class TestDatanodeStateMachine {
|
|||
datanodeDetails.setPort(port);
|
||||
|
||||
try (DatanodeStateMachine stateMachine =
|
||||
new DatanodeStateMachine(datanodeDetails, conf)) {
|
||||
new DatanodeStateMachine(datanodeDetails, conf, null)) {
|
||||
DatanodeStateMachine.DatanodeStates currentState =
|
||||
stateMachine.getContext().getState();
|
||||
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
|
||||
|
@ -388,7 +388,7 @@ public class TestDatanodeStateMachine {
|
|||
perTestConf.setStrings(entry.getKey(), entry.getValue());
|
||||
LOG.info("Test with {} = {}", entry.getKey(), entry.getValue());
|
||||
try (DatanodeStateMachine stateMachine = new DatanodeStateMachine(
|
||||
getNewDatanodeDetails(), perTestConf)) {
|
||||
getNewDatanodeDetails(), perTestConf, null)) {
|
||||
DatanodeStateMachine.DatanodeStates currentState =
|
||||
stateMachine.getContext().getState();
|
||||
Assert.assertEquals(DatanodeStateMachine.DatanodeStates.INIT,
|
||||
|
|
|
@ -39,6 +39,7 @@ 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.junit.AfterClass;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
@ -49,6 +50,7 @@ import java.io.IOException;
|
|||
import java.util.Collections;
|
||||
import java.util.Random;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
/**
|
||||
* Test cases to verify CloseContainerCommandHandler in datanode.
|
||||
|
@ -274,7 +276,7 @@ public class TestCloseContainerCommandHandler {
|
|||
.thenReturn(datanodeDetails);
|
||||
Mockito.when(context.getParent()).thenReturn(datanodeStateMachine);
|
||||
final OzoneContainer ozoneContainer = new OzoneContainer(
|
||||
datanodeDetails, conf, context);
|
||||
datanodeDetails, conf, context, null);
|
||||
ozoneContainer.getDispatcher().setScmId(UUID.randomUUID().toString());
|
||||
return ozoneContainer;
|
||||
}
|
||||
|
@ -289,8 +291,10 @@ public class TestCloseContainerCommandHandler {
|
|||
final RaftGroup group = RatisHelper.newRaftGroup(raftGroupId,
|
||||
Collections.singleton(datanodeDetails));
|
||||
final int maxOutstandingRequests = 100;
|
||||
final RaftClient client = RatisHelper.newRaftClient(SupportedRpcType.GRPC,
|
||||
peer, retryPolicy, maxOutstandingRequests, null);
|
||||
final RaftClient client = RatisHelper
|
||||
.newRaftClient(SupportedRpcType.GRPC, peer, retryPolicy,
|
||||
maxOutstandingRequests,
|
||||
TimeDuration.valueOf(3, TimeUnit.SECONDS));
|
||||
Assert.assertTrue(client.groupAdd(group, peer.getId()).isSuccess());
|
||||
Thread.sleep(2000);
|
||||
final ContainerID containerId = ContainerID.valueof(
|
||||
|
|
|
@ -332,6 +332,19 @@ public class TestKeyValueContainer {
|
|||
keyValueContainerData.getState());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReportOfUnhealthyContainer() throws Exception {
|
||||
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
||||
Assert.assertNotNull(keyValueContainer.getContainerReport());
|
||||
keyValueContainer.markContainerUnhealthy();
|
||||
File containerFile = keyValueContainer.getContainerFile();
|
||||
keyValueContainerData = (KeyValueContainerData) ContainerDataYaml
|
||||
.readContainerFile(containerFile);
|
||||
assertEquals(ContainerProtos.ContainerDataProto.State.UNHEALTHY,
|
||||
keyValueContainerData.getState());
|
||||
Assert.assertNotNull(keyValueContainer.getContainerReport());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testUpdateContainer() throws IOException {
|
||||
keyValueContainer.create(volumeSet, volumeChoosingPolicy, scmId);
|
||||
|
|
|
@ -59,6 +59,7 @@ import static org.mockito.Mockito.times;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.HashSet;
|
||||
import java.util.UUID;
|
||||
|
||||
/**
|
||||
|
@ -88,6 +89,8 @@ public class TestKeyValueHandler {
|
|||
Mockito.when(dispatcher.dispatch(any(), any())).thenCallRealMethod();
|
||||
Mockito.when(dispatcher.getContainer(anyLong())).thenReturn(
|
||||
Mockito.mock(KeyValueContainer.class));
|
||||
Mockito.when(dispatcher.getMissingContainerSet())
|
||||
.thenReturn(new HashSet<>());
|
||||
Mockito.when(handler.handle(any(), any(), any())).thenCallRealMethod();
|
||||
doCallRealMethod().when(dispatcher).setMetricsForTesting(any());
|
||||
dispatcher.setMetricsForTesting(Mockito.mock(ContainerMetrics.class));
|
||||
|
|
|
@ -98,7 +98,7 @@ public class TestOzoneContainer {
|
|||
// When OzoneContainer is started, the containers from disk should be
|
||||
// loaded into the containerSet.
|
||||
OzoneContainer ozoneContainer = new
|
||||
OzoneContainer(datanodeDetails, conf, context);
|
||||
OzoneContainer(datanodeDetails, conf, context, null);
|
||||
ContainerSet containerset = ozoneContainer.getContainerSet();
|
||||
assertEquals(10, containerset.containerCount());
|
||||
}
|
||||
|
|
|
@ -2,8 +2,8 @@
|
|||
title: Bucket Commands
|
||||
menu:
|
||||
main:
|
||||
parent: Client
|
||||
weight: 3
|
||||
parent: OzoneShell
|
||||
weight: 2
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -122,7 +122,7 @@ The above command gives user bilbo read/write permission to the bucket.
|
|||
The bucket command to provide ozone mapping for s3 bucket (Created via aws cli)
|
||||
|
||||
{{< highlight bash >}}
|
||||
ozone sh bucket path <<s3Bucket>>
|
||||
ozone s3 path <<s3Bucket>>
|
||||
{{< /highlight >}}
|
||||
|
||||
The above command will print VolumeName and the mapping created for s3Bucket.
|
||||
|
|
|
@ -4,7 +4,7 @@ weight: 1
|
|||
menu:
|
||||
main:
|
||||
parent: Starting
|
||||
weight: 5
|
||||
weight: 6
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -35,12 +35,11 @@ the ozone build command. This instruction assumes that you have all the
|
|||
dependencies to build Hadoop on your build machine. If you need instructions
|
||||
on how to build Hadoop, please look at the Apache Hadoop Website.
|
||||
|
||||
{{< highlight bash >}}
|
||||
mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true -Phdds -Pdist -Dtar -DskipShade
|
||||
{{< /highlight >}}
|
||||
```bash
|
||||
mvn -f pom.ozone.xml clean package -DskipTests=true
|
||||
```
|
||||
|
||||
|
||||
This will build an ozone-\<version\>.tar.gz in your target directory.
|
||||
This will build an ozone-\<version\>.tar.gz in your `hadoop-ozone/dist/target` directory.
|
||||
|
||||
You can copy this tarball and use this instead of binary artifacts that are
|
||||
provided along with the official release.
|
||||
|
@ -48,7 +47,20 @@ provided along with the official release.
|
|||
## How to test the build
|
||||
You can run the acceptance tests in the hadoop-ozone directory to make sure
|
||||
that your build is functional. To launch the acceptance tests, please follow
|
||||
the instructions in the **README.md** in the
|
||||
```$hadoop_src/hadoop-ozone/acceptance-test``` directory. Acceptance tests
|
||||
the instructions in the **README.md** in the `smoketest` directory.
|
||||
|
||||
```bash
|
||||
cd smoketest
|
||||
./test.sh
|
||||
```
|
||||
|
||||
You can also execute only a minimal subset of the tests:
|
||||
|
||||
```bash
|
||||
cd smoketest
|
||||
./test.sh --env ozone basic
|
||||
```
|
||||
|
||||
Acceptance tests
|
||||
will start a small ozone cluster and verify that ozone shell and ozone file
|
||||
system is fully functional.
|
|
@ -4,6 +4,7 @@ menu:
|
|||
main:
|
||||
parent: Client
|
||||
weight: 1
|
||||
identifier: OzoneShell
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -30,6 +31,7 @@ The commands supported by ozone are:
|
|||
|
||||
* **classpath** - Prints the class path needed to get the hadoop jar and the
|
||||
required libraries.
|
||||
* **dtutil** - Operations related to delegation tokens
|
||||
* **fs** - Runs a command on ozone file system.
|
||||
* **datanode** - Via daemon command, the HDDS data nodes can be started or
|
||||
stopped.
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
title: Key Commands
|
||||
menu:
|
||||
main:
|
||||
parent: Client
|
||||
parent: OzoneShell
|
||||
weight: 3
|
||||
---
|
||||
<!---
|
||||
|
|
|
@ -1,10 +1,12 @@
|
|||
---
|
||||
title: Ozone File System
|
||||
weight: 1
|
||||
date: 2017-09-14
|
||||
menu: main
|
||||
menu:
|
||||
main:
|
||||
parent: Client
|
||||
parent: Starting
|
||||
weight: 4
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
@ -51,16 +53,14 @@ Please add the following entry to the core-site.xml.
|
|||
</property>
|
||||
{{< /highlight >}}
|
||||
|
||||
This will make this bucket to be the default file system for HDFS dfs commands and register the o3fs file system type..
|
||||
This will make this bucket to be the default file system for HDFS dfs commands and register the o3fs file system type.
|
||||
|
||||
You also need to add the ozone-filesystem.jar file to the classpath:
|
||||
|
||||
{{< highlight bash >}}
|
||||
export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-.*.jar:$HADOOP_CLASSPATH
|
||||
export HADOOP_CLASSPATH=/opt/ozone/share/ozonefs/lib/hadoop-ozone-filesystem-lib-current*.jar:$HADOOP_CLASSPATH
|
||||
{{< /highlight >}}
|
||||
|
||||
|
||||
|
||||
Once the default Filesystem has been setup, users can run commands like ls, put, mkdir, etc.
|
||||
For example,
|
||||
|
||||
|
@ -76,20 +76,52 @@ hdfs dfs -mkdir /users
|
|||
|
||||
|
||||
Or put command etc. In other words, all programs like Hive, Spark, and Distcp will work against this file system.
|
||||
Please note that any keys created/deleted in the bucket using methods apart from OzoneFileSystem will show up as diectories and files in the Ozone File System.
|
||||
Please note that any keys created/deleted in the bucket using methods apart from OzoneFileSystem will show up as directories and files in the Ozone File System.
|
||||
|
||||
## Legacy mode
|
||||
Note: Bucket and volume names are not allowed to have a period in them.
|
||||
Moreover, the filesystem URI can take a fully qualified form with the OM host and port as a part of the path following the volume name.
|
||||
For example,
|
||||
|
||||
There are two ozonefs files which includes all the dependencies:
|
||||
{{< highlight bash>}}
|
||||
hdfs dfs -ls o3fs://bucket.volume.om-host.example.com:5678/key
|
||||
{{< /highlight >}}
|
||||
|
||||
* share/ozone/lib/hadoop-ozone-filesystem-lib-VERSION.jar
|
||||
|
||||
## Supporting older Hadoop version (Legacy jar, BasicOzoneFilesystem)
|
||||
|
||||
There are two ozonefs files, both of them include all the dependencies:
|
||||
|
||||
* share/ozone/lib/hadoop-ozone-filesystem-lib-current-VERSION.jar
|
||||
* share/ozone/lib/hadoop-ozone-filesystem-lib-legacy-VERSION.jar
|
||||
|
||||
The first one contains all the required dependency to use ozonefs with a
|
||||
compatible hadoop version (hadoop 3.2 / 3.1).
|
||||
compatible hadoop version (hadoop 3.2).
|
||||
|
||||
The second one contains all the dependency in an internal, separated directory,
|
||||
and a special class loader is used to load all the classes from the location.
|
||||
|
||||
With this method the hadoop-ozone-filesystem-lib-legacy.jar can be used from
|
||||
any older hadoop version (eg. hadoop 3.1, hadoop 2.7 or spark+hadoop 2.7)
|
||||
|
||||
Similar to the dependency jar, there are two OzoneFileSystem implementation.
|
||||
|
||||
For hadoop 3.0 and newer, you can use `org.apache.hadoop.fs.ozone.OzoneFileSystem`
|
||||
which is a full implementation of the Hadoop compatible File System API.
|
||||
|
||||
For Hadoop 2.x you should use the Basic version: `org.apache.hadoop.fs.ozone.BasicOzoneFileSystem`.
|
||||
|
||||
This is the same implementation but doesn't include the features/dependencies which are added with
|
||||
Hadoop 3.0. (eg. FS statistics, encryption zones).
|
||||
|
||||
### Summary
|
||||
|
||||
The following table summarize which jar files and implementation should be used:
|
||||
|
||||
Hadoop version | Required jar | OzoneFileSystem implementation
|
||||
---------------|-------------------------|----------------------------------------------------
|
||||
3.2 | filesystem-lib-current | org.apache.hadoop.fs.ozone.OzoneFileSystem
|
||||
3.1 | filesystem-lib-legacy | org.apache.hadoop.fs.ozone.OzoneFileSystem
|
||||
2.9 | filesystem-lib-legacy | org.apache.hadoop.fs.ozone.BasicOzoneFileSystem
|
||||
2.7 | filesystem-lib-legacy | org.apache.hadoop.fs.ozone.BasicOzoneFileSystem
|
||||
With this method the hadoop-ozone-filesystem-lib-legacy.jar can be used from
|
||||
any older hadoop version (eg. hadoop 2.7 or spark+hadoop 2.7)
|
|
@ -0,0 +1,110 @@
|
|||
---
|
||||
title: "Ozone Security Overview"
|
||||
date: "2019-April-03"
|
||||
menu:
|
||||
main:
|
||||
parent: Architecture
|
||||
weight: 11
|
||||
---
|
||||
<!---
|
||||
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.
|
||||
-->
|
||||
|
||||
# Security in Ozone #
|
||||
Starting with badlands release (ozone-0.4.0-alpha) ozone cluster can be secured against external threats. Specifically it can be configured for following security features:
|
||||
|
||||
1. Authentication
|
||||
2. Authorization
|
||||
3. Audit
|
||||
4. Transparent Data Encryption (TDE)
|
||||
|
||||
## Authentication ##
|
||||
|
||||
### Kerberos ###
|
||||
Similar to hadoop, Ozone allows kerberos-based authentication. So one way to setup identities for all the daemons and clients is to create kerberos keytabs and configure it like any other service in hadoop.
|
||||
|
||||
### Tokens ###
|
||||
Tokens are widely used in Hadoop to achieve lightweight authentication without compromising on security. Main motivation for using tokens inside Ozone is to prevent the unauthorized access while keeping the protocol lightweight and without sharing secret over the wire. Ozone utilizes three types of token:
|
||||
|
||||
#### Delegation token ####
|
||||
|
||||
Once client establishes their identity via kerberos they can request a delegation token from OzoneManager. This token can be used by a client to prove its identity until the token expires. Like Hadoop delegation tokens, an Ozone delegation token has 3 important fields:
|
||||
|
||||
1. **Renewer**: User responsible for renewing the token.
|
||||
2. **Issue date**: Time at which token was issued.
|
||||
3. **Max date**: Time after which token can’t be renewed.
|
||||
|
||||
Token operations like get, renew and cancel can only be performed over an Kerberos authenticated connection. Clients can use delegation token to establish connection with OzoneManager and perform any file system/object store related operations like, listing the objects in a bucket or creating a volume etc.
|
||||
|
||||
#### Block Tokens ####
|
||||
|
||||
Block tokens are similar to delegation tokens in sense that they are signed by OzoneManager. Block tokens are created by OM (OzoneManager) when a client request involves interaction with DataNodes such as read/write Ozone keys.
|
||||
|
||||
Unlike delegation tokens there is no client API to request block tokens. Instead, they are handed transparently to client along with key/block locations. Block tokens are validated by Datanodes when receiving read/write requests from clients. Block token can't be renewed explicitly by client. Client with expired block token will need to refetch the key/block locations to get new block tokens.
|
||||
|
||||
#### S3Token ####
|
||||
|
||||
Like block tokens S3Tokens are handled transparently for clients. It is signed by S3secret created by client. S3Gateway creates this token for every s3 client request. To create an S3Token user must have a S3 secret.
|
||||
|
||||
### Certificates ###
|
||||
Apart from kerberos and tokens Ozone utilizes certificate based authentication for Ozone service components. To enable this, SCM (StorageContainerManager) bootstraps itself as an Certificate Authority when security is enabled. This allows all daemons inside Ozone to have an SCM signed certificate. Below is brief descriptions of steps involved:
|
||||
|
||||
1. Datanodes and OzoneManagers submits a CSR (certificate signing request) to SCM.
|
||||
2. SCM verifies identity of DN (Datanode) or OM via Kerberos and generates a certificate.
|
||||
3. This certificate is used by OM and DN to prove their identities.
|
||||
4. Datanodes use OzoneManager certificate to validate block tokens. This is possible because both of them trust SCM signed certificates. (i.e OzoneManager and Datanodes)
|
||||
|
||||
## Authorization ##
|
||||
Ozone provides a pluggable API to control authorization of all client related operations. Default implementation allows every request. Clearly it is not meant for production environments. To configure a more fine grained policy one may configure Ranger plugin for Ozone. Since it is a pluggable module clients can also implement their own custom authorization policy and configure it using `ozone.acl.authorizer.class`.
|
||||
|
||||
## Audit ##
|
||||
|
||||
Ozone provides ability to audit all read & write operations to OM, SCM and Datanodes. Ozone audit leverages the Marker feature which enables user to selectively audit only READ or WRITE operations by a simple config change without restarting the service(s).
|
||||
|
||||
To enable/disable audit of READ operations, set filter.read.onMatch to NEUTRAL or DENY respectively. Similarly, the audit of WRITE operations can be controlled using filter.write.onMatch.
|
||||
|
||||
Generating audit logs is only half the job, so Ozone also provides AuditParser - a sqllite based command line utility to parse/query audit logs with predefined templates(ex. Top 5 commands) and options for custom query. Once the log file has been loaded to AuditParser, one can simply run a template as shown below:
|
||||
ozone auditparser <path to db file> template top5cmds
|
||||
|
||||
Similarly, users can also execute custom query using:
|
||||
|
||||
```bash
|
||||
ozone auditparser <path to db file> query "select * from audit where level=='FATAL'"
|
||||
```
|
||||
|
||||
## Transparent Data Encryption ##
|
||||
|
||||
Ozone TDE setup process and usage are very similar to HDFS TDE. The major difference is that Ozone TDE is enabled at Ozone bucket level when a bucket is created.
|
||||
|
||||
To create an encrypted bucket, client need to
|
||||
|
||||
* Create a bucket encryption key with hadoop key CLI (same as you do for HDFS encryption zone key)
|
||||
|
||||
```bash
|
||||
hadoop key create key1
|
||||
```
|
||||
|
||||
* Create an encrypted bucket with -k option
|
||||
|
||||
```bash
|
||||
ozone sh bucket create -k key1 /vol1/ez1
|
||||
```
|
||||
|
||||
After that the usage will be transparent to the client and end users, i.e., all data written to encrypted bucket are encrypted at datanodes.
|
||||
|
||||
To know more about how to setup a secure Ozone cluster refer to [How to setup secure Ozone cluster]({{< ref "SetupSecureOzone.md" >}})
|
||||
|
||||
Ozone [security architecture document](https://issues.apache.org/jira/secure/attachment/12911638/HadoopStorageLayerSecurity.pdf) can be referred for a deeper dive into Ozone Security architecture.
|
|
@ -21,7 +21,7 @@ menu:
|
|||
limitations under the License.
|
||||
-->
|
||||
|
||||
[Prometheus](https://prometheus.io/) is an open-source monitoring server developed under under the [Cloud Native Foundation](Cloud Native Foundation).
|
||||
[Prometheus](https://prometheus.io/) is an open-source monitoring server developed under under the [Cloud Native Computing Foundation](https://www.cncf.io/).
|
||||
|
||||
Ozone supports Prometheus out of the box. The servers start a prometheus
|
||||
compatible metrics endpoint where all the available hadoop metrics are published in prometheus exporter format.
|
||||
|
@ -75,14 +75,14 @@ prometheus
|
|||
|
||||
http://localhost:9090/targets
|
||||
|
||||
![Prometheus target page example](../../prometheus.png)
|
||||
![Prometheus target page example](prometheus.png)
|
||||
|
||||
|
||||
(6) Check any metrics on the prometheus web ui. For example:
|
||||
|
||||
http://localhost:9090/graph?g0.range_input=1h&g0.expr=om_metrics_num_key_allocate&g0.tab=1
|
||||
|
||||
![Prometheus target page example](../../prometheus-key-allocate.png)
|
||||
![Prometheus target page example](prometheus-key-allocate.png)
|
||||
|
||||
## Note
|
||||
|
||||
|
|
|
@ -44,16 +44,16 @@ including the data nodes and ozone services.
|
|||
ozone instance on your machine.
|
||||
|
||||
{{< highlight bash >}}
|
||||
cd ozone-0.2.1-SNAPSHOT/compose/ozone/
|
||||
cd compose/ozone/
|
||||
|
||||
docker-compose up -d
|
||||
{{< /highlight >}}
|
||||
|
||||
|
||||
To verify that ozone is working as expected, let us log into a data node and
|
||||
run _freon_, the load generator for Ozone. The ```exec datanode bash``` command
|
||||
will open a bash shell on the datanode. The ozone freon command is executed
|
||||
within the datanode container. You can quit freon via CTRL-C any time. The
|
||||
will open a bash shell on the datanode.
|
||||
|
||||
The `ozone freon` command is executed within the datanode container. You can quit freon via CTRL-C any time. The
|
||||
```rk``` profile instructs freon to generate random keys.
|
||||
|
||||
{{< highlight bash >}}
|
||||
|
|
|
@ -4,7 +4,7 @@ weight: 1
|
|||
menu:
|
||||
main:
|
||||
parent: Starting
|
||||
weight: 4
|
||||
weight: 5
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
|
|
@ -83,18 +83,39 @@ Endpoint | Status | Notes
|
|||
------------------------------------|-----------------|---------------
|
||||
PUT Object | implemented |
|
||||
GET Object | implemented | Range headers are not supported
|
||||
Multipart Uplad | not implemented |
|
||||
Multipart Uplad | implemented |Except the listing of the current MultiPartUploads.
|
||||
DELETE Object | implemented |
|
||||
HEAD Object | implemented |
|
||||
|
||||
|
||||
## Security
|
||||
|
||||
Security is not yet implemented, you can *use* any AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY
|
||||
If security is not enabled, you can *use* **any** AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY
|
||||
|
||||
Note: Ozone has a notion for 'volumes' which is missing from the S3 Rest endpoint. Under the hood S3 bucket names are mapped to Ozone 'volume/bucket' locations (depending on the given authentication information).
|
||||
If security is enabled, you can get the key and the secret with the `ozone s3 getsecret` command (*kerberos based authentication is required).
|
||||
|
||||
To show the storage location of a S3 bucket, use the `ozone sh bucket path <bucketname>` command.
|
||||
```
|
||||
/etc/security/keytabs/testuser.keytab testuser/scm@EXAMPLE.COM
|
||||
ozone s3 getsecret
|
||||
awsAccessKey=testuser/scm@EXAMPLE.COM
|
||||
awsSecret=c261b6ecabf7d37d5f9ded654b1c724adac9bd9f13e247a235e567e8296d2999
|
||||
|
||||
```
|
||||
|
||||
Now, you can use the key and the secret to access the S3 endpoint:
|
||||
|
||||
```
|
||||
export AWS_ACCESS_KEY_ID=testuser/scm@EXAMPLE.COM
|
||||
export AWS_SECRET_ACCESS_KEY=c261b6ecabf7d37d5f9ded654b1c724adac9bd9f13e247a235e567e8296d2999
|
||||
aws s3api --endpoint http://localhost:9878 create-bucket --bucket bucket1
|
||||
```
|
||||
|
||||
|
||||
## S3 bucket name mapping to Ozone buckets
|
||||
|
||||
**Note**: Ozone has a notion for 'volumes' which is missing from the S3 Rest endpoint. Under the hood S3 bucket names are mapped to Ozone 'volume/bucket' locations (depending on the given authentication information).
|
||||
|
||||
To show the storage location of a S3 bucket, use the `ozone s3 path <bucketname>` command.
|
||||
|
||||
```
|
||||
aws s3api --endpoint-url http://localhost:9878 create-bucket --bucket=bucket1
|
||||
|
|
|
@ -1,41 +0,0 @@
|
|||
---
|
||||
title: S3 Commands
|
||||
menu:
|
||||
main:
|
||||
parent: Client
|
||||
weight: 3
|
||||
---
|
||||
<!---
|
||||
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.
|
||||
-->
|
||||
|
||||
Ozone shell supports the following S3 commands.
|
||||
|
||||
* [getsecret](#get secret)
|
||||
|
||||
### Get Secret
|
||||
|
||||
User should get the kerberos ticket before using this option.
|
||||
|
||||
|
||||
{{< highlight bash >}}
|
||||
ozone sh s3 getkey
|
||||
{{< /highlight >}}
|
||||
Prints the AWS_SECRET_ACCESS_KEY and AWS_ACCESS_KEY_ID for the current user.
|
||||
|
||||
|
||||
You can try out these commands from the docker instance of the [Alpha
|
||||
Cluster](runningviadocker.html).
|
|
@ -71,7 +71,7 @@ Here is an example,
|
|||
{{< highlight xml >}}
|
||||
<property>
|
||||
<name>ozone.enabled</name>
|
||||
<value>True</value>
|
||||
<value>true</value>
|
||||
</property>
|
||||
{{< /highlight >}}
|
||||
|
||||
|
|
|
@ -0,0 +1,98 @@
|
|||
---
|
||||
title: "Setup secure ozone cluster"
|
||||
date: "2019-April-03"
|
||||
menu:
|
||||
main:
|
||||
parent: Architecture
|
||||
weight: 11
|
||||
---
|
||||
<!---
|
||||
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.
|
||||
-->
|
||||
# Setup secure ozone cluster #
|
||||
To enable security in ozone cluster **ozone.security.enabled** should be set to true.
|
||||
|
||||
Property|Value
|
||||
----------------------|------
|
||||
ozone.security.enabled| true
|
||||
|
||||
## Kerberos ##
|
||||
Configuration for service daemons:
|
||||
|
||||
Property|Description
|
||||
--------|------------------------------------------------------------
|
||||
hdds.scm.kerberos.principal | The SCM service principal. Ex scm/_HOST@REALM.COM_
|
||||
hdds.scm.kerberos.keytab.file |The keytab file used by SCM daemon to login as its service principal.
|
||||
ozone.om.kerberos.principal |The OzoneManager service principal. Ex om/_HOST@REALM.COM
|
||||
ozone.om.kerberos.keytab.file |The keytab file used by SCM daemon to login as its service principal.
|
||||
hdds.scm.http.kerberos.principal|SCM http server service principal.
|
||||
hdds.scm.http.kerberos.keytab.file|The keytab file used by SCM http server to login as its service principal.
|
||||
ozone.om.http.kerberos.principal|OzoneManager http server principal.
|
||||
ozone.om.http.kerberos.keytab.file|The keytab file used by OM http server to login as its service principal.
|
||||
ozone.s3g.keytab.file |The keytab file used by S3 gateway. Ex /etc/security/keytabs/HTTP.keytab
|
||||
ozone.s3g.authentication.kerberos.principal|S3 Gateway principal. Ex HTTP/_HOST@EXAMPLE.COM
|
||||
## Tokens ##
|
||||
|
||||
## Delegation token ##
|
||||
|
||||
Delegation tokens are enabled by default when security is enabled.
|
||||
|
||||
## Block Tokens ##
|
||||
|
||||
Property|Value
|
||||
-----------------------------|------
|
||||
hdds.block.token.enabled | true
|
||||
|
||||
## S3Token ##
|
||||
|
||||
S3 token are enabled by default when security is enabled.
|
||||
To use S3 tokens users need to perform following steps:
|
||||
|
||||
* S3 clients should get the secret access id and user secret from OzoneManager.
|
||||
|
||||
```
|
||||
ozone s3 getsecret
|
||||
```
|
||||
|
||||
* Setup secret in aws configs:
|
||||
|
||||
```
|
||||
aws configure set default.s3.signature_version s3v4
|
||||
aws configure set aws_access_key_id ${accessId}
|
||||
aws configure set aws_secret_access_key ${secret}
|
||||
aws configure set region us-west-1
|
||||
```
|
||||
|
||||
## Certificates ##
|
||||
|
||||
Certificates are used internally inside Ozone. Its enabled be default when security is enabled.
|
||||
|
||||
## Authorization ##
|
||||
|
||||
Default access authorizer for Ozone approves every request. It is not suitable for production environments. It is recommended that clients use ranger plugin for Ozone to manage authorizations.
|
||||
|
||||
Property|Value
|
||||
--------|------------------------------------------------------------
|
||||
ozone.acl.enabled | true
|
||||
ozone.acl.authorizer.class| org.apache.ranger.authorization.ozone.authorizer.RangerOzoneAuthorizer
|
||||
|
||||
## TDE ##
|
||||
|
||||
To use TDE clients must set KMS URI.
|
||||
|
||||
Property|Value
|
||||
-----------------------------------|-----------------------------------------
|
||||
hadoop.security.key.provider.path | KMS uri. Ex kms://http@kms-host:9600/kms
|
|
@ -78,11 +78,13 @@ And create a custom `core-site.xml`:
|
|||
<configuration>
|
||||
<property>
|
||||
<name>fs.o3fs.impl</name>
|
||||
<value>org.apache.hadoop.fs.ozone.OzoneFileSystem</value>
|
||||
<value>org.apache.hadoop.fs.ozone.BasicOzoneFileSystem</value>
|
||||
</property>
|
||||
</configuration>
|
||||
```
|
||||
|
||||
_Note_: You may also use `org.apache.hadoop.fs.ozone.OzoneFileSystem` without the `Basic` prefix. The `Basic` version doesn't support FS statistics and encryption zones but can work together with older hadoop versions.
|
||||
|
||||
Copy the `ozonefs.jar` file from an ozone distribution (__use the legacy version!__)
|
||||
|
||||
```
|
||||
|
@ -118,7 +120,7 @@ Download any text file and put it to the `/tmp/alice.txt` first.
|
|||
kubectl port-forward s3g-0 9878:9878
|
||||
aws s3api --endpoint http://localhost:9878 create-bucket --bucket=test
|
||||
aws s3api --endpoint http://localhost:9878 put-object --bucket test --key alice.txt --body /tmp/alice.txt
|
||||
kubectl exec -it scm-0 ozone sh bucket path test
|
||||
kubectl exec -it scm-0 ozone s3 path test
|
||||
```
|
||||
|
||||
The output of the last command is something like this:
|
||||
|
@ -134,17 +136,17 @@ Write down the ozone filesystem uri as it should be used with the spark-submit c
|
|||
|
||||
```
|
||||
kubectl create serviceaccount spark -n yournamespace
|
||||
kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=poc:yournamespace --namespace=yournamespace
|
||||
kubectl create clusterrolebinding spark-role --clusterrole=edit --serviceaccount=yournamespace:spark --namespace=yournamespace
|
||||
```
|
||||
## Execute the job
|
||||
|
||||
Execute the following spar-submit command, but change at least the following values:
|
||||
Execute the following spark-submit command, but change at least the following values:
|
||||
|
||||
* the kubernetes master url (you can check your ~/.kube/config to find the actual value)
|
||||
* the kubernetes namespace (yournamespace in this example)
|
||||
* serviceAccountName (you can use the _spark_ value if you folllowed the previous steps)
|
||||
* container.image (in this example this is myrepo/spark-ozone. This is pushed to the registry in the previous steps)
|
||||
* location of the input file (o3fs://...), use the string which is identified earlier with the `ozone sh bucket path` command
|
||||
* location of the input file (o3fs://...), use the string which is identified earlier with the `ozone s3 path <bucketname>` command
|
||||
|
||||
```
|
||||
bin/spark-submit \
|
||||
|
|
|
@ -2,8 +2,8 @@
|
|||
title: Volume Commands
|
||||
menu:
|
||||
main:
|
||||
parent: Client
|
||||
weight: 2
|
||||
parent: OzoneShell
|
||||
weight: 1
|
||||
---
|
||||
<!---
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
|
|
|
@ -31,7 +31,6 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<dependencies>
|
||||
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
|
@ -44,17 +43,11 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<goal>exec</goal>
|
||||
</goals>
|
||||
<phase>compile</phase>
|
||||
<configuration>
|
||||
<executable>${shell-executable}</executable>
|
||||
<workingDirectory>${project.build.directory}</workingDirectory>
|
||||
<arguments>
|
||||
<argument>${basedir}/dev-support/bin/generate-site.sh</argument>
|
||||
<argument>${hdds.version}</argument>
|
||||
<argument>${project.build.directory}</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
<configuration>
|
||||
<executable>dev-support/bin/generate-site.sh</executable>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
|
|
Binary file not shown.
After Width: | Height: | Size: 50 KiB |
Binary file not shown.
After Width: | Height: | Size: 38 KiB |
|
@ -21,6 +21,14 @@
|
|||
{{ range .Site.Menus.main }}
|
||||
{{ if .HasChildren }}
|
||||
<li class="{{ if $currentPage.IsMenuCurrent "main" . }}active{{ end }}">
|
||||
<a href="{{ .URL }}">
|
||||
{{ .Pre }}
|
||||
<span>{{ .Name }}</span>
|
||||
</a>
|
||||
<ul class="nav">
|
||||
{{ range .Children }}
|
||||
<li class="{{ if $currentPage.IsMenuCurrent "main" . }}active{{ end }}">
|
||||
{{ if .HasChildren }}
|
||||
<a href="{{ .URL }}">
|
||||
{{ .Pre }}
|
||||
<span>{{ .Name }}</span>
|
||||
|
@ -32,6 +40,12 @@
|
|||
</li>
|
||||
{{ end }}
|
||||
</ul>
|
||||
{{ else }}
|
||||
<a href="{{ .URL }}">{{ .Name }}</a>
|
||||
{{ end }}
|
||||
</li>
|
||||
{{ end }}
|
||||
</ul>
|
||||
</li>
|
||||
{{ else }}
|
||||
<li class="{{ if $currentPage.IsMenuCurrent "main" . }}active{{ end }}">
|
||||
|
|
|
@ -92,6 +92,10 @@ a:hover {
|
|||
padding-right: 40px;
|
||||
padding-left: 40px;
|
||||
}
|
||||
.nav-sidebar > li > ul > li > ul > li > a {
|
||||
padding-right: 60px;
|
||||
padding-left: 60px;
|
||||
}
|
||||
.nav-sidebar .active > a,
|
||||
.nav-sidebar .active > a:hover,
|
||||
.nav-sidebar .active > a:focus {
|
||||
|
@ -136,3 +140,10 @@ a:hover {
|
|||
display: inline-block;
|
||||
border-radius: 50%;
|
||||
}
|
||||
|
||||
|
||||
|
||||
|
||||
h4 {
|
||||
font-weight: bold;
|
||||
}
|
|
@ -65,14 +65,10 @@ public abstract class BaseHttpServer {
|
|||
public BaseHttpServer(Configuration conf, String name) throws IOException {
|
||||
this.name = name;
|
||||
this.conf = conf;
|
||||
if (isEnabled()) {
|
||||
policy = DFSUtil.getHttpPolicy(conf);
|
||||
if (policy.isHttpEnabled()) {
|
||||
if (isEnabled()) {
|
||||
this.httpAddress = getHttpBindAddress();
|
||||
}
|
||||
if (policy.isHttpsEnabled()) {
|
||||
this.httpsAddress = getHttpsBindAddress();
|
||||
}
|
||||
HttpServer2.Builder builder = null;
|
||||
builder = DFSUtil.httpServerTemplateForNNAndJN(conf, this.httpAddress,
|
||||
this.httpsAddress, name, getSpnegoPrincipal(), getKeytabFile());
|
||||
|
|
|
@ -308,7 +308,6 @@
|
|||
ctrl.convertToArray(response.data);
|
||||
ctrl.configs = Object.values(ctrl.keyTagMap);
|
||||
ctrl.component = 'All';
|
||||
console.log("ajay -> " + JSON.stringify(ctrl.configs));
|
||||
ctrl.sortBy('name');
|
||||
});
|
||||
};
|
||||
|
@ -326,7 +325,6 @@
|
|||
|
||||
if (ctrl.component != 'All' && (item['tag'].indexOf(ctrl
|
||||
.component) < 0)) {
|
||||
console.log(item['name'] + " false tag " + item['tag']);
|
||||
return false;
|
||||
}
|
||||
|
||||
|
|
|
@ -46,7 +46,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
|||
<hdds.version>0.4.0-SNAPSHOT</hdds.version>
|
||||
|
||||
<!-- Apache Ratis version -->
|
||||
<ratis.version>0.4.0-f283ffa-SNAPSHOT</ratis.version>
|
||||
<ratis.version>0.3.0</ratis.version>
|
||||
|
||||
<bouncycastle.version>1.60</bouncycastle.version>
|
||||
|
||||
|
|
|
@ -35,7 +35,6 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps;
|
|||
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
|
||||
import org.apache.hadoop.hdds.scm.ScmUtils;
|
||||
import org.apache.hadoop.hdds.scm.chillmode.ChillModePrecheck;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerID;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.container.ContainerManager;
|
||||
import org.apache.hadoop.hdds.scm.container.common.helpers.AllocatedBlock;
|
||||
|
@ -62,7 +61,6 @@ import static org.apache.hadoop.ozone.OzoneConfigKeys
|
|||
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT;
|
||||
import static org.apache.hadoop.ozone.OzoneConfigKeys
|
||||
.OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT;
|
||||
import java.util.function.Predicate;
|
||||
|
||||
|
||||
/** Block Manager manages the block access for SCM. */
|
||||
|
@ -203,15 +201,10 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|||
}
|
||||
|
||||
// look for OPEN containers that match the criteria.
|
||||
containerInfo = containerManager
|
||||
.getMatchingContainer(size, owner, pipeline);
|
||||
containerInfo = containerManager.getMatchingContainer(size, owner,
|
||||
pipeline, excludeList.getContainerIds());
|
||||
|
||||
// TODO: if getMachingContainer results in containers which are in exclude
|
||||
// list, we may end up in this loop forever. This case needs to be
|
||||
// addressed.
|
||||
if (containerInfo != null && (excludeList.getContainerIds() == null
|
||||
|| !discardContainer(containerInfo.containerID(),
|
||||
excludeList.getContainerIds()))) {
|
||||
if (containerInfo != null) {
|
||||
return newBlock(containerInfo);
|
||||
}
|
||||
}
|
||||
|
@ -224,11 +217,6 @@ public class BlockManagerImpl implements EventHandler<Boolean>,
|
|||
return null;
|
||||
}
|
||||
|
||||
private boolean discardContainer(ContainerID containerId,
|
||||
List<ContainerID> containers) {
|
||||
Predicate<ContainerID> predicate = p -> p.equals(containerId);
|
||||
return containers.parallelStream().anyMatch(predicate);
|
||||
}
|
||||
/**
|
||||
* newBlock - returns a new block assigned to a container.
|
||||
*
|
||||
|
|
|
@ -29,12 +29,15 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
|||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
|
||||
/**
|
||||
* Class defining Chill mode exit criteria for Containers.
|
||||
*/
|
||||
public class ContainerChillModeRule implements
|
||||
ChillModeExitRule<NodeRegistrationContainerReport> {
|
||||
ChillModeExitRule<NodeRegistrationContainerReport>,
|
||||
EventHandler<NodeRegistrationContainerReport> {
|
||||
|
||||
// Required cutoff % for containers with at least 1 reported replica.
|
||||
private double chillModeCutoff;
|
||||
|
@ -68,9 +71,6 @@ public class ContainerChillModeRule implements
|
|||
|
||||
@Override
|
||||
public boolean validate() {
|
||||
if (maxContainer == 0) {
|
||||
return true;
|
||||
}
|
||||
return getCurrentContainerThreshold() >= chillModeCutoff;
|
||||
}
|
||||
|
||||
|
@ -84,10 +84,6 @@ public class ContainerChillModeRule implements
|
|||
|
||||
@Override
|
||||
public void process(NodeRegistrationContainerReport reportsProto) {
|
||||
if (maxContainer == 0) {
|
||||
// No container to check.
|
||||
return;
|
||||
}
|
||||
|
||||
reportsProto.getReport().getReportsList().forEach(c -> {
|
||||
if (containerMap.containsKey(c.getContainerID())) {
|
||||
|
@ -96,12 +92,33 @@ public class ContainerChillModeRule implements
|
|||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMessage(NodeRegistrationContainerReport
|
||||
nodeRegistrationContainerReport, EventPublisher publisher) {
|
||||
|
||||
// TODO: when we have remove handlers, we can remove getInChillmode check
|
||||
|
||||
if (chillModeManager.getInChillMode()) {
|
||||
if (validate()) {
|
||||
return;
|
||||
}
|
||||
|
||||
process(nodeRegistrationContainerReport);
|
||||
if (chillModeManager.getInChillMode()) {
|
||||
SCMChillModeManager.getLogger().info(
|
||||
"SCM in chill mode. {} % containers have at least one"
|
||||
+ " reported replica.",
|
||||
(containerWithMinReplicas.get() / maxContainer) * 100);
|
||||
}
|
||||
|
||||
if (validate()) {
|
||||
chillModeManager.validateChillModeExitRules(publisher);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -25,13 +25,16 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
|||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer.NodeRegistrationContainerReport;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
|
||||
/**
|
||||
* Class defining Chill mode exit criteria according to number of DataNodes
|
||||
* registered with SCM.
|
||||
*/
|
||||
public class DataNodeChillModeRule implements
|
||||
ChillModeExitRule<NodeRegistrationContainerReport> {
|
||||
ChillModeExitRule<NodeRegistrationContainerReport>,
|
||||
EventHandler<NodeRegistrationContainerReport> {
|
||||
|
||||
// Min DataNodes required to exit chill mode.
|
||||
private int requiredDns;
|
||||
|
@ -62,18 +65,34 @@ public class DataNodeChillModeRule implements
|
|||
|
||||
@Override
|
||||
public void process(NodeRegistrationContainerReport reportsProto) {
|
||||
if (requiredDns == 0) {
|
||||
// No dn check required.
|
||||
|
||||
registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
|
||||
registeredDns = registeredDnSet.size();
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMessage(NodeRegistrationContainerReport
|
||||
nodeRegistrationContainerReport, EventPublisher publisher) {
|
||||
// TODO: when we have remove handlers, we can remove getInChillmode check
|
||||
|
||||
if (chillModeManager.getInChillMode()) {
|
||||
if (validate()) {
|
||||
return;
|
||||
}
|
||||
|
||||
process(nodeRegistrationContainerReport);
|
||||
|
||||
if (chillModeManager.getInChillMode()) {
|
||||
registeredDnSet.add(reportsProto.getDatanodeDetails().getUuid());
|
||||
registeredDns = registeredDnSet.size();
|
||||
SCMChillModeManager.getLogger().info(
|
||||
"SCM in chill mode. {} DataNodes registered, {} required.",
|
||||
registeredDns, requiredDns);
|
||||
}
|
||||
|
||||
if (validate()) {
|
||||
chillModeManager.validateChillModeExitRules(publisher);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -27,10 +27,6 @@ import org.apache.hadoop.hdds.HddsConfigKeys;
|
|||
import org.apache.hadoop.hdds.scm.container.ContainerInfo;
|
||||
import org.apache.hadoop.hdds.scm.events.SCMEvents;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
||||
import org.apache.hadoop.hdds.scm.server.SCMDatanodeProtocolServer
|
||||
.NodeRegistrationContainerReport;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
import org.apache.hadoop.hdds.server.events.EventQueue;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -48,8 +44,7 @@ import org.slf4j.LoggerFactory;
|
|||
* for reported containers and validates if cutoff threshold for
|
||||
* containers is meet.
|
||||
*/
|
||||
public class SCMChillModeManager implements
|
||||
EventHandler<NodeRegistrationContainerReport> {
|
||||
public class SCMChillModeManager {
|
||||
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(SCMChillModeManager.class);
|
||||
|
@ -78,9 +73,16 @@ public class SCMChillModeManager implements
|
|||
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED,
|
||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_ENABLED_DEFAULT);
|
||||
if (isChillModeEnabled) {
|
||||
exitRules.put(CONT_EXIT_RULE,
|
||||
new ContainerChillModeRule(config, allContainers, this));
|
||||
exitRules.put(DN_EXIT_RULE, new DataNodeChillModeRule(config, this));
|
||||
ContainerChillModeRule containerChillModeRule =
|
||||
new ContainerChillModeRule(config, allContainers, this);
|
||||
DataNodeChillModeRule dataNodeChillModeRule =
|
||||
new DataNodeChillModeRule(config, this);
|
||||
exitRules.put(CONT_EXIT_RULE, containerChillModeRule);
|
||||
exitRules.put(DN_EXIT_RULE, dataNodeChillModeRule);
|
||||
eventPublisher.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||
containerChillModeRule);
|
||||
eventPublisher.addHandler(SCMEvents.NODE_REGISTRATION_CONT_REPORT,
|
||||
dataNodeChillModeRule);
|
||||
|
||||
if (conf.getBoolean(
|
||||
HddsConfigKeys.HDDS_SCM_CHILLMODE_PIPELINE_AVAILABILITY_CHECK,
|
||||
|
@ -142,19 +144,7 @@ public class SCMChillModeManager implements
|
|||
emitChillModeStatus();
|
||||
// TODO: #CLUTIL if we reenter chill mode the fixed interval pipeline
|
||||
// creation job needs to stop
|
||||
RatisPipelineUtils
|
||||
.scheduleFixedIntervalPipelineCreator(pipelineManager, config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onMessage(
|
||||
NodeRegistrationContainerReport nodeRegistrationContainerReport,
|
||||
EventPublisher publisher) {
|
||||
if (getInChillMode()) {
|
||||
exitRules.get(CONT_EXIT_RULE).process(nodeRegistrationContainerReport);
|
||||
exitRules.get(DN_EXIT_RULE).process(nodeRegistrationContainerReport);
|
||||
validateChillModeExitRules(publisher);
|
||||
}
|
||||
pipelineManager.startPipelineCreator();
|
||||
}
|
||||
|
||||
public boolean getInChillMode() {
|
||||
|
|
|
@ -152,4 +152,15 @@ public interface ContainerManager extends Closeable {
|
|||
*/
|
||||
ContainerInfo getMatchingContainer(long size, String owner,
|
||||
Pipeline pipeline);
|
||||
|
||||
/**
|
||||
* Returns ContainerInfo which matches the requirements.
|
||||
* @param size - the amount of space required in the container
|
||||
* @param owner - the user which requires space in its owned container
|
||||
* @param pipeline - pipeline to which the container should belong.
|
||||
* @param excludedContainerIDS - containerIds to be excluded.
|
||||
* @return ContainerInfo for the matching container.
|
||||
*/
|
||||
ContainerInfo getMatchingContainer(long size, String owner,
|
||||
Pipeline pipeline, List<ContainerID> excludedContainerIDS);
|
||||
}
|
||||
|
|
|
@ -353,6 +353,12 @@ public class SCMContainerManager implements ContainerManager {
|
|||
*/
|
||||
public ContainerInfo getMatchingContainer(final long sizeRequired,
|
||||
String owner, Pipeline pipeline) {
|
||||
return getMatchingContainer(sizeRequired, owner, pipeline, Collections
|
||||
.emptyList());
|
||||
}
|
||||
|
||||
public ContainerInfo getMatchingContainer(final long sizeRequired,
|
||||
String owner, Pipeline pipeline, List<ContainerID> excludedContainers) {
|
||||
try {
|
||||
//TODO: #CLUTIL See if lock is required here
|
||||
NavigableSet<ContainerID> containerIDs =
|
||||
|
@ -378,6 +384,7 @@ public class SCMContainerManager implements ContainerManager {
|
|||
}
|
||||
}
|
||||
|
||||
containerIDs.removeAll(excludedContainers);
|
||||
ContainerInfo containerInfo =
|
||||
containerStateManager.getMatchingContainer(sizeRequired, owner,
|
||||
pipeline.getId(), containerIDs);
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.apache.hadoop.hdds.scm.node;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.PipelineManager;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
|
||||
|
@ -41,7 +40,6 @@ public class NewNodeHandler implements EventHandler<DatanodeDetails> {
|
|||
@Override
|
||||
public void onMessage(DatanodeDetails datanodeDetails,
|
||||
EventPublisher publisher) {
|
||||
RatisPipelineUtils
|
||||
.triggerPipelineCreation(pipelineManager, conf, 0);
|
||||
pipelineManager.triggerPipelineCreation();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ 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.pipeline.PipelineManager;
|
||||
import org.apache.hadoop.hdds.scm.pipeline.RatisPipelineUtils;
|
||||
import org.apache.hadoop.hdds.server.events.EventHandler;
|
||||
import org.apache.hadoop.hdds.server.events.EventPublisher;
|
||||
|
||||
|
@ -43,6 +42,6 @@ public class NonHealthyToHealthyNodeHandler
|
|||
@Override
|
||||
public void onMessage(DatanodeDetails datanodeDetails,
|
||||
EventPublisher publisher) {
|
||||
RatisPipelineUtils.triggerPipelineCreation(pipelineManager, conf, 0);
|
||||
pipelineManager.triggerPipelineCreation();
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue