diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java index 75ae01c1005..d845a835db1 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java @@ -19,8 +19,10 @@ import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.OzoneSecurityUtil; @@ -41,6 +43,8 @@ public static void enableErrorInjection(ErrorInjector injector) { private final boolean topologyAwareRead; private final ClientTrustManager trustManager; private final boolean securityEnabled; + private boolean shortCircuitEnabled; + private DomainSocketFactory domainSocketFactory; public XceiverClientCreator(ConfigurationSource conf) { this(conf, null); @@ -56,20 +60,36 @@ public XceiverClientCreator(ConfigurationSource conf, ClientTrustManager trustMa if (securityEnabled) { Preconditions.checkNotNull(trustManager); } + shortCircuitEnabled = conf.getObject(OzoneClientConfig.class).isShortCircuitEnabled(); + if (shortCircuitEnabled) { + domainSocketFactory = DomainSocketFactory.getInstance(conf); + } } public boolean isSecurityEnabled() { return securityEnabled; } + public boolean isShortCircuitEnabled() { + return shortCircuitEnabled && domainSocketFactory.isServiceReady(); + } + protected XceiverClientSpi newClient(Pipeline pipeline) throws IOException { + return newClient(pipeline, null); + } + + protected XceiverClientSpi newClient(Pipeline pipeline, DatanodeDetails dn) throws IOException { XceiverClientSpi client; switch (pipeline.getType()) { case RATIS: client = XceiverClientRatis.newXceiverClientRatis(pipeline, conf, trustManager, errorInjector); break; case STAND_ALONE: - client = new XceiverClientGrpc(pipeline, conf, trustManager); + if (dn != null) { + client = new XceiverClientShortCircuit(pipeline, conf, dn); + } else { + client = new XceiverClientGrpc(pipeline, conf, trustManager); + } break; case EC: client = new ECXceiverClientGrpc(pipeline, conf, trustManager); @@ -97,7 +117,14 @@ public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClie } @Override - public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) throws IOException { + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException { + return acquireClient(pipeline); + } + + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) + throws IOException { return acquireClient(pipeline); } @@ -117,7 +144,10 @@ public void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClie } @Override - public void close() throws Exception { - // clients are not tracked, closing each client is the responsibility of users of this class + public void close() { + // clients are not tracked, closing each client is the responsibility of users of this classclass + if (domainSocketFactory != null) { + domainSocketFactory.close(); + } } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java index b7276d645b4..c47321b815d 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientFactory.java @@ -53,8 +53,10 @@ public interface XceiverClientFactory extends AutoCloseable { * @return XceiverClientSpi connected to a container * @throws IOException if a XceiverClientSpi cannot be acquired */ - XceiverClientSpi acquireClientForReadData(Pipeline pipeline) - throws IOException; + default XceiverClientSpi acquireClientForReadData(Pipeline pipeline) + throws IOException { + return acquireClientForReadData(pipeline, false); + } /** * Releases a read XceiverClientSpi after use. @@ -73,10 +75,20 @@ void releaseClientForReadData(XceiverClientSpi client, * @return XceiverClientSpi connected to a container * @throws IOException if a XceiverClientSpi cannot be acquired */ - XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) + default XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) throws IOException { + return acquireClient(pipeline, topologyAware, false); + } + + XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException; + + XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) throws IOException; void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient, boolean topologyAware); + default boolean isShortCircuitEnabled() { + return false; + } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java index c02306f8af8..4cfc75c5a92 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java @@ -86,7 +86,7 @@ * how it works, and how it is integrated with the Ozone client. */ public class XceiverClientGrpc extends XceiverClientSpi { - private static final Logger LOG = + public static final Logger LOG = LoggerFactory.getLogger(XceiverClientGrpc.class); private final Pipeline pipeline; private final ConfigurationSource config; @@ -133,6 +133,7 @@ public XceiverClientGrpc(Pipeline pipeline, ConfigurationSource config, OzoneConfigKeys.OZONE_NETWORK_TOPOLOGY_AWARE_READ_DEFAULT); this.trustManager = trustManager; this.getBlockDNcache = new ConcurrentHashMap<>(); + LOG.info("{} is created for pipeline {}", XceiverClientGrpc.class.getSimpleName(), pipeline); } /** @@ -246,6 +247,10 @@ public synchronized void close() { } } + public boolean isClosed() { + return closed; + } + @Override public Pipeline getPipeline() { return pipeline; diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java index 07b70441721..42b109c1aef 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java @@ -19,6 +19,8 @@ package org.apache.hadoop.hdds.scm; import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.TimeUnit; import org.apache.hadoop.hdds.conf.Config; @@ -29,6 +31,9 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.ozone.util.OzoneNetUtils; import org.apache.hadoop.security.UserGroupInformation; import com.google.common.annotations.VisibleForTesting; @@ -38,6 +43,7 @@ import com.google.common.cache.RemovalListener; import com.google.common.cache.RemovalNotification; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static org.apache.hadoop.hdds.DatanodeVersion.SHORT_CIRCUIT_READS; import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE; import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE; import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.NO_REPLICA_FOUND; @@ -64,8 +70,8 @@ public class XceiverClientManager extends XceiverClientCreator { private final Cache clientCache; private final CacheMetrics cacheMetrics; - private static XceiverClientMetrics metrics; + private final ConcurrentHashMap localDNCache; /** * Creates a new XceiverClientManager for non secured ozone cluster. @@ -105,6 +111,7 @@ public void onRemoval( }).build(); cacheMetrics = CacheMetrics.create(clientCache, this); + this.localDNCache = new ConcurrentHashMap<>(); } @VisibleForTesting @@ -117,17 +124,54 @@ public Cache getClientCache() { * * If there is already a cached XceiverClientSpi, simply return * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ + @Override + public XceiverClientSpi acquireClient(Pipeline pipeline) + throws IOException { + return acquireClient(pipeline, false, false); + } + + /** + * Acquires a XceiverClientSpi connected to a container for read. + * + * If there is already a cached XceiverClientSpi, simply return + * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired + */ + @Override + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) + throws IOException { + return acquireClient(pipeline, false, allowShortCircuit); + } + + /** + * Acquires a XceiverClientSpi connected to a container capable of + * storing the specified key. + * + * If there is already a cached XceiverClientSpi, simply return + * the cached otherwise create a new one. + * + * @param pipeline the container pipeline for the client connection + * @return XceiverClientSpi connected to a container + * @throws IOException if a XceiverClientSpi cannot be acquired */ @Override public XceiverClientSpi acquireClient(Pipeline pipeline, - boolean topologyAware) throws IOException { + boolean topologyAware, boolean allowShortCircuit) throws IOException { Preconditions.checkNotNull(pipeline); Preconditions.checkArgument(pipeline.getNodes() != null); Preconditions.checkArgument(!pipeline.getNodes().isEmpty(), NO_REPLICA_FOUND); synchronized (clientCache) { - XceiverClientSpi info = getClient(pipeline, topologyAware); + XceiverClientSpi info = getClient(pipeline, topologyAware, allowShortCircuit); info.incrementReference(); return info; } @@ -141,7 +185,7 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, client.decrementReference(); if (invalidateClient) { Pipeline pipeline = client.getPipeline(); - String key = getPipelineCacheKey(pipeline, topologyAware); + String key = getPipelineCacheKey(pipeline, topologyAware, client instanceof XceiverClientShortCircuit); XceiverClientSpi cachedClient = clientCache.getIfPresent(key); if (cachedClient == client) { clientCache.invalidate(key); @@ -150,24 +194,44 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, } } - protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware) + protected XceiverClientSpi getClient(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) throws IOException { try { - // create different client different pipeline node based on - // network topology - String key = getPipelineCacheKey(pipeline, topologyAware); - return clientCache.get(key, () -> newClient(pipeline)); + // create different client different pipeline node based on network topology + String key = getPipelineCacheKey(pipeline, topologyAware, allowShortCircuit); + return clientCache.get(key, () -> newClient(pipeline, localDNCache.get(key))); } catch (Exception e) { throw new IOException( "Exception getting XceiverClient: " + e, e); } } - private String getPipelineCacheKey(Pipeline pipeline, - boolean topologyAware) { - String key = pipeline.getId().getId().toString() + pipeline.getType(); + private String getPipelineCacheKey(Pipeline pipeline, boolean topologyAware, boolean allowShortCircuit) { + String key = pipeline.getId().getId().toString() + "-" + pipeline.getType(); boolean isEC = pipeline.getType() == HddsProtos.ReplicationType.EC; - if (topologyAware || isEC) { + DatanodeDetails localDN = null; + + if ((!isEC) && allowShortCircuit && isShortCircuitEnabled()) { + int port = 0; + InetSocketAddress localAddr = null; + for (DatanodeDetails dn : pipeline.getNodes()) { + // read port from the data node, on failure use default configured port. + port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); + InetSocketAddress addr = NetUtils.createSocketAddr(dn.getIpAddress(), port); + if (OzoneNetUtils.isAddressLocal(addr) && + dn.getCurrentVersion() >= SHORT_CIRCUIT_READS.toProtoValue()) { + localAddr = addr; + localDN = dn; + break; + } + } + if (localAddr != null) { + // Find a local DN and short circuit read is enabled + key += "@" + localAddr.getHostName() + ":" + port + "/" + DomainSocketFactory.FEATURE_FLAG; + } + } + + if (localDN == null && (topologyAware || isEC)) { try { DatanodeDetails closestNode = pipeline.getClosestNode(); // Pipeline cache key uses host:port suffix to handle @@ -185,7 +249,7 @@ private String getPipelineCacheKey(Pipeline pipeline, // Standalone port is chosen since all datanodes should have a // standalone port regardless of version and this port should not // have any collisions. - key += closestNode.getHostName() + closestNode.getPort( + key += closestNode.getHostName() + ":" + closestNode.getPort( DatanodeDetails.Port.Name.STANDALONE); } catch (IOException e) { LOG.error("Failed to get closest node to create pipeline cache key:" + @@ -197,12 +261,16 @@ private String getPipelineCacheKey(Pipeline pipeline, // Append user short name to key to prevent a different user // from using same instance of xceiverClient. try { - key += UserGroupInformation.getCurrentUser().getShortUserName(); + key = UserGroupInformation.getCurrentUser().getShortUserName() + "@" + key; } catch (IOException e) { LOG.error("Failed to get current user to create pipeline cache key:" + e.getMessage()); } } + + if (localDN != null) { + localDNCache.put(key, localDN); + } return key; } @@ -211,12 +279,14 @@ private String getPipelineCacheKey(Pipeline pipeline, */ @Override public void close() { + super.close(); //closing is done through RemovalListener clientCache.invalidateAll(); clientCache.cleanUp(); if (LOG.isDebugEnabled()) { LOG.debug("XceiverClient cache stats: {}", clientCache.stats()); } + localDNCache.clear(); cacheMetrics.unregister(); if (metrics != null) { diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java new file mode 100644 index 00000000000..a40c288b9da --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java @@ -0,0 +1,629 @@ +/* + * 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; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.security.exception.SCMSecurityException; +import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.thirdparty.protobuf.CodedInputStream; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.LimitInputStream; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.ratis.thirdparty.io.grpc.Status; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.net.InetSocketAddress; +import java.nio.channels.ClosedChannelException; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Timer; +import java.util.TimerTask; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantLock; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.apache.hadoop.hdds.HddsUtils.processForDebug; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION; + +/** + * {@link XceiverClientSpi} implementation, the client to read local replica through short circuit. + */ +public class XceiverClientShortCircuit extends XceiverClientSpi { + public static final Logger LOG = + LoggerFactory.getLogger(XceiverClientShortCircuit.class); + private final Pipeline pipeline; + private final ConfigurationSource config; + private final XceiverClientMetrics metrics; + private int readTimeoutMs; + private int writeTimeoutMs; + // Cache the stream of blocks + private final Map blockStreamCache; + private final Map sentRequests; + private final Daemon readDaemon; + private Timer timer; + + private boolean closed = false; + private final DatanodeDetails dn; + private final InetSocketAddress dnAddr; + private final DomainSocketFactory domainSocketFactory; + private DomainSocket domainSocket; + private AtomicBoolean isDomainSocketOpen = new AtomicBoolean(false); + private Lock lock = new ReentrantLock(); + private final int bufferSize; + private final ByteString clientId = ByteString.copyFrom(UUID.randomUUID().toString().getBytes(UTF_8)); + private final AtomicLong callId = new AtomicLong(0); + private String prefix; + + /** + * Constructs a client that can communicate with the Container framework on local datanode through DomainSocket. + */ + public XceiverClientShortCircuit(Pipeline pipeline, ConfigurationSource config, DatanodeDetails dn) { + super(); + Preconditions.checkNotNull(config); + this.readTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + this.writeTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + + this.pipeline = pipeline; + this.dn = dn; + this.domainSocketFactory = DomainSocketFactory.getInstance(config); + this.config = config; + this.metrics = XceiverClientManager.getXceiverClientMetrics(); + this.blockStreamCache = new ConcurrentHashMap<>(); + this.sentRequests = new ConcurrentHashMap<>(); + int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); + this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port); + this.bufferSize = config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize(); + this.readDaemon = new Daemon(new ReceiveResponseTask()); + LOG.info("{} is created for pipeline {}", XceiverClientShortCircuit.class.getSimpleName(), pipeline); + } + + /** + * Create the DomainSocket to connect to the local DataNode. + */ + @Override + public void connect() throws IOException { + // Even the in & out stream has returned EOFException, domainSocket.isOpen() is still true. + if (domainSocket != null && domainSocket.isOpen() && isDomainSocketOpen.get()) { + return; + } + domainSocket = domainSocketFactory.createSocket(readTimeoutMs, writeTimeoutMs, dnAddr); + isDomainSocketOpen.set(true); + prefix = XceiverClientShortCircuit.class.getSimpleName() + "-" + domainSocket.toString(); + timer = new Timer(prefix + "-Timer"); + readDaemon.start(); + LOG.info("{} is started", prefix); + } + + /** + * Close the DomainSocket. + */ + @Override + public synchronized void close() { + closed = true; + timer.cancel(); + if (domainSocket != null) { + try { + isDomainSocketOpen.set(false); + domainSocket.close(); + LOG.info("{} is closed for {}", domainSocket.toString(), dn); + } catch (IOException e) { + LOG.warn("Failed to close domain socket for datanode {}", dn, e); + } + } + readDaemon.interrupt(); + try { + readDaemon.join(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + + public boolean isClosed() { + return closed; + } + + @Override + public Pipeline getPipeline() { + return pipeline; + } + + public DatanodeDetails getDn() { + return this.dn; + } + + public ByteString getClientId() { + return clientId; + } + + public long getCallId() { + return callId.incrementAndGet(); + } + + @Override + public ContainerCommandResponseProto sendCommand(ContainerCommandRequestProto request) throws IOException { + try { + return sendCommandWithTraceID(request, null).getResponse().get(); + } catch (ExecutionException e) { + throw getIOExceptionForSendCommand(request, e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted."); + Thread.currentThread().interrupt(); + throw (IOException) new InterruptedIOException( + "Command " + processForDebug(request) + " was interrupted.") + .initCause(e); + } + } + + @Override + public Map + sendCommandOnAllNodes( + ContainerCommandRequestProto request) throws IOException { + throw new UnsupportedOperationException("Operation Not supported for " + + DomainSocketFactory.FEATURE + " client"); + } + + @Override + public ContainerCommandResponseProto sendCommand( + ContainerCommandRequestProto request, List validators) + throws IOException { + try { + XceiverClientReply reply; + reply = sendCommandWithTraceID(request, validators); + return reply.getResponse().get(); + } catch (ExecutionException e) { + throw getIOExceptionForSendCommand(request, e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted."); + Thread.currentThread().interrupt(); + throw (IOException) new InterruptedIOException( + "Command " + processForDebug(request) + " was interrupted.") + .initCause(e); + } + } + + private XceiverClientReply sendCommandWithTraceID( + ContainerCommandRequestProto request, List validators) + throws IOException { + String spanName = "XceiverClientGrpc." + request.getCmdType().name(); + return TracingUtil.executeInNewSpan(spanName, + () -> { + ContainerCommandRequestProto finalPayload = + ContainerCommandRequestProto.newBuilder(request) + .setTraceID(TracingUtil.exportCurrentSpan()).build(); + ContainerCommandResponseProto responseProto = null; + IOException ioException = null; + + // In case of an exception or an error, we will try to read from the + // datanodes in the pipeline in a round-robin fashion. + XceiverClientReply reply = new XceiverClientReply(null); + + if (request.getCmdType() != ContainerProtos.Type.GetBlock && + request.getCmdType() != ContainerProtos.Type.Echo) { + throw new UnsupportedOperationException("Command " + request.getCmdType() + + " is not supported for " + DomainSocketFactory.FEATURE + " client"); + } + + try { + if (LOG.isDebugEnabled()) { + LOG.debug("Executing command {} on datanode {}", request, dn); + } + reply.addDatanode(dn); + responseProto = sendCommandInternal(finalPayload).getResponse().get(); + if (validators != null && !validators.isEmpty()) { + for (Validator validator : validators) { + validator.accept(request, responseProto); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("request {} {} {} finished", request.getCmdType(), + request.getClientId().toStringUtf8(), request.getCallId()); + } + } catch (IOException e) { + ioException = e; + responseProto = null; + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to execute command {} on datanode {}", request, dn, e); + } + } catch (ExecutionException e) { + if (LOG.isDebugEnabled()) { + LOG.debug("Failed to execute command {} on datanode {}", request, dn, e); + } + if (Status.fromThrowable(e.getCause()).getCode() + == Status.UNAUTHENTICATED.getCode()) { + throw new SCMSecurityException("Failed to authenticate with " + + "datanode DomainSocket XceiverServer with Ozone block token."); + } + ioException = new IOException(e); + } catch (InterruptedException e) { + LOG.error("Command execution was interrupted ", e); + Thread.currentThread().interrupt(); + } + + if (responseProto != null) { + reply.setResponse(CompletableFuture.completedFuture(responseProto)); + return reply; + } else { + Objects.requireNonNull(ioException); + String message = "Failed to execute command {}"; + if (LOG.isDebugEnabled()) { + LOG.debug(message + " on the datanode {}.", request, dn, ioException); + } else { + LOG.error(message + " on the datanode {}.", request, dn, ioException); + } + throw ioException; + } + }); + } + + @VisibleForTesting + public XceiverClientReply sendCommandInternal(ContainerCommandRequestProto request) + throws IOException, InterruptedException { + checkOpen(); + final CompletableFuture replyFuture = + new CompletableFuture<>(); + RequestEntry entry = new RequestEntry(request, replyFuture); + sendRequest(entry); + return new XceiverClientReply(replyFuture); + } + + @Override + public XceiverClientReply sendCommandAsync( + ContainerCommandRequestProto request) + throws IOException, ExecutionException, InterruptedException { + throw new UnsupportedOperationException("Operation Not supported for " + DomainSocketFactory.FEATURE + " client"); + } + + public synchronized void checkOpen() throws IOException { + if (closed) { + throw new IOException("DomainSocket is not connected."); + } + + if (!isDomainSocketOpen.get()) { + throw new IOException("DomainSocket stream is not open."); + } + } + + @Override + public CompletableFuture watchForCommit(long index) { + // there is no notion of watch for commit index in short-circuit local reads + return null; + } + + @Override + public long getReplicatedMinCommitIndex() { + return 0; + } + + public FileInputStream getFileInputStream(long id, DatanodeBlockID blockID) { + String mapKey = id + blockID.toString(); + return blockStreamCache.remove(mapKey); + } + + @Override + public HddsProtos.ReplicationType getPipelineType() { + return HddsProtos.ReplicationType.STAND_ALONE; + } + + public ConfigurationSource getConfig() { + return config; + } + + @VisibleForTesting + public static Logger getLogger() { + return LOG; + } + + public void setReadTimeout(int timeout) { + this.readTimeoutMs = timeout; + } + + public int getReadTimeout() { + return this.readTimeoutMs; + } + + String getRequestUniqueID(ContainerCommandRequestProto request) { + return request.getClientId().toStringUtf8() + request.getCallId(); + } + + String getRequestUniqueID(ContainerCommandResponseProto response) { + return response.getClientId().toStringUtf8() + response.getCallId(); + } + + void requestTimeout(String requestId) { + final RequestEntry entry = sentRequests.remove(requestId); + if (entry != null) { + LOG.warn("Timeout to receive response for command {}", entry.getRequest()); + ContainerProtos.Type type = entry.getRequest().getCmdType(); + metrics.decrPendingContainerOpsMetrics(type); + entry.getFuture().completeExceptionally(new TimeoutException("Timeout to receive response")); + } + } + + public void sendRequest(RequestEntry entry) { + ContainerCommandRequestProto request = entry.getRequest(); + try { + String key = getRequestUniqueID(request); + TimerTask task = new TimerTask() { + @Override + public void run() { + requestTimeout(key); + } + }; + entry.setTimerTask(task); + timer.schedule(task, readTimeoutMs); + sentRequests.put(key, entry); + ContainerProtos.Type type = request.getCmdType(); + metrics.incrPendingContainerOpsMetrics(type); + byte[] bytes = request.toByteArray(); + if (bytes.length != request.getSerializedSize()) { + throw new IOException("Serialized request " + request.getCmdType() + + " size mismatch, byte array size " + bytes.length + + ", serialized size " + request.getSerializedSize()); + } + + DataOutputStream dataOut = + new DataOutputStream(new BufferedOutputStream(domainSocket.getOutputStream(), bufferSize)); + lock.lock(); + try { + // send version number + dataOut.writeShort(DATA_TRANSFER_VERSION); + // send command type + dataOut.writeShort(type.getNumber()); + // send request body + request.writeDelimitedTo(dataOut); + dataOut.flush(); + } finally { + lock.unlock(); + entry.setSentTimeNs(); + } + } catch (IOException e) { + LOG.error("Failed to send command {}", request, e); + entry.getFuture().completeExceptionally(e); + metrics.decrPendingContainerOpsMetrics(request.getCmdType()); + metrics.addContainerOpsLatency(request.getCmdType(), System.nanoTime() - entry.getCreateTimeNs()); + } + } + + @Override + public String toString() { + final StringBuilder b = + new StringBuilder(getClass().getSimpleName()).append("["); + b.append(" DomainSocket: ").append(domainSocket.toString()); + b.append(" Pipeline: ").append(pipeline.toString()); + return b.toString(); + } + + /** + * Task to receive responses from server. + */ + public class ReceiveResponseTask implements Runnable { + @Override + public void run() { + long timerTaskCancelledCount = 0; + do { + Thread.currentThread().setName(prefix + "-ReceiveResponse"); + RequestEntry entry = null; + try { + DataInputStream dataIn = new DataInputStream(domainSocket.getInputStream()); + final short version = dataIn.readShort(); + if (version != DATA_TRANSFER_VERSION) { + throw new IOException("Version Mismatch (Expected: " + + DATA_TRANSFER_VERSION + ", Received: " + version + ")"); + } + long receiveStartTime = System.nanoTime(); + final short typeNumber = dataIn.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + ContainerCommandResponseProto responseProto = + ContainerCommandResponseProto.parseFrom(vintPrefixed(dataIn)); + if (LOG.isDebugEnabled()) { + LOG.debug("received response {} callId {}", type, responseProto.getCallId()); + } + String key = getRequestUniqueID(responseProto); + entry = sentRequests.remove(key); + if (entry == null) { + // This could be two cases + // 1. there is bug in the code + // 2. the response is too late, the request is removed from sentRequests after it is timeout. + throw new IOException("Failed to find request for response, type " + type + + ", clientId " + responseProto.getClientId().toStringUtf8() + ", callId " + responseProto.getCallId()); + } + + // cancel timeout timer task + if (entry.getTimerTask().cancel()) { + timerTaskCancelledCount++; + // purge timer every 1000 cancels + if (timerTaskCancelledCount == 1000) { + timer.purge(); + timerTaskCancelledCount = 0; + } + } + + long processStartTime = System.nanoTime(); + ContainerProtos.Result result = responseProto.getResult(); + if (result == ContainerProtos.Result.SUCCESS) { + if (type == ContainerProtos.Type.GetBlock) { + try { + ContainerProtos.GetBlockResponseProto getBlockResponse = responseProto.getGetBlock(); + if (!getBlockResponse.getShortCircuitAccessGranted()) { + throw new IOException("Short-circuit access is denied on " + dn); + } + // read FS from domainSocket + FileInputStream[] fis = new FileInputStream[1]; + byte[] buf = new byte[1]; + int ret = domainSocket.recvFileInputStreams(fis, buf, 0, buf.length); + if (ret == -1) { + throw new IOException("failed to get a file descriptor from datanode " + dn + + " for peer is shutdown."); + } + if (fis[0] == null) { + throw new IOException("the datanode " + dn + " failed to " + + "pass a file descriptor (might have reached open file limit)."); + } + if (buf[0] != DATA_TRANSFER_MAGIC_CODE) { + throw new IOException("Magic Code Mismatch (Expected: " + + DATA_TRANSFER_MAGIC_CODE + ", Received: " + buf[0] + ")"); + } + DatanodeBlockID blockID = getBlockResponse.getBlockData().getBlockID(); + String mapKey = responseProto.getCallId() + blockID.toString(); + blockStreamCache.put(mapKey, fis[0]); + if (LOG.isDebugEnabled()) { + LOG.debug("received fd {} ", fis[0]); + } + } catch (IOException e) { + LOG.warn("Failed to handle short-circuit information exchange", e); + // disable docket socket for a while + domainSocketFactory.disableShortCircuit(); + entry.getFuture().completeExceptionally(e); + continue; + } + } + entry.getFuture().complete(responseProto); + } else { + // response result is not SUCCESS + entry.getFuture().complete(responseProto); + } + long currentTime = System.nanoTime(); + long endToEndCost = currentTime - entry.getCreateTimeNs(); + long sentCost = entry.getSentTimeNs() - entry.getCreateTimeNs(); + long receiveCost = processStartTime - receiveStartTime; + long processCost = currentTime - processStartTime; + if (LOG.isDebugEnabled()) { + LOG.debug("Executed command {} {} on datanode {}, end-to-end {} ns, sent {} ns, receive {} ns, " + + " process {} ns, cmdType = {}, {}, {}", + entry.getRequest().getClientId().toStringUtf8(), entry.getRequest().getCallId(), dn, endToEndCost, + sentCost, receiveCost, processCost, type, entry.getSentTimeNs(), entry.getCreateTimeNs()); + } + metrics.decrPendingContainerOpsMetrics(type); + metrics.addContainerOpsLatency(type, endToEndCost); + } catch (EOFException | ClosedChannelException e) { + isDomainSocketOpen.set(false); + LOG.info("ReceiveResponseTask is closed due to {}", e.getClass().getName()); + // fail all requests pending responses + sentRequests.values().forEach(i -> i.fail(e)); + } catch (Throwable e) { + isDomainSocketOpen.set(false); + LOG.error("Failed to receive response", e); + if (entry != null) { + entry.getFuture().completeExceptionally(e); + } + sentRequests.values().forEach(i -> i.fail(e)); + break; + } + } while (!isClosed()); + } + } + + public static InputStream vintPrefixed(final InputStream input) + throws IOException { + final int firstByte = input.read(); + if (firstByte == -1) { + throw new EOFException( + "Unexpected EOF while trying to read response from server"); + } + + int size = CodedInputStream.readRawVarint32(firstByte, input); + assert size >= 0; + return new LimitInputStream(input, size); + } + + /** + * Class wraps a container command request. + */ + public static class RequestEntry { + private ContainerCommandRequestProto request; + private CompletableFuture future; + private long createTimeNs; + private long sentTimeNs; + private TimerTask timerTask; + + RequestEntry(ContainerCommandRequestProto requestProto, + CompletableFuture future) { + this.request = requestProto; + this.future = future; + this.createTimeNs = System.nanoTime(); + } + + public ContainerCommandRequestProto getRequest() { + return request; + } + + public CompletableFuture getFuture() { + return future; + } + + public long getCreateTimeNs() { + return createTimeNs; + } + + public long getSentTimeNs() { + return sentTimeNs; + } + + public void setSentTimeNs() { + sentTimeNs = System.nanoTime(); + } + + public void setTimerTask(TimerTask task) { + timerTask = task; + } + + public TimerTask getTimerTask() { + return timerTask; + } + + public void fail(Throwable e) { + timerTask.cancel(); + future.completeExceptionally(e); + } + } +} diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java index d6353be9d22..5fa6828484e 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.storage; import java.io.EOFException; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.time.Instant; @@ -26,12 +27,14 @@ import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import com.google.common.base.Preconditions; import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; @@ -40,6 +43,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.XceiverClientSpi.Validator; import org.apache.hadoop.hdds.scm.client.HddsClientUtils; @@ -77,7 +81,10 @@ public class BlockInputStream extends BlockExtendedInputStream { new AtomicReference<>(); private final boolean verifyChecksum; private XceiverClientFactory xceiverClientFactory; - private XceiverClientSpi xceiverClient; + private XceiverClientSpi xceiverClientGrpc; + private XceiverClientShortCircuit xceiverClientShortCircuit; + private final AtomicBoolean fallbackToGrpc = new AtomicBoolean(false); + private FileInputStream blockInputStream; private boolean initialized = false; // TODO: do we need to change retrypolicy based on exception. private final RetryPolicy retryPolicy; @@ -269,14 +276,75 @@ protected BlockData getBlockData() throws IOException { * @return BlockData. */ protected BlockData getBlockDataUsingClient() throws IOException { - Pipeline pipeline = pipelineRef.get(); + if (xceiverClientShortCircuit != null) { + try { + return getBlockDataUsingSCClient(); + } catch (IOException e) { + LOG.warn("Failed to get blockData using short-circuit client", e); + // acquire client again if xceiverClientGrpc is not acquired. + acquireClient(); + } + } + return getBlockDataUsingGRPCClient(); + } + + @VisibleForTesting + protected BlockData getBlockDataUsingSCClient() throws IOException { + final Pipeline pipeline = xceiverClientShortCircuit.getPipeline(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Initializing BlockInputStream for get key to access {}", + blockID.getContainerID()); + } + + DatanodeBlockID.Builder blkIDBuilder = + DatanodeBlockID.newBuilder().setContainerID(blockID.getContainerID()) + .setLocalID(blockID.getLocalID()) + .setBlockCommitSequenceId(blockID.getBlockCommitSequenceId()); + + int replicaIndex = pipeline.getReplicaIndex(xceiverClientShortCircuit.getDn()); + if (replicaIndex > 0) { + blkIDBuilder.setReplicaIndex(replicaIndex); + } + DatanodeBlockID datanodeBlockID = blkIDBuilder.build(); + ContainerProtos.GetBlockRequestProto.Builder readBlockRequest = + ContainerProtos.GetBlockRequestProto.newBuilder().setBlockID(datanodeBlockID) + .setRequestShortCircuitAccess(true); + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.GetBlock) + .setContainerID(datanodeBlockID.getContainerID()) + .setGetBlock(readBlockRequest) + .setClientId(xceiverClientShortCircuit.getClientId()) + .setCallId(xceiverClientShortCircuit.getCallId()); + if (tokenRef.get() != null) { + builder.setEncodedToken(tokenRef.get().encodeToUrlString()); + } + GetBlockResponseProto response = ContainerProtocolCalls.getBlock(xceiverClientShortCircuit, + VALIDATORS, builder, xceiverClientShortCircuit.getDn()); + + blockInputStream = xceiverClientShortCircuit.getFileInputStream(builder.getCallId(), datanodeBlockID); + if (blockInputStream == null) { + throw new IOException("Failed to get file InputStream for block " + datanodeBlockID); + } else { + if (LOG.isDebugEnabled()) { + LOG.debug("Get the FileInputStream of block {}", datanodeBlockID); + } + } + return response.getBlockData(); + } + + @VisibleForTesting + protected BlockData getBlockDataUsingGRPCClient() throws IOException { + final Pipeline pipeline = xceiverClientGrpc.getPipeline(); + if (LOG.isDebugEnabled()) { LOG.debug("Initializing BlockInputStream for get key to access block {}", blockID); } GetBlockResponseProto response = ContainerProtocolCalls.getBlock( - xceiverClient, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); + xceiverClientGrpc, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); return response.getBlockData(); } @@ -332,13 +400,35 @@ private static void validate(ContainerCommandResponseProto response) } private void acquireClient() throws IOException { - if (xceiverClientFactory != null && xceiverClient == null) { - final Pipeline pipeline = pipelineRef.get(); + final Pipeline pipeline = pipelineRef.get(); + // xceiverClientGrpc not-null indicates there is fall back to GRPC reads + if (xceiverClientFactory != null && xceiverClientFactory.isShortCircuitEnabled() && !fallbackToGrpc.get() + && xceiverClientShortCircuit == null) { try { - xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline); + XceiverClientSpi newClient = xceiverClientFactory.acquireClientForReadData(pipeline, true); + if (newClient instanceof XceiverClientShortCircuit) { + xceiverClientShortCircuit = (XceiverClientShortCircuit) newClient; + if (LOG.isDebugEnabled()) { + LOG.debug("acquired short-circuit client {} for block {}", xceiverClientShortCircuit.toString(), blockID); + } + } else { + xceiverClientGrpc = newClient; + fallbackToGrpc.set(true); + } + return; + } catch (Exception e) { + LOG.warn("Failed to acquire {} client for pipeline {}, block {}. Fallback to Grpc client.", + DomainSocketFactory.FEATURE, pipeline, blockID, e); + fallbackToGrpc.set(true); + } + } + + // fall back to acquire GRPC client + if (xceiverClientFactory != null && xceiverClientGrpc == null) { + try { + xceiverClientGrpc = xceiverClientFactory.acquireClientForReadData(pipeline, false); } catch (IOException ioe) { - LOG.warn("Failed to acquire client for pipeline {}, block {}", - pipeline, blockID); + LOG.warn("Failed to acquire client for pipeline {}, block {}", pipeline, blockID); throw ioe; } } @@ -354,8 +444,13 @@ protected synchronized void addStream(ChunkInfo chunkInfo) { } protected ChunkInputStream createChunkInputStream(ChunkInfo chunkInfo) { - return new ChunkInputStream(chunkInfo, blockID, - xceiverClientFactory, pipelineRef::get, verifyChecksum, tokenRef::get); + if (blockInputStream != null) { + return new ShortCircuitChunkInputStream(chunkInfo, blockID, xceiverClientFactory, + pipelineRef::get, verifyChecksum, tokenRef::get, xceiverClientShortCircuit, blockInputStream); + } else { + return new ChunkInputStream(chunkInfo, blockID, + xceiverClientFactory, pipelineRef::get, verifyChecksum, tokenRef::get); + } } @Override @@ -527,12 +622,23 @@ public synchronized void close() { is.close(); } } + if (blockInputStream != null) { + try { + blockInputStream.close(); + } catch (IOException e) { + LOG.error("Failed to close file InputStream for block " + blockID, e); + } + } } private void releaseClient() { - if (xceiverClientFactory != null && xceiverClient != null) { - xceiverClientFactory.releaseClientForReadData(xceiverClient, false); - xceiverClient = null; + if (xceiverClientFactory != null && xceiverClientGrpc != null) { + xceiverClientFactory.releaseClientForReadData(xceiverClientGrpc, false); + xceiverClientGrpc = null; + } + if (xceiverClientFactory != null && xceiverClientShortCircuit != null) { + xceiverClientFactory.releaseClientForReadData(xceiverClientShortCircuit, false); + xceiverClientShortCircuit = null; } } @@ -567,6 +673,10 @@ synchronized long getBlockPosition() { return blockPosition; } + public synchronized FileInputStream getBlockInputStream() { + return blockInputStream; + } + @Override public synchronized void unbuffer() { storePosition(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java index 983bb74989a..8b02536d815 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java @@ -414,7 +414,7 @@ private synchronized void readChunkFromContainer(int len) throws IOException { adjustBufferPosition(startByteIndex - bufferOffsetWrtChunkData); } - private void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) + protected void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) throws IOException { buffers = readChunk(readChunkInfo); buffersSize = readChunkInfo.getLen(); diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java index e62e2a6bfd2..ee2bd62afd2 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java @@ -26,7 +26,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.Closeable; +import java.io.File; import java.io.IOException; import java.net.InetSocketAddress; import java.util.Timer; @@ -36,7 +36,7 @@ /** * A factory to help create DomainSocket. */ -public final class DomainSocketFactory implements Closeable { +public final class DomainSocketFactory { private static final Logger LOG = LoggerFactory.getLogger( DomainSocketFactory.class); @@ -138,14 +138,14 @@ public static DomainSocketFactory getInstance(ConfigurationSource conf) { private DomainSocketFactory(ConfigurationSource conf) { OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); boolean shortCircuitEnabled = clientConfig.isShortCircuitEnabled(); + domainSocketPath = conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT); PathInfo pathInfo; long startTime = System.nanoTime(); if (!shortCircuitEnabled) { LOG.info(FEATURE + " is disabled."); pathInfo = PathInfo.NOT_CONFIGURED; } else { - domainSocketPath = conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, - OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT); if (domainSocketPath.isEmpty()) { throw new IllegalArgumentException(FEATURE + " is enabled but " + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH + " is not set."); @@ -153,6 +153,12 @@ private DomainSocketFactory(ConfigurationSource conf) { LOG.warn(FEATURE + " cannot be used because " + nativeLibraryLoadFailureReason); pathInfo = PathInfo.DISABLED; } else { + File file = new File(domainSocketPath); + if (file.exists()) { + throw new IllegalArgumentException(FEATURE + " is enabled but " + + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH + " is an existing " + + (file.isDirectory() ? "directory" : "file")); + } pathInfo = PathInfo.VALID; isEnabled = true; timer = new Timer(DomainSocketFactory.class.getSimpleName() + "-Timer"); @@ -263,10 +269,14 @@ public long getPathExpireMills() { return pathExpireMills; } - @Override - public void close() { - if (timer != null) { - timer.cancel(); + public Timer getTimer() { + return timer; + } + + public static synchronized void close() { + if (instance.getTimer() != null) { + instance.getTimer().cancel(); } + DomainSocketFactory.instance = null; } } diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java new file mode 100644 index 00000000000..676aa47cd4f --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java @@ -0,0 +1,123 @@ +/* + * 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.annotations.VisibleForTesting; +import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.fs.CanUnbuffer; +import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo; +import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; +import org.apache.hadoop.hdds.scm.XceiverClientSpi.ShortCircuitValidator; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChecksumData; +import org.apache.hadoop.ozone.common.OzoneChecksumException; +import org.apache.hadoop.ozone.common.utils.BufferUtils; +import org.apache.hadoop.security.token.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.Arrays; +import java.util.List; +import java.util.function.Supplier; + +/** + * An {@link InputStream} called from BlockInputStream to read a chunk from the + * container. Each chunk may contain multiple underlying {@link ByteBuffer} + * instances. + */ +public class ShortCircuitChunkInputStream extends ChunkInputStream + implements Seekable, CanUnbuffer, ByteBufferReadable { + + private final ChunkInfo chunkInfo; + private final FileInputStream blockInputStream; + private final FileChannel dataIn; + private final ShortCircuitValidator validator; + private final XceiverClientShortCircuit xceiverClientShortCircuit; + private final boolean verifyChecksum; + public static final Logger LOG = + LoggerFactory.getLogger(ShortCircuitChunkInputStream.class); + + @SuppressWarnings("checkstyle:parameternumber") + ShortCircuitChunkInputStream(ChunkInfo chunkInfo, BlockID blockId, XceiverClientFactory xceiverClientFactory, + Supplier pipelineSupplier, boolean verifyChecksum, Supplier> tokenSupplier, + XceiverClientShortCircuit xceiverClientShortCircuit, FileInputStream blockInputStream) { + super(chunkInfo, blockId, xceiverClientFactory, pipelineSupplier, verifyChecksum, tokenSupplier); + this.chunkInfo = chunkInfo; + this.blockInputStream = blockInputStream; + this.dataIn = blockInputStream.getChannel(); + this.xceiverClientShortCircuit = xceiverClientShortCircuit; + this.validator = this::validateChunk; + this.verifyChecksum = verifyChecksum; + if (LOG.isDebugEnabled()) { + LOG.debug("{} is created for {}", ShortCircuitChunkInputStream.class.getSimpleName(), blockId); + } + } + + /** + * Send RPC call to get the chunk from the container. + */ + @VisibleForTesting + @Override + protected ByteBuffer[] readChunk(ChunkInfo readChunkInfo) + throws IOException { + int bytesPerChecksum = chunkInfo.getChecksumData().getBytesPerChecksum(); + final ByteBuffer[] buffers = BufferUtils.assignByteBuffers(readChunkInfo.getLen(), + bytesPerChecksum); + dataIn.position(readChunkInfo.getOffset()).read(buffers); + Arrays.stream(buffers).forEach(ByteBuffer::flip); + validator.accept(Arrays.asList(buffers), readChunkInfo); + return buffers; + } + + private void validateChunk(List bufferList, ChunkInfo readChunkInfo) + throws OzoneChecksumException { + if (verifyChecksum) { + ChecksumData checksumData = ChecksumData.getFromProtoBuf( + chunkInfo.getChecksumData()); + + // ChecksumData stores checksum for each 'numBytesPerChecksum' + // number of bytes in a list. Compute the index of the first + // checksum to match with the read data + + long relativeOffset = readChunkInfo.getOffset() - + chunkInfo.getOffset(); + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + int startIndex = (int) (relativeOffset / bytesPerChecksum); + Checksum.verifyChecksum(bufferList, checksumData, startIndex); + } + } + + + /** + * Acquire short-circuit local read client. + */ + @Override + protected synchronized void acquireClient() throws IOException { + // do nothing, read data doesn't need short-circuit client + } +} diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java index e35d20d53e1..a5c313073ba 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/DatanodeVersion.java @@ -34,6 +34,8 @@ public enum DatanodeVersion implements ComponentVersion { COMBINED_PUTBLOCK_WRITECHUNK_RPC(2, "WriteChunk can optionally support " + "a PutBlock request"), + SHORT_CIRCUIT_READS(3, "Version with short-circuit read support."), + FUTURE_VERSION(-1, "Used internally in the client when the server side is " + " newer and an unknown server version has arrived to the client."); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java index 36d4d90e1af..d0e2e6e8d09 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/RatisReplicationConfig.java @@ -23,6 +23,8 @@ import net.jcip.annotations.Immutable; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Objects; @@ -45,6 +47,8 @@ public final class RatisReplicationConfig private static final RatisReplicationConfig RATIS_THREE_CONFIG = new RatisReplicationConfig(THREE); + private static final Logger LOG = + LoggerFactory.getLogger(RatisReplicationConfig.class); /** * Get an instance of Ratis Replication Config with the requested factor. * The same static instance will be returned for all requests for the same diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java index 9ac32c469ca..ddf1e2df733 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientSpi.java @@ -20,6 +20,7 @@ import java.io.Closeable; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.List; import java.util.Map; import java.util.concurrent.CompletableFuture; @@ -28,6 +29,7 @@ import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; @@ -50,6 +52,15 @@ public interface Validator extends // just a shortcut to avoid having to repeat long list of generic parameters } + /** + * Validator for container read chunk through short-circuit local reads. + */ + public interface ShortCircuitValidator extends + CheckedBiConsumer, ContainerProtos.ChunkInfo, IOException> { + // just a shortcut to avoid having to repeat long list of generic parameters + } + + private final AtomicInteger referenceCount; private boolean isEvicted; @@ -94,6 +105,10 @@ public int getRefcount() { @Override public abstract void close(); + public boolean isClosed() { + return false; + } + /** * Returns the pipeline of machines that host the container used by this * client. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java index 6c5b4aff57f..ae5fdcb62d1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java @@ -524,9 +524,9 @@ public String toString() { b.append(" ReplicaIndex: ").append(this.getReplicaIndex(datanodeDetails)); } b.append(", ReplicationConfig: ").append(replicationConfig); - b.append(", State:").append(getPipelineState()); - b.append(", leaderId:").append(leaderId != null ? leaderId.toString() : ""); - b.append(", CreationTimestamp").append(getCreationTimestamp() + b.append(", State: ").append(getPipelineState()); + b.append(", leaderId: ").append(leaderId != null ? leaderId.toString() : ""); + b.append(", CreationTimestamp: ").append(getCreationTimestamp() .atZone(ZoneId.systemDefault())); b.append("]"); return b.toString(); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index d3f39c023b7..5dbfdc8fd7b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java @@ -65,11 +65,15 @@ public final class ContainerCommandResponseBuilders { public static Builder getContainerCommandResponse( ContainerCommandRequestProto request, Result result, String message) { - return ContainerCommandResponseProto.newBuilder() + ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setCmdType(request.getCmdType()) .setTraceID(request.getTraceID()) .setResult(result) .setMessage(message); + if (request.hasClientId() && request.hasCallId()) { + builder.setClientId(request.getClientId()).setCallId(request.getCallId()); + } + return builder; } /** @@ -82,10 +86,14 @@ public static Builder getContainerCommandResponse( public static Builder getSuccessResponseBuilder( ContainerCommandRequestProto request) { - return ContainerCommandResponseProto.newBuilder() + ContainerCommandResponseProto.Builder builder = ContainerCommandResponseProto.newBuilder() .setCmdType(request.getCmdType()) .setTraceID(request.getTraceID()) .setResult(Result.SUCCESS); + if (request.hasClientId() && request.hasCallId()) { + builder.setClientId(request.getClientId()).setCallId(request.getCallId()); + } + return builder; } /** @@ -149,10 +157,10 @@ public static ContainerCommandResponseProto putBlockResponseSuccess( } public static ContainerCommandResponseProto getBlockDataResponse( - ContainerCommandRequestProto msg, BlockData data) { + ContainerCommandRequestProto msg, BlockData data, boolean shortCircuitGranted) { GetBlockResponseProto.Builder getBlock = GetBlockResponseProto.newBuilder() - .setBlockData(data); + .setBlockData(data).setShortCircuitAccessGranted(shortCircuitGranted); return getSuccessResponseBuilder(msg) .setGetBlock(getBlock) @@ -365,9 +373,7 @@ public static ContainerCommandResponseProto getEchoResponse( .newBuilder() .setPayload(UnsafeByteOperations.unsafeWrap(RandomUtils.nextBytes(responsePayload))); - return getSuccessResponseBuilder(msg) - .setEcho(echo) - .build(); + return getSuccessResponseBuilder(msg).setEcho(echo).build(); } private ContainerCommandResponseBuilders() { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java index cb41479b5f3..051697cf9e8 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/storage/ContainerProtocolCalls.java @@ -239,6 +239,19 @@ private static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, return response.getGetBlock(); } + public static GetBlockResponseProto getBlock(XceiverClientSpi xceiverClient, + List validators, ContainerCommandRequestProto.Builder builder, + DatanodeDetails datanode) throws IOException { + String traceId = TracingUtil.exportCurrentSpan(); + if (traceId != null) { + builder.setTraceID(traceId); + } + final ContainerCommandRequestProto request = builder.setDatanodeUuid(datanode.getUuidString()).build(); + ContainerCommandResponseProto response = xceiverClient.sendCommand(request, validators); + return response.getGetBlock(); + } + + /** * Calls the container protocol to get the length of a committed block. * @@ -738,6 +751,19 @@ public static GetSmallFileResponseProto readSmallFile(XceiverClientSpi client, public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID, long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly) throws IOException { + return echo(client, encodedContainerID, containerID, payloadReqBytes, payloadRespSizeKB, + sleepTimeMs, readOnly, null, 0, false); + } + + /** + * Send an echo to DataNode with clientId and callId in request. + * + * @return EchoResponseProto + */ + @SuppressWarnings("checkstyle:parameternumber") + public static EchoResponseProto echo(XceiverClientSpi client, String encodedContainerID, + long containerID, ByteString payloadReqBytes, int payloadRespSizeKB, int sleepTimeMs, boolean readOnly, + ByteString clientId, long callID, boolean noValidation) throws IOException { ContainerProtos.EchoRequestProto getEcho = EchoRequestProto .newBuilder() @@ -754,6 +780,9 @@ public static EchoResponseProto echo(XceiverClientSpi client, String encodedCont .setContainerID(containerID) .setDatanodeUuid(id) .setEcho(getEcho); + if (clientId != null) { + builder.setClientId(clientId).setCallId(callID); + } if (!encodedContainerID.isEmpty()) { builder.setEncodedToken(encodedContainerID); } @@ -763,7 +792,7 @@ public static EchoResponseProto echo(XceiverClientSpi client, String encodedCont } ContainerCommandRequestProto request = builder.build(); ContainerCommandResponseProto response = - client.sendCommand(request, getValidatorList()); + client.sendCommand(request, noValidation ? new ArrayList<>() : getValidatorList()); return response.getEcho(); } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java index f4efe1790bf..d598aea4ca9 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/upgrade/HDDSLayoutFeature.java @@ -43,7 +43,8 @@ public enum HDDSLayoutFeature implements LayoutFeature { HADOOP_PRC_PORTS_IN_DATANODEDETAILS(7, "Adding Hadoop RPC ports " + "to DatanodeDetails."), HBASE_SUPPORT(8, "Datanode RocksDB Schema Version 3 has an extra table " + - "for the last chunk of blocks to support HBase.)"); + "for the last chunk of blocks to support HBase.)"), + SHORT_CIRCUIT_READS(9, "Short-circuit read support"); ////////////////////////////// ////////////////////////////// diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java index 5719803b94c..c5b375646d2 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java @@ -471,8 +471,12 @@ public final class OzoneConfigKeys { public static final int OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE = 8 * 1024; public static final String OZONE_CLIENT_READ_TIMEOUT - = "ozone.client.read.timeout"; + = "ozone.client.read.timeout"; public static final String OZONE_CLIENT_READ_TIMEOUT_DEFAULT = "30s"; + public static final String OZONE_CLIENT_WRITE_TIMEOUT + = "ozone.client.write.timeout"; + public static final String OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT = "30s"; + public static final String OZONE_ACL_AUTHORIZER_CLASS = "ozone.acl.authorizer.class"; public static final String OZONE_ACL_AUTHORIZER_CLASS_DEFAULT = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java index f8b3febfeca..190b7a6cfaa 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/common/Checksum.java @@ -296,6 +296,20 @@ public static boolean verifyChecksum(List byteStrings, return checksumData.verifyChecksumDataMatches(computed, startIndex); } + public static boolean verifyChecksum(List bufferList, ChecksumData checksumData, + int startIndex) throws OzoneChecksumException { + ChecksumType checksumType = checksumData.getChecksumType(); + if (checksumType == ChecksumType.NONE) { + // Checksum is set to NONE. No further verification is required. + return true; + } + int bytesPerChecksum = checksumData.getBytesPerChecksum(); + Checksum checksum = new Checksum(checksumType, bytesPerChecksum); + final ChecksumData computed = checksum.computeChecksum( + ChunkBuffer.wrap(bufferList)); + return checksumData.verifyChecksumDataMatches(computed, startIndex); + } + /** * Returns a ChecksumData with type NONE for testing. */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java index 855153f39ed..4f449d5b69e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/util/OzoneNetUtils.java @@ -24,6 +24,9 @@ import java.net.InetAddress; import java.net.InetSocketAddress; import java.security.Security; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_JVM_NETWORK_ADDRESS_CACHE_ENABLED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_JVM_NETWORK_ADDRESS_CACHE_ENABLED_DEFAULT; @@ -89,16 +92,32 @@ private static String getHostNameWithoutDomain(final String fqdn) { } + private static final Map CACHE = + Collections.synchronizedMap(new HashMap()); /** * Match input address to local address. - * Return true if it matches, false otherwsie. + * Return true if it matches, false otherwise. */ public static boolean isAddressLocal(InetSocketAddress addr) { InetAddress inetAddress = addr.getAddress(); - return inetAddress != null && NetUtils.isLocalAddress(inetAddress); + if (inetAddress == null) { + return false; + } + Boolean cached = CACHE.get(inetAddress.getHostAddress()); + if (cached != null) { + if (LOG.isDebugEnabled()) { + LOG.debug("Address {} is {} local", addr, (cached ? "" : "not")); + } + return cached; + } + boolean local = NetUtils.isLocalAddress(inetAddress); + if (LOG.isDebugEnabled()) { + LOG.debug("Address {} is {} local", addr, (local ? "" : "not")); + } + CACHE.put(inetAddress.getHostAddress(), local); + return local; } - public static boolean isUnresolved(boolean flexibleFqdnResolutionEnabled, InetSocketAddress addr) { return !flexibleFqdnResolutionEnabled && addr.isUnresolved() diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java index c1b65e10bc9..bc05a15a034 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/utils/TestProtobufUtils.java @@ -17,8 +17,10 @@ */ package org.apache.hadoop.hdds.utils; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.ozone.util.ProtobufUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.InvalidProtocolBufferException; import org.junit.jupiter.api.Test; import java.util.UUID; @@ -26,6 +28,7 @@ import static org.apache.hadoop.ozone.util.ProtobufUtils.fromProtobuf; import static org.apache.hadoop.ozone.util.ProtobufUtils.toProtobuf; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Test-cases for {@link ProtobufUtils}. @@ -46,4 +49,37 @@ public void testUuidConversion() { UUID deserialized = fromProtobuf(protobuf); assertEquals(original, deserialized); } + + @Test + public void testContainerCommandRequestProtoConversion() throws InvalidProtocolBufferException { + long containerID = 1L; + long localBlockID = 2L; + long bcsid = 3L; + String datanodeID = UUID.randomUUID().toString(); + ContainerProtos.DatanodeBlockID.Builder blkIDBuilder = + ContainerProtos.DatanodeBlockID.newBuilder().setContainerID(containerID) + .setLocalID(localBlockID) + .setBlockCommitSequenceId(bcsid); + ContainerProtos.GetBlockRequestProto.Builder readBlockRequest = + ContainerProtos.GetBlockRequestProto.newBuilder().setBlockID(blkIDBuilder.build()); + + ContainerProtos.ContainerCommandRequestProto.Builder builder = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.GetBlock) + .setContainerID(containerID) + .setDatanodeUuid(datanodeID) + .setGetBlock(readBlockRequest.build()); + + ContainerProtos.ContainerCommandRequestProto request = builder.build(); + byte[] requestInBytes = request.toByteArray(); + + request = ContainerProtos.ContainerCommandRequestProto.parseFrom(requestInBytes); + assertTrue(request.hasGetBlock()); + assertEquals(ContainerProtos.Type.GetBlock, request.getCmdType()); + assertEquals(containerID, request.getContainerID()); + assertEquals(datanodeID, request.getDatanodeUuid()); + assertEquals(localBlockID, request.getGetBlock().getBlockID().getLocalID()); + assertEquals(containerID, request.getGetBlock().getBlockID().getContainerID()); + assertEquals(bcsid, request.getGetBlock().getBlockID().getBlockCommitSequenceId()); + } } diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java index 2b7592e1c35..952396084aa 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/ozone/container/ContainerTestHelper.java @@ -632,7 +632,7 @@ public static ContainerCommandRequestProto getDummyCommandRequestProto( break; case GetBlock: builder.setGetBlock(ContainerProtos.GetBlockRequestProto.newBuilder() - .setBlockID(fakeBlockId).build()); + .setBlockID(fakeBlockId).setRequestShortCircuitAccess(true).build()); break; case GetCommittedBlockLength: builder.setGetCommittedBlockLength( diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java index 46887441f0a..27da6937745 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/helpers/ContainerMetrics.java @@ -33,7 +33,9 @@ import org.apache.hadoop.ozone.util.MetricUtil; import java.io.Closeable; +import java.util.ArrayList; import java.util.EnumMap; +import java.util.List; /** * @@ -59,16 +61,20 @@ public class ContainerMetrics implements Closeable { @Metric private MutableCounterLong numReadStateMachine; @Metric private MutableCounterLong bytesReadStateMachine; - + /** for remote requests. */ private final EnumMap numOpsArray; private final EnumMap opsBytesArray; private final EnumMap opsLatency; private final EnumMap opsLatQuantiles; + /** for local short-circuit requests. */ + private final EnumMap numLocalOpsArray; + private final EnumMap opsLocalBytesArray; + private final EnumMap opsLocalLatencyNs; + private final EnumMap opsLocalInQueueLatencyNs; private MetricsRegistry registry = null; public ContainerMetrics(int[] intervals) { final int len = intervals.length; - MutableQuantiles[] latQuantiles = new MutableQuantiles[len]; this.numOpsArray = new EnumMap<>(ContainerProtos.Type.class); this.opsBytesArray = new EnumMap<>(ContainerProtos.Type.class); this.opsLatency = new EnumMap<>(ContainerProtos.Type.class); @@ -82,6 +88,7 @@ public ContainerMetrics(int[] intervals) { "bytes" + type, "bytes used by " + type + "op", (long) 0)); opsLatency.put(type, registry.newRate("latencyNs" + type, type + " op")); + MutableQuantiles[] latQuantiles = new MutableQuantiles[len]; for (int j = 0; j < len; j++) { int interval = intervals[j]; String quantileName = type + "Nanos" + interval + "s"; @@ -90,6 +97,23 @@ public ContainerMetrics(int[] intervals) { } opsLatQuantiles.put(type, latQuantiles); } + + this.numLocalOpsArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalBytesArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalLatencyNs = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalInQueueLatencyNs = new EnumMap<>(ContainerProtos.Type.class); + + List localTypeList = new ArrayList<>(); + localTypeList.add(ContainerProtos.Type.GetBlock); + localTypeList.add(ContainerProtos.Type.Echo); + for (ContainerProtos.Type type : localTypeList) { + numLocalOpsArray.put(type, registry.newCounter( + "numLocal" + type, "number of " + type + " ops", (long) 0)); + opsLocalBytesArray.put(type, registry.newCounter( + "localBytes" + type, "bytes used by " + type + "op", (long) 0)); + opsLocalLatencyNs.put(type, registry.newRate("localLatencyNs" + type, type + " op")); + opsLocalInQueueLatencyNs.put(type, registry.newRate("localInQueueLatencyNs" + type, type + " op")); + } } public static ContainerMetrics create(ConfigurationSource conf) { @@ -128,6 +152,27 @@ public void incContainerBytesStats(ContainerProtos.Type type, long bytes) { opsBytesArray.get(type).incr(bytes); } + public void incContainerLocalOpsMetrics(ContainerProtos.Type type) { + numOps.incr(); + numLocalOpsArray.get(type).incr(); + } + + public long getContainerLocalOpsMetrics(ContainerProtos.Type type) { + return numLocalOpsArray.get(type).value(); + } + + public void incContainerLocalOpsLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalLatencyNs.get(type).add(nanoSeconds); + } + + public void incContainerLocalOpsInQueueLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalInQueueLatencyNs.get(type).add(nanoSeconds); + } + + public void incContainerLocalBytesStats(ContainerProtos.Type type, long bytes) { + opsLocalBytesArray.get(type).incr(bytes); + } + public void incContainerDeleteFailedBlockCountNotZero() { containerDeleteFailedBlockCountNotZero.incr(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java index bfdff69be46..fc977bcfd87 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/interfaces/Handler.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.container.common.interfaces; +import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -37,6 +38,7 @@ import org.apache.hadoop.ozone.container.common.volume.VolumeSet; import org.apache.hadoop.ozone.container.keyvalue.KeyValueHandler; import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.ratis.statemachine.StateMachine; import static org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult; @@ -73,11 +75,19 @@ public static Handler getHandlerForContainerType( final String datanodeId, final ContainerSet contSet, final VolumeSet volumeSet, final ContainerMetrics metrics, IncrementalReportSender icrSender) { + return getHandlerForContainerType(containerType, config, datanodeId, contSet, volumeSet, metrics, icrSender, null); + } + + @SuppressWarnings("checkstyle:parameternumber") + public static Handler getHandlerForContainerType( + final ContainerType containerType, final ConfigurationSource config, + final String datanodeId, final ContainerSet contSet, + final VolumeSet volumeSet, final ContainerMetrics metrics, + IncrementalReportSender icrSender, OzoneContainer ozoneContainer) { switch (containerType) { case KeyValueContainer: - return new KeyValueHandler(config, - datanodeId, contSet, volumeSet, metrics, - icrSender); + return new KeyValueHandler(config, datanodeId, contSet, volumeSet, metrics, + icrSender, ozoneContainer); default: throw new IllegalArgumentException("Handler for ContainerType: " + containerType + "doesn't exist."); @@ -221,4 +231,6 @@ public void setClusterID(String clusterID) { this.clusterId = clusterID; } + public abstract FileInputStream getBlockInputStream(ContainerCommandRequestProto request) + throws IOException; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java new file mode 100644 index 00000000000..48145a716b2 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java @@ -0,0 +1,370 @@ +/** + * 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.container.common.transport.server; + +import io.opentracing.Scope; +import io.opentracing.Span; +import io.opentracing.util.GlobalTracer; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainPeer; +import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.protobuf.CodedInputStream; +import org.apache.hadoop.util.LimitInputStream; +import org.slf4j.Logger; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.FileDescriptor; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.io.OutputStream; +import java.nio.channels.ClosedChannelException; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION; +import static org.apache.hadoop.hdds.scm.protocolPB.ContainerCommandResponseBuilders.getContainerCommandResponse; + +/** + * Class for processing incoming/outgoing requests. + */ +final class Receiver implements Runnable { + public static final Logger LOG = LoggerFactory.getLogger(Receiver.class); + + private DomainPeer peer; + private final String remoteAddress; // address of remote side + private final String remoteAddressWithoutPort; // only the address, no port + private final String localAddress; // local address of this daemon + private final XceiverServerDomainSocket domainSocketServer; + private final ContainerDispatcher dispatcher; + private final ContainerMetrics metrics; + private final InputStream socketIn; + private OutputStream socketOut; + private final int bufferSize; + private final ThreadPoolExecutor readExecutors; + private DataInputStream input; + private DataOutputStream output; + + public static Receiver create(DomainPeer peer, ConfigurationSource conf, XceiverServerDomainSocket server, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, ContainerMetrics metrics) throws IOException { + return new Receiver(peer, conf, server, dispatcher, executor, metrics); + } + + private Receiver(DomainPeer peer, ConfigurationSource conf, XceiverServerDomainSocket server, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, ContainerMetrics metrics) throws IOException { + this.peer = peer; + this.socketIn = peer.getInputStream(); + this.socketOut = peer.getOutputStream(); + this.domainSocketServer = server; + this.dispatcher = dispatcher; + this.readExecutors = executor; + this.metrics = metrics; + this.bufferSize = conf.getObject(OzoneClientConfig.class).getShortCircuitBufferSize(); + remoteAddress = peer.getRemoteAddressString(); + localAddress = peer.getLocalAddressString(); + final int colonIdx = remoteAddress.indexOf(':'); + remoteAddressWithoutPort = (colonIdx < 0) ? remoteAddress : remoteAddress.substring(0, colonIdx); + } + + @Override + public void run() { + long opsReceived = 0; + final AtomicLong opsHandled = new AtomicLong(0); + TaskEntry entry = null; + try { + domainSocketServer.addPeer(peer, Thread.currentThread(), this); + input = new DataInputStream(new BufferedInputStream(socketIn, bufferSize)); + output = new DataOutputStream(new BufferedOutputStream(socketOut, bufferSize)); + + // We process requests in a loop, and stay around for a short timeout. + // This optimistic behaviour allows the other end to reuse connections. + // Setting keepalive timeout to 0 disable this behavior. + do { + try { + entry = readRequest(input); + } catch (InterruptedIOException ignored) { + // Time out while we wait for client rpc + throw ignored; + } catch (EOFException | ClosedChannelException e) { + // Since we optimistically expect the next request, it's quite normal to + // get EOF here. + LOG.warn("{} is closed with {} after received {} ops and handled {} ops.", + peer, e.getClass().getName(), opsReceived, opsHandled.get()); + throw e; + } + + readExecutors.submit(new ProcessRequestTask(entry, opsHandled)); + ++opsReceived; + // reset request variable + entry = null; + } while (peer != null && !peer.isClosed()); + } catch (Throwable t) { + String s = "Receiver error" + + ((entry == null) ? ", " : "processing " + entry.getRequest().getCmdType() + " operation, ") + + "src: " + remoteAddress + " dst: " + localAddress; + LOG.warn(s, t); + } finally { + if (peer != null) { + try { + domainSocketServer.closePeer(peer); + } catch (IOException e) { + LOG.warn("Failed to close peer {}", peer, e); + } + } + if (input != null) { + IOUtils.closeStream(input); + } + if (output != null) { + IOUtils.closeStream(output); + } + } + } + + /** Read the request. **/ + private TaskEntry readRequest(DataInputStream in) throws IOException { + // first short is DATA_TRANSFER_VERSION + final short version = in.readShort(); + if (version != DATA_TRANSFER_VERSION) { + throw new IOException("Version Mismatch (Expected: " + + DATA_TRANSFER_VERSION + ", Received: " + version + " )"); + } + long startTime = System.nanoTime(); + // second short is ContainerProtos#Type + final short typeNumber = in.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + + ContainerCommandRequestProto requestProto = + ContainerCommandRequestProto.parseFrom(vintPrefixed(in)); + if (requestProto.getCmdType() != type) { + throw new IOException("Type mismatch, " + type + " in header while " + requestProto.getCmdType() + + " in request body"); + } + TaskEntry entry = new TaskEntry(requestProto, startTime); + return entry; + } + + public static InputStream vintPrefixed(final InputStream input) + throws IOException { + final int firstByte = input.read(); + if (firstByte == -1) { + throw new EOFException( + "Unexpected EOF while trying to read response from server"); + } + + int size = CodedInputStream.readRawVarint32(firstByte, input); + assert size >= 0; + return new LimitInputStream(input, size); + } + + /** Process the request. **/ + public class ProcessRequestTask implements Runnable { + private final TaskEntry entry; + private final ContainerCommandRequestProto request; + private final AtomicLong counter; + + ProcessRequestTask(TaskEntry entry, AtomicLong counter) { + this.entry = entry; + this.request = entry.getRequest(); + this.counter = counter; + this.entry.setInQueueStartTimeNs(); + } + + @Override + public void run() { + entry.setOutQueueStartTimeNs(); + ContainerProtos.Type type = request.getCmdType(); + if (isSupportedCmdType(type)) { + metrics.incContainerLocalOpsMetrics(type); + metrics.incContainerLocalOpsInQueueLatencies(type, entry.getInQueueTimeNs()); + } + Span span = TracingUtil.importAndCreateSpan("XceiverServerDomainSocket." + type.name(), + request.getTraceID()); + try (Scope scope = GlobalTracer.get().activateSpan(span)) { + ContainerCommandResponseProto responseProto; + if (isSupportedCmdType(type)) { + responseProto = dispatcher.dispatch(request, null); + } else { + responseProto = getContainerCommandResponse(request, ContainerProtos.Result.UNSUPPORTED_REQUEST, + "This command is not supported through DomainSocket channel.") + .build(); + } + if (responseProto.getResult() == ContainerProtos.Result.SUCCESS && type == ContainerProtos.Type.GetBlock) { + // get FileDescriptor + Handler handler = dispatcher.getHandler(ContainerProtos.ContainerType.KeyValueContainer); + FileInputStream fis = handler.getBlockInputStream(request); + Preconditions.checkNotNull(fis, + "Failed to get block InputStream for block " + request.getGetBlock().getBlockID()); + entry.setFis(fis); + } + entry.setResponse(responseProto); + sendResponse(entry); + } catch (Throwable e) { + LOG.error("Failed to processRequest {} {} {}", type, request.getClientId(), request.getCallId(), e); + } finally { + span.finish(); + LOG.info("before counter is incremented {}", counter); + counter.incrementAndGet(); + LOG.info("counter is incremented {}", counter); + } + } + } + + void sendResponse(TaskEntry entry) { + entry.setSendStartTimeNs(); + byte[] buf = new byte[1]; + buf[0] = DATA_TRANSFER_MAGIC_CODE; + ContainerCommandResponseProto responseProto = entry.getResponse(); + ContainerProtos.Type type = responseProto.getCmdType(); + synchronized (output) { + FileInputStream fis = entry.getFis(); + try { + output.writeShort(DATA_TRANSFER_VERSION); + output.writeShort(type.getNumber()); + responseProto.writeDelimitedTo(output); + output.flush(); + if (fis != null) { + // send FileDescriptor + FileDescriptor[] fds = new FileDescriptor[1]; + fds[0] = fis.getFD(); + DomainSocket sock = peer.getDomainSocket(); + // this API requires send at least one byte buf. + sock.sendFileDescriptors(fds, buf, 0, buf.length); + if (LOG.isDebugEnabled()) { + LOG.info("send fd {}", fis.getFD()); + } + } + } catch (Throwable e) { + LOG.error("Failed to send response {}", responseProto.getCmdType(), e); + } finally { + entry.setSendFinishTimeNs(); + if (fis != null) { + try { + fis.close(); + LOG.info("fis {} for {} is closed", fis.getFD(), + responseProto.getClientId().toStringUtf8() + "-" + responseProto.getCallId()); + } catch (IOException e) { + LOG.warn("Failed to close {}", fis, e); + } + } + if (LOG.isDebugEnabled()) { + LOG.debug("handle request {} callId {} clientId {}, receive {} ns, in queue {} ns, " + + " handle {} ns, send out {} ns, total {} ns", type, responseProto.getCallId(), + responseProto.getClientId().toStringUtf8(), entry.getReceiveTimeNs(), entry.getInQueueTimeNs(), + entry.getProcessTimeNs(), entry.getSendTimeNs(), entry.getTotalTimeNs()); + } + if (isSupportedCmdType(type)) { + metrics.incContainerLocalOpsLatencies(type, entry.getTotalTimeNs()); + } + } + } + } + + private boolean isSupportedCmdType(ContainerProtos.Type type) { + return type == ContainerProtos.Type.GetBlock || type == ContainerProtos.Type.Echo; + } + + static class TaskEntry { + private ContainerCommandRequestProto request; + private ContainerCommandResponseProto response; + private FileInputStream fis; + private long receiveStartTimeNs; + private long inQueueStartTimeNs; + private long outQueueStartTimeNs; + private long sendStartTimeNs; + private long sendFinishTimeNs; + + TaskEntry(ContainerCommandRequestProto requestProto, long startTimeNs) { + this.request = requestProto; + this.receiveStartTimeNs = startTimeNs; + } + + public ContainerCommandResponseProto getResponse() { + return response; + } + + public FileInputStream getFis() { + return fis; + } + + public ContainerCommandRequestProto getRequest() { + return request; + } + + public void setInQueueStartTimeNs() { + inQueueStartTimeNs = System.nanoTime(); + } + + public void setOutQueueStartTimeNs() { + outQueueStartTimeNs = System.nanoTime(); + } + + public long getReceiveTimeNs() { + return inQueueStartTimeNs - receiveStartTimeNs; + } + + public long getInQueueTimeNs() { + return outQueueStartTimeNs - inQueueStartTimeNs; + } + + public long getProcessTimeNs() { + return sendStartTimeNs - outQueueStartTimeNs; + } + + public long getSendTimeNs() { + return sendFinishTimeNs - sendStartTimeNs; + } + + public void setResponse(ContainerCommandResponseProto responseProto) { + this.response = responseProto; + } + + public void setFis(FileInputStream is) { + this.fis = is; + } + + public void setSendStartTimeNs() { + this.sendStartTimeNs = System.nanoTime(); + } + + public void setSendFinishTimeNs() { + this.sendFinishTimeNs = System.nanoTime(); + } + + public long getTotalTimeNs() { + return this.sendFinishTimeNs - this.receiveStartTimeNs; + } + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java new file mode 100644 index 00000000000..715db66d0d8 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java @@ -0,0 +1,316 @@ +/* + * 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.container.common.transport.server; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import org.apache.commons.io.IOUtils; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReport; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainPeer; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.utils.HddsServerUtil; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; +import org.apache.hadoop.util.Daemon; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.nio.channels.AsynchronousCloseException; +import java.util.HashMap; +import java.util.List; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Creates a DomainSocket server endpoint that acts as the communication layer for Ozone containers. + */ +public final class XceiverServerDomainSocket implements XceiverServerSpi, Runnable { + public static final Logger LOG = LoggerFactory.getLogger(XceiverServerDomainSocket.class); + private int port; + private Daemon server; + private ContainerDispatcher dispatcher; + private ContainerMetrics metrics; + private final AtomicBoolean isRunning = new AtomicBoolean(false); + + /** + * Maximal number of concurrent readers per node. + * Enforcing the limit is required in order to avoid data-node + * running out of memory. + */ + private final int maxXceiverCount; + private final AtomicInteger xceriverCount; + private DomainSocket domainSocket; + private final ConfigurationSource config; + private final String threadPrefix; + private final HashMap peers = new HashMap<>(); + private final HashMap peersReceiver = new HashMap<>(); + private int readTimeoutMs; + private int writeTimeoutMs; + private final ThreadPoolExecutor readExecutors; + + /** + * Constructs a DomainSocket server class, used to listen for requests from local clients. + */ + public XceiverServerDomainSocket(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, + ContainerMetrics metrics, DomainSocketFactory domainSocketFactory) { + Preconditions.checkNotNull(conf); + this.port = conf.getInt(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT, + OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT); + if (conf.getBoolean(OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT, + OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT_DEFAULT)) { + this.port = 0; + } + this.config = conf; + final int threadCountPerDisk = + conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = HddsServerUtil.getDatanodeStorageDirs(conf).size(); + this.maxXceiverCount = threadCountPerDisk * numberOfDisks * 5; + this.xceriverCount = new AtomicInteger(0); + this.dispatcher = dispatcher; + this.readExecutors = executor; + this.metrics = metrics; + LOG.info("Max allowed {} xceiver", maxXceiverCount); + this.threadPrefix = datanodeDetails.threadNamePrefix() + XceiverServerDomainSocket.class.getSimpleName(); + + if (domainSocketFactory.isServiceEnabled() && domainSocketFactory.isServiceReady()) { + this.readTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + this.writeTimeoutMs = (int) config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT, + OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); + try { + domainSocket = DomainSocket.bindAndListen( + DomainSocket.getEffectivePath(conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT), port)); + OzoneClientConfig ozoneClientConfig = conf.getObject(OzoneClientConfig.class); + domainSocket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, readTimeoutMs); + domainSocket.setAttribute(DomainSocket.SEND_TIMEOUT, writeTimeoutMs); + LOG.info("UNIX domain socket {} is created: {}, timeout for read {} ms, timeout for write {} ms, " + + "send/receive buffer {} bytes", domainSocket, domainSocket.getPath(), readTimeoutMs, writeTimeoutMs, + ozoneClientConfig.getShortCircuitBufferSize()); + } catch (IOException e) { + LOG.warn("Although short-circuit local reads are configured, we cannot " + + "enable the short circuit read because DomainSocket operation failed", e); + domainSocket = null; + throw new IllegalArgumentException(e); + } + } + } + + @Override + public int getIPCPort() { + return this.port; + } + + /** + * Returns the Replication type supported by this end-point. + * + * @return enum -- {Stand_Alone, Ratis, Grpc, Chained} + */ + @Override + public HddsProtos.ReplicationType getServerType() { + return HddsProtos.ReplicationType.STAND_ALONE; + } + + @Override + public void start() throws IOException { + if (isRunning.compareAndSet(false, true)) { + if (domainSocket != null) { + this.server = new Daemon(this); + this.server.setName(threadPrefix); + this.server.start(); + LOG.info("Listening on UNIX domain socket: {}", domainSocket.getPath()); + isRunning.set(true); + } else { + LOG.warn("Cannot start XceiverServerDomainSocket because domainSocket is null"); + } + } else { + LOG.info("UNIX domain socket server listening on {} is already stopped", domainSocket.getPath()); + } + } + + @Override + public void stop() { + if (isRunning.compareAndSet(true, false)) { + if (server != null) { + try { + if (domainSocket != null) { + // TODO: once HADOOP-19261 is merged, change it to domainSocket.close(true); + domainSocket.close(true); + LOG.info("UNIX domain socket server listening on {} is stopped", domainSocket.getPath()); + } + } catch (IOException e) { + LOG.error("Failed to force close DomainSocket", e); + } + server.interrupt(); + try { + server.join(); + } catch (InterruptedException e) { + LOG.error("Failed to shutdown XceiverServerDomainSocket", e); + Thread.currentThread().interrupt(); + } + } + } else { + LOG.info("UNIX domain socket server listening on {} is already stopped", domainSocket.getPath()); + } + } + + @Override + public boolean isStarted() { + return isRunning.get(); + } + + @Override + public void submitRequest(ContainerCommandRequestProto request, + HddsProtos.PipelineID pipelineID) throws IOException { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public boolean isExist(HddsProtos.PipelineID pipelineId) { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public List getPipelineReport() { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + @Override + public void run() { + while (isRunning.get()) { + DomainPeer peer = null; + try { + DomainSocket connSock = domainSocket.accept(); + peer = new DomainPeer(connSock); + peer.setReadTimeout(readTimeoutMs); + peer.setWriteTimeout(writeTimeoutMs); + LOG.info("Accepted a new connection. xceriverCount {}", xceriverCount.get()); + + // Make sure the xceiver count is not exceeded + if (xceriverCount.get() >= maxXceiverCount) { + throw new IOException("Xceiver count exceeds the limit " + maxXceiverCount); + } + Daemon daemon = new Daemon(Receiver.create(peer, config, this, dispatcher, readExecutors, metrics)); + daemon.setName(threadPrefix + "@" + peer.getDomainSocket().toString()); + daemon.start(); + } catch (SocketTimeoutException ignored) { + // wake up to see if should continue to run + } catch (AsynchronousCloseException ace) { + // another thread closed our listener socket - that's expected during shutdown, but not in other circumstances + LOG.info("XceiverServerDomainSocket is closed", ace); + } catch (IOException ie) { + // usually when the xceiver count limit is hit. + LOG.warn("Got an exception. Peer {}", peer, ie); + IOUtils.closeQuietly(peer); + } catch (OutOfMemoryError ie) { + IOUtils.closeQuietly(peer); + // DataNode can run out of memory if there is too many transfers. + // Log the event, Sleep for 30 seconds, other transfers may complete by + // then. + LOG.error("DataNode is out of memory. Will retry in 30 seconds.", ie); + try { + Thread.sleep(TimeUnit.SECONDS.toMillis(30L)); + } catch (InterruptedException e) { + // ignore + } + } catch (Throwable te) { + LOG.error("XceiverServerDomainSocket: Exiting.", te); + } + } + + close(); + } + + void close() { + try { + // Close the server to accept more requests. + if (domainSocket != null) { + domainSocket.getChannel().close(); + LOG.info("DomainSocket {} is closed", domainSocket.toString()); + } + } catch (IOException ie) { + LOG.warn("Failed to close domainSocket {}", domainSocket.toString(), ie); + } + + closeAllPeers(); + } + + /** + * Notify all Receiver thread of the shutdown. + */ + void closeAllPeers() { + // interrupt each and every Receiver thread. + peers.values().forEach(t -> t.interrupt()); + + // wait 3s for peers to close + long mills = 3000; + try { + while (!peers.isEmpty() && mills > 0) { + Thread.sleep(1000); + mills -= 1000; + } + } catch (InterruptedException e) { + LOG.info("Interrupted waiting for peers to close"); + Thread.currentThread().interrupt(); + } + + peers.keySet().forEach(org.apache.hadoop.io.IOUtils::closeStream); + peers.clear(); + peersReceiver.clear(); + } + + void addPeer(DomainPeer peer, Thread t, Receiver receiver) throws IOException { + if (!isRunning.get()) { + throw new IOException("XceiverServerDomainSocket is closed."); + } + peers.put(peer, t); + peersReceiver.put(peer, receiver); + xceriverCount.incrementAndGet(); + LOG.info("Peer {} is added", peer.getDomainSocket()); + } + + void closePeer(DomainPeer peer) throws IOException { + if (!isRunning.get()) { + throw new IOException("XceiverServerDomainSocket is closed."); + } + peers.remove(peer); + peersReceiver.remove(peer); + org.apache.hadoop.io.IOUtils.closeStream(peer); + xceriverCount.decrementAndGet(); + LOG.info("Peer {} is closed", peer.getDomainSocket()); + } + + @VisibleForTesting + public void setContainerDispatcher(ContainerDispatcher containerDispatcher) { + this.dispatcher = containerDispatcher; + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java index 624f153e876..8824c96211f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerGrpc.java @@ -66,6 +66,7 @@ import org.apache.ratis.thirdparty.io.netty.handler.ssl.SslContextBuilder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import com.google.common.annotations.VisibleForTesting; /** * Creates a Grpc server endpoint that acts as the communication layer for @@ -81,7 +82,6 @@ public final class XceiverServerGrpc implements XceiverServerSpi { private final ContainerDispatcher storageContainer; private boolean isStarted; private DatanodeDetails datanodeDetails; - private ThreadPoolExecutor readExecutors; private EventLoopGroup eventLoopGroup; private Class channelType; @@ -91,7 +91,7 @@ public final class XceiverServerGrpc implements XceiverServerSpi { * @param conf - Configuration */ public XceiverServerGrpc(DatanodeDetails datanodeDetails, - ConfigurationSource conf, + ConfigurationSource conf, ThreadPoolExecutor executor, ContainerDispatcher dispatcher, CertificateClient caClient) { Preconditions.checkNotNull(conf); @@ -105,19 +105,23 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, this.port = 0; } - final int threadCountPerDisk = - conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); - final int numberOfDisks = - HddsServerUtil.getDatanodeStorageDirs(conf).size(); - final int poolSize = threadCountPerDisk * numberOfDisks; + ThreadPoolExecutor readExecutors = executor; + if (readExecutors == null) { + // this branch is to avoid updating all existing related tests + final int threadCountPerDisk = + conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = + HddsServerUtil.getDatanodeStorageDirs(conf).size(); + final int poolSize = threadCountPerDisk * numberOfDisks; - readExecutors = new ThreadPoolExecutor(poolSize, poolSize, - 60, TimeUnit.SECONDS, - new LinkedBlockingQueue<>(), - new ThreadFactoryBuilder().setDaemon(true) - .setNameFormat(datanodeDetails.threadNamePrefix() + - "ChunkReader-%d") - .build()); + readExecutors = new ThreadPoolExecutor(poolSize, poolSize, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(datanodeDetails.threadNamePrefix() + + "ChunkReader-%d") + .build()); + } ThreadFactory factory = new ThreadFactoryBuilder() .setDaemon(true) @@ -126,10 +130,10 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, .build(); if (Epoll.isAvailable()) { - eventLoopGroup = new EpollEventLoopGroup(poolSize / 10, factory); + eventLoopGroup = new EpollEventLoopGroup(readExecutors.getPoolSize() / 10, factory); channelType = EpollServerSocketChannel.class; } else { - eventLoopGroup = new NioEventLoopGroup(poolSize / 10, factory); + eventLoopGroup = new NioEventLoopGroup(readExecutors.getPoolSize() / 10, factory); channelType = NioServerSocketChannel.class; } @@ -161,6 +165,12 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, storageContainer = dispatcher; } + @VisibleForTesting + public XceiverServerGrpc(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, CertificateClient caClient) { + this(datanodeDetails, conf, null, dispatcher, caClient); + } + @Override public int getIPCPort() { return this.port; @@ -210,8 +220,6 @@ public void start() throws IOException { public void stop() { if (isStarted) { try { - readExecutors.shutdown(); - readExecutors.awaitTermination(5L, TimeUnit.SECONDS); server.shutdown(); server.awaitTermination(5, TimeUnit.SECONDS); eventLoopGroup.shutdownGracefully().sync(); @@ -223,6 +231,11 @@ public void stop() { } } + @Override + public boolean isStarted() { + return isStarted; + } + @Override public void submitRequest(ContainerCommandRequestProto request, HddsProtos.PipelineID pipelineID) throws IOException { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java index 480561270ec..7cafeaecc53 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerSpi.java @@ -99,4 +99,6 @@ default List getStorageReport() throws IOException { return null; } + + boolean isStarted(); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index a4c14343985..3762cb3ead2 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -607,6 +607,11 @@ public void stop() { } } + @Override + public boolean isStarted() { + return isStarted; + } + @Override public int getIPCPort() { return clientPort; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java index aa9c4bd953c..4147e9cddd0 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/KeyValueHandler.java @@ -19,6 +19,7 @@ package org.apache.hadoop.ozone.container.keyvalue; import java.io.File; +import java.io.FileInputStream; import java.io.FilenameFilter; import java.io.IOException; import java.io.InputStream; @@ -32,6 +33,7 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.locks.Lock; import java.util.function.Function; @@ -88,6 +90,7 @@ import org.apache.hadoop.ozone.container.keyvalue.impl.ChunkManagerFactory; import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager; import org.apache.hadoop.ozone.container.keyvalue.interfaces.ChunkManager; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import com.google.common.annotations.VisibleForTesting; @@ -147,6 +150,9 @@ public class KeyValueHandler extends Handler { // A striped lock that is held during container creation. private final Striped containerCreationLocks; private static FaultInjector injector; + // map temporarily carries FileInputStreams for short-circuit read requests + private final Map streamMap = new ConcurrentHashMap<>(); + private OzoneContainer ozoneContainer; public KeyValueHandler(ConfigurationSource config, String datanodeId, @@ -154,7 +160,18 @@ public KeyValueHandler(ConfigurationSource config, VolumeSet volSet, ContainerMetrics metrics, IncrementalReportSender icrSender) { + this(config, datanodeId, contSet, volSet, metrics, icrSender, null); + } + + public KeyValueHandler(ConfigurationSource config, + String datanodeId, + ContainerSet contSet, + VolumeSet volSet, + ContainerMetrics metrics, + IncrementalReportSender icrSender, + OzoneContainer ozoneContainer) { super(config, datanodeId, contSet, volSet, metrics, icrSender); + this.ozoneContainer = ozoneContainer; blockManager = new BlockManagerImpl(config); validateChunkChecksumData = conf.getObject( DatanodeConfiguration.class).isChunkDataValidationCheck(); @@ -648,14 +665,34 @@ ContainerCommandResponseProto handleGetBlock( } ContainerProtos.BlockData responseData; + boolean shortCircuitGranted = false; try { - BlockID blockID = BlockID.getFromProtobuf( - request.getGetBlock().getBlockID()); + ContainerProtos.GetBlockRequestProto getBlock = request.getGetBlock(); + BlockID blockID = BlockID.getFromProtobuf(getBlock.getBlockID()); BlockUtils.verifyReplicaIdx(kvContainer, blockID); responseData = blockManager.getBlock(kvContainer, blockID).getProtoBufMessage(); + if (getBlock.hasRequestShortCircuitAccess() && getBlock.getRequestShortCircuitAccess()) { + if (!VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.SHORT_CIRCUIT_READS)) { + throw new StorageContainerException("DataNode has not finalized " + + "upgrading to support short-circuit read.", UNSUPPORTED_REQUEST); + } + boolean domainSocketServerEnabled = ozoneContainer != null + && ozoneContainer.getReadDomainSocketChannel() != null + && ozoneContainer.getReadDomainSocketChannel().isStarted(); + if (domainSocketServerEnabled) { + FileInputStream fis = chunkManager.getShortCircuitFd(kvContainer, blockID); + Preconditions.checkState(fis != null); + String mapKey = getMapKey(request); + streamMap.put(mapKey, fis); + shortCircuitGranted = true; + } + } final long numBytes = responseData.getSerializedSize(); - metrics.incContainerBytesStats(Type.GetBlock, numBytes); - + if (shortCircuitGranted) { + metrics.incContainerLocalBytesStats(Type.GetBlock, numBytes); + } else { + metrics.incContainerBytesStats(Type.GetBlock, numBytes); + } } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ex) { @@ -664,7 +701,22 @@ ContainerCommandResponseProto handleGetBlock( request); } - return getBlockDataResponse(request, responseData); + return getBlockDataResponse(request, responseData, shortCircuitGranted); + } + + public FileInputStream getBlockInputStream(ContainerCommandRequestProto request) throws IOException { + if (request.getCmdType() != Type.GetBlock) { + throw new StorageContainerException("Request type mismatch, expected " + Type.GetBlock + + ", received " + request.getCmdType(), ContainerProtos.Result.MALFORMED_REQUEST); + } + String mapKey = getMapKey(request); + FileInputStream fis = streamMap.remove(mapKey); + LOG.info("streamMap remove stream {} for {}", fis.getFD(), mapKey); + return fis; + } + + private String getMapKey(ContainerCommandRequestProto request) { + return request.getClientId().toStringUtf8() + "-" + request.getCallId(); } /** diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java index 6a1d5533cf2..0df62ad8e0f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/ChunkManagerDispatcher.java @@ -39,6 +39,8 @@ import org.slf4j.LoggerFactory; import jakarta.annotation.Nonnull; + +import java.io.FileInputStream; import java.io.IOException; import java.util.EnumMap; import java.util.Map; @@ -144,6 +146,12 @@ public void deleteChunks(Container container, BlockData blockData) selectHandler(container).deleteChunks(container, blockData); } + @Override + public FileInputStream getShortCircuitFd(Container container, BlockID blockID) + throws StorageContainerException { + return selectHandler(container).getShortCircuitFd(container, blockID); + } + @Override public void shutdown() { handlers.values().forEach(ChunkManager::shutdown); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java index 4ca578d7717..d83657cda09 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerBlockStrategy.java @@ -27,6 +27,8 @@ import org.apache.hadoop.hdds.client.BlockID; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; +import org.apache.hadoop.hdds.utils.IOUtils; +import org.apache.hadoop.io.nativeio.NativeIO; import org.apache.hadoop.ozone.common.ChunkBuffer; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.helpers.ChunkInfo; @@ -46,6 +48,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.RandomAccessFile; @@ -55,6 +58,7 @@ import java.time.Duration; import java.util.concurrent.ExecutionException; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.GET_SHORT_CIRCUIT_FD_FAILED; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; import static org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion.FILE_PER_BLOCK; import static org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext.WriteChunkStage.COMMIT_DATA; @@ -208,6 +212,22 @@ public ChunkBuffer readChunk(Container container, BlockID blockID, readMappedBufferThreshold, readMappedBufferMaxCount > 0, mappedBufferManager); } + @Override + public FileInputStream getShortCircuitFd(Container container, BlockID blockID) throws StorageContainerException { + checkLayoutVersion(container); + final File chunkFile = getChunkFile(container, blockID); + FileInputStream fis = null; + try { + fis = new FileInputStream(NativeIO.getShareDeleteFileDescriptor(chunkFile, 0)); + return fis; + } catch (Exception e) { + IOUtils.closeQuietly(fis); + LOG.warn("getShortCircuitFds failed", e); + throw new StorageContainerException("getShortCircuitFds " + + "for short-circuit local reads failed", GET_SHORT_CIRCUIT_FD_FAILED); + } + } + @Override public void deleteChunk(Container container, BlockID blockID, ChunkInfo info) throws StorageContainerException { @@ -374,5 +394,4 @@ public void close() { } } } - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java index 7751dba429d..04a268eec53 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/ChunkManager.java @@ -31,9 +31,12 @@ import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainer; import org.apache.ratis.statemachine.StateMachine; +import java.io.FileInputStream; import java.io.IOException; import java.nio.ByteBuffer; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; + /** * Chunk Manager allows read, write, delete and listing of chunks in * a container. @@ -78,6 +81,20 @@ default void writeChunk(Container container, BlockID blockID, ChunkInfo info, ChunkBuffer readChunk(Container container, BlockID blockID, ChunkInfo info, DispatcherContext dispatcherContext) throws StorageContainerException; + /** + * Get the FileInputStream of a given chunk, to share with client for short circuit read. + * + * @param container - Container for the chunk + * @param blockID - ID of the block. + * @return FileInputStream - input stream of block file + * @throws StorageContainerException + */ + default FileInputStream getShortCircuitFd(Container container, BlockID blockID) + throws StorageContainerException { + throw new StorageContainerException("Operation is not supported for " + this.getClass().getSimpleName(), + UNSUPPORTED_REQUEST); + } + /** * Deletes a given chunk. * diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java index 56c42338366..597c3b5463f 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OzoneContainer.java @@ -29,10 +29,12 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerReplicaProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.IncrementalContainerReportProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.PipelineReportsProto; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.symmetric.SecretKeyVerifierClient; import org.apache.hadoop.hdds.security.token.TokenVerifier; import org.apache.hadoop.hdds.security.x509.certificate.client.CertificateClient; +import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.HddsDatanodeService; @@ -47,6 +49,7 @@ import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; import org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration; import org.apache.hadoop.ozone.container.common.statemachine.StateContext; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerDomainSocket; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerGrpc; import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; import org.apache.hadoop.ozone.container.common.transport.server.ratis.XceiverServerRatis; @@ -61,6 +64,7 @@ import org.apache.hadoop.ozone.container.replication.ContainerImporter; import org.apache.hadoop.ozone.container.replication.ReplicationServer; import org.apache.hadoop.ozone.container.replication.ReplicationServer.ReplicationConfig; +import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures; import org.apache.hadoop.ozone.container.upgrade.VersionedDatanodeFeatures.SchemaV3; import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException; import org.apache.hadoop.util.Timer; @@ -78,7 +82,9 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadFactory; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -114,6 +120,9 @@ public class OzoneContainer { private final ContainerSet containerSet; private final XceiverServerSpi writeChannel; private final XceiverServerSpi readChannel; + private XceiverServerSpi readDomainSocketChannel; + private final ThreadPoolExecutor readExecutors; + private DomainSocketFactory domainSocketFactory; private final ContainerController controller; private BackgroundContainerMetadataScanner metadataScanner; private List dataScanners; @@ -143,15 +152,13 @@ enum InitializingStatus { * @throws DiskOutOfSpaceException * @throws IOException */ - public OzoneContainer(HddsDatanodeService hddsDatanodeService, - DatanodeDetails datanodeDetails, ConfigurationSource conf, - StateContext context, CertificateClient certClient, + public OzoneContainer(HddsDatanodeService hddsDatanodeService, DatanodeDetails datanodeDetails, + ConfigurationSource conf, StateContext context, CertificateClient certClient, SecretKeyVerifierClient secretKeyClient) throws IOException { config = conf; this.datanodeDetails = datanodeDetails; this.context = context; - this.volumeChecker = new StorageVolumeChecker(conf, new Timer(), - datanodeDetails.threadNamePrefix()); + this.volumeChecker = new StorageVolumeChecker(conf, new Timer(), datanodeDetails.threadNamePrefix()); volumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf, context, VolumeType.DATA_VOLUME, volumeChecker); @@ -162,8 +169,7 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, dbVolumeSet = HddsServerUtil.getDatanodeDbDirs(conf).isEmpty() ? null : new MutableVolumeSet(datanodeDetails.getUuidString(), conf, context, VolumeType.DB_VOLUME, volumeChecker); - final DatanodeConfiguration dnConf = - conf.getObject(DatanodeConfiguration.class); + final DatanodeConfiguration dnConf = conf.getObject(DatanodeConfiguration.class); if (SchemaV3.isFinalizedAndEnabled(config)) { HddsVolumeUtil.loadAllHddsVolumeDbStore( volumeSet, dbVolumeSet, false, LOG); @@ -180,8 +186,7 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, } } - long recoveringContainerTimeout = config.getTimeDuration( - OZONE_RECOVERING_CONTAINER_TIMEOUT, + long recoveringContainerTimeout = config.getTimeDuration(OZONE_RECOVERING_CONTAINER_TIMEOUT, OZONE_RECOVERING_CONTAINER_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); containerSet = new ContainerSet(recoveringContainerTimeout); @@ -193,7 +198,6 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, IncrementalReportSender icrSender = container -> { synchronized (containerSet) { ContainerReplicaProto containerReport = container.getContainerReport(); - IncrementalContainerReportProto icr = IncrementalContainerReportProto .newBuilder() .addReport(containerReport) @@ -208,13 +212,12 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, Handler.getHandlerForContainerType( containerType, conf, context.getParent().getDatanodeDetails().getUuidString(), - containerSet, volumeSet, metrics, icrSender)); + containerSet, volumeSet, metrics, icrSender, this)); } SecurityConfig secConf = new SecurityConfig(conf); hddsDispatcher = new HddsDispatcher(config, containerSet, volumeSet, - handlers, context, metrics, - TokenVerifier.create(secConf, secretKeyClient)); + handlers, context, metrics, TokenVerifier.create(secConf, secretKeyClient)); /* * ContainerController is the control plane @@ -224,30 +227,47 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, controller = new ContainerController(containerSet, handlers); writeChannel = XceiverServerRatis.newXceiverServerRatis(hddsDatanodeService, - datanodeDetails, config, hddsDispatcher, controller, certClient, - context); + datanodeDetails, config, hddsDispatcher, controller, certClient, context); replicationServer = new ReplicationServer( controller, conf.getObject(ReplicationConfig.class), secConf, certClient, - new ContainerImporter(conf, containerSet, controller, - volumeSet), + new ContainerImporter(conf, containerSet, controller, volumeSet), datanodeDetails.threadNamePrefix()); - readChannel = new XceiverServerGrpc( - datanodeDetails, config, hddsDispatcher, certClient); - Duration blockDeletingSvcInterval = dnConf.getBlockDeletionInterval(); + final int threadCountPerDisk = conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume(); + final int numberOfDisks = HddsServerUtil.getDatanodeStorageDirs(conf).size(); + final int poolSize = threadCountPerDisk * numberOfDisks; + + readExecutors = new ThreadPoolExecutor(poolSize, poolSize, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + new ThreadFactoryBuilder().setDaemon(true) + .setNameFormat(datanodeDetails.threadNamePrefix() + + "ChunkReader-%d") + .build()); + + readChannel = new XceiverServerGrpc(datanodeDetails, config, readExecutors, hddsDispatcher, certClient); + if (VersionedDatanodeFeatures.isFinalized(HDDSLayoutFeature.SHORT_CIRCUIT_READS)) { + domainSocketFactory = DomainSocketFactory.getInstance(config); + if (domainSocketFactory.isServiceEnabled() && domainSocketFactory.isServiceReady()) { + readDomainSocketChannel = new XceiverServerDomainSocket(datanodeDetails, config, + hddsDispatcher, readExecutors, metrics, domainSocketFactory); + } else { + readDomainSocketChannel = null; + } + } + Duration blockDeletingSvcInterval = conf.getObject( + DatanodeConfiguration.class).getBlockDeletionInterval(); long blockDeletingServiceTimeout = config .getTimeDuration(OZONE_BLOCK_DELETING_SERVICE_TIMEOUT, OZONE_BLOCK_DELETING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); - - int blockDeletingServiceWorkerSize = config - .getInt(OZONE_BLOCK_DELETING_SERVICE_WORKERS, - OZONE_BLOCK_DELETING_SERVICE_WORKERS_DEFAULT); + int blockDeletingServiceWorkerSize = + config.getInt(OZONE_BLOCK_DELETING_SERVICE_WORKERS, OZONE_BLOCK_DELETING_SERVICE_WORKERS_DEFAULT); blockDeletingService = new BlockDeletingService(this, blockDeletingSvcInterval.toMillis(), blockDeletingServiceTimeout, TimeUnit.MILLISECONDS, @@ -255,18 +275,14 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, datanodeDetails.threadNamePrefix(), context.getParent().getReconfigurationHandler()); - Duration recoveringContainerScrubbingSvcInterval = - dnConf.getRecoveringContainerScrubInterval(); - + Duration recoveringContainerScrubbingSvcInterval = dnConf.getRecoveringContainerScrubInterval(); long recoveringContainerScrubbingServiceTimeout = config .getTimeDuration(OZONE_RECOVERING_CONTAINER_SCRUBBING_SERVICE_TIMEOUT, OZONE_RECOVERING_CONTAINER_SCRUBBING_SERVICE_TIMEOUT_DEFAULT, TimeUnit.MILLISECONDS); - int recoveringContainerScrubbingServiceWorkerSize = config .getInt(OZONE_RECOVERING_CONTAINER_SCRUBBING_SERVICE_WORKERS, OZONE_RECOVERING_CONTAINER_SCRUBBING_SERVICE_WORKERS_DEFAULT); - recoveringContainerScrubbingService = new StaleRecoveringContainerScrubbingService( recoveringContainerScrubbingSvcInterval.toMillis(), @@ -283,8 +299,7 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, tlsClientConfig = null; } - initializingStatus = - new AtomicReference<>(InitializingStatus.UNINITIALIZED); + initializingStatus = new AtomicReference<>(InitializingStatus.UNINITIALIZED); } /** @@ -495,6 +510,9 @@ public void start(String clusterId) throws IOException { hddsDispatcher.setClusterId(clusterId); writeChannel.start(); readChannel.start(); + if (readDomainSocketChannel != null) { + readDomainSocketChannel.start(); + } blockDeletingService.start(); recoveringContainerScrubbingService.start(); @@ -513,7 +531,19 @@ public void stop() { stopContainerScrub(); replicationServer.stop(); writeChannel.stop(); + readExecutors.shutdown(); + try { + readExecutors.awaitTermination(5L, TimeUnit.SECONDS); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } readChannel.stop(); + if (readDomainSocketChannel != null) { + readDomainSocketChannel.stop(); + } + if (domainSocketFactory != null) { + domainSocketFactory.close(); + } this.handlers.values().forEach(Handler::stop); hddsDispatcher.shutdown(); volumeChecker.shutdownAndWait(0, TimeUnit.SECONDS); @@ -563,6 +593,10 @@ public XceiverServerSpi getReadChannel() { return readChannel; } + public XceiverServerSpi getReadDomainSocketChannel() { + return readDomainSocketChannel; + } + public ContainerController getController() { return controller; } diff --git a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 6cfae24d41e..f6f00aff8e9 100644 --- a/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto +++ b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto @@ -158,6 +158,7 @@ enum Result { EXPORT_CONTAINER_METADATA_FAILED = 45; IMPORT_CONTAINER_METADATA_FAILED = 46; BLOCK_ALREADY_FINALIZED = 47; + GET_SHORT_CIRCUIT_FD_FAILED = 48; } /** @@ -217,6 +218,10 @@ message ContainerCommandRequestProto { optional FinalizeBlockRequestProto finalizeBlock = 25; optional EchoRequestProto echo = 26; + + // clientId and callId are used to distinguish different requests from different local clients for shortCircuitRead + optional bytes clientId = 27; + optional uint64 callId = 28; } message ContainerCommandResponseProto { @@ -250,6 +255,10 @@ message ContainerCommandResponseProto { optional FinalizeBlockResponseProto finalizeBlock = 22; optional EchoResponseProto echo = 23; + + // clientId and callId are used to distinguish different requests from different local clients for shortCircuitRead + optional bytes clientId = 24; + optional uint64 callId = 25; } message ContainerDataProto { @@ -360,10 +369,12 @@ message FinalizeBlockResponseProto { message GetBlockRequestProto { required DatanodeBlockID blockID = 1; + optional bool requestShortCircuitAccess = 2 [default = false]; } message GetBlockResponseProto { required BlockData blockData = 1; + optional bool shortCircuitAccessGranted = 2 [default = false]; } diff --git a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java index 6edcca65f92..25187145189 100644 --- a/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java +++ b/hadoop-ozone/client/src/test/java/org/apache/hadoop/ozone/client/MockXceiverClientFactory.java @@ -93,7 +93,7 @@ public void releaseClient(XceiverClientSpi xceiverClient, } @Override - public XceiverClientSpi acquireClientForReadData(Pipeline pipeline) + public XceiverClientSpi acquireClientForReadData(Pipeline pipeline, boolean allowShortCircuit) throws IOException { return new MockXceiverClientSpi(pipeline, storage .computeIfAbsent(pipeline.getFirstNode(), @@ -108,7 +108,7 @@ public void releaseClientForReadData(XceiverClientSpi xceiverClient, @Override public XceiverClientSpi acquireClient(Pipeline pipeline, - boolean topologyAware) throws IOException { + boolean topologyAware, boolean allowShortCircuit) throws IOException { MockXceiverClientSpi mockXceiverClientSpi = new MockXceiverClientSpi(pipeline, storage .computeIfAbsent(topologyAware ? pipeline.getClosestNode() : diff --git a/hadoop-ozone/integration-test/libhadoop.dylib b/hadoop-ozone/integration-test/libhadoop.dylib new file mode 100755 index 00000000000..573a2b9ebb8 Binary files /dev/null and b/hadoop-ozone/integration-test/libhadoop.dylib differ diff --git a/hadoop-ozone/integration-test/libhadoop.so b/hadoop-ozone/integration-test/libhadoop.so new file mode 100755 index 00000000000..48d9ef7c372 Binary files /dev/null and b/hadoop-ozone/integration-test/libhadoop.so differ diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java index 83a9b106c8f..026f2fa86d0 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManager.java @@ -99,6 +99,9 @@ public void testCaching(boolean securityEnabled, @TempDir Path metaDir) throws I XceiverClientSpi client1 = clientManager .acquireClient(container1.getPipeline()); assertEquals(1, client1.getRefcount()); + // although allowShortCircuit true when calling acquireClientForReadData, + // XceiverClientGrpc client will be allocated since short-circuit is by default disabled. + assertThat(client1 instanceof XceiverClientGrpc); ContainerWithPipeline container2 = storageContainerLocationClient .allocateContainer( @@ -108,6 +111,7 @@ public void testCaching(boolean securityEnabled, @TempDir Path metaDir) throws I XceiverClientSpi client2 = clientManager .acquireClient(container2.getPipeline()); assertEquals(1, client2.getRefcount()); + assertThat(client2 instanceof XceiverClientGrpc); XceiverClientSpi client3 = clientManager .acquireClient(container1.getPipeline()); @@ -264,7 +268,7 @@ public void testFreeByRetryFailure() throws IOException { clientManager.releaseClient(client1, true); assertEquals(0, client1.getRefcount()); assertNotNull(cache.getIfPresent( - container1.getContainerInfo().getPipelineID().getId().toString() + container1.getContainerInfo().getPipelineID().getId().toString() + "-" + container1.getContainerInfo().getReplicationType())); // cleanup diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java new file mode 100644 index 00000000000..696ca8aadd2 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverClientManagerSC.java @@ -0,0 +1,103 @@ +/* + * 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; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.scm.XceiverClientManager.ScmClientConfig; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.ozone.container.common.SCMTestUtils; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Test for short-circuit enabled XceiverClientManager. + */ +@Timeout(300) +public class TestXceiverClientManagerSC { + + private static OzoneConfiguration config; + private static MiniOzoneCluster cluster; + private static StorageContainerLocationProtocolClientSideTranslatorPB + storageContainerLocationClient; + @TempDir + private static File dir; + + @BeforeAll + public static void init() throws Exception { + config = new OzoneConfiguration(); + OzoneClientConfig clientConfig = config.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + config.setFromObject(clientConfig); + config.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + cluster = MiniOzoneCluster.newBuilder(config) + .setNumDatanodes(3) + .build(); + cluster.waitForClusterToBeReady(); + storageContainerLocationClient = cluster + .getStorageContainerLocationClient(); + } + + @AfterAll + public static void shutdown() { + if (cluster != null) { + cluster.shutdown(); + } + IOUtils.cleanupWithLogger(null, storageContainerLocationClient); + } + + @Test + public void testAllocateShortCircuitClient() throws IOException { + try (XceiverClientManager clientManager = new XceiverClientManager(config, + config.getObject(ScmClientConfig.class), null)) { + + ContainerWithPipeline container1 = storageContainerLocationClient + .allocateContainer( + SCMTestUtils.getReplicationType(config), + HddsProtos.ReplicationFactor.THREE, + OzoneConsts.OZONE); + XceiverClientSpi client1 = clientManager.acquireClientForReadData(container1.getPipeline(), true); + assertEquals(1, client1.getRefcount()); + assertTrue(client1 instanceof XceiverClientShortCircuit); + XceiverClientSpi client2 = clientManager.acquireClientForReadData(container1.getPipeline(), true); + assertTrue(client2 instanceof XceiverClientShortCircuit); + assertEquals(2, client2.getRefcount()); + assertEquals(2, client1.getRefcount()); + assertEquals(client1, client2); + clientManager.releaseClient(client1, true); + clientManager.releaseClient(client2, true); + assertEquals(0, clientManager.getClientCache().size()); + + XceiverClientSpi client3 = clientManager.acquireClientForReadData(container1.getPipeline(), false); + assertTrue(client3 instanceof XceiverClientGrpc); + } + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java new file mode 100644 index 00000000000..b592b1627a7 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/hdds/scm/TestXceiverServerDomainSocket.java @@ -0,0 +1,701 @@ +/* + * 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; + +import com.google.common.collect.Maps; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomStringUtils; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.net.unix.DomainSocket; +import org.apache.hadoop.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChunkBuffer; +import org.apache.hadoop.ozone.container.ContainerTestHelper; +import org.apache.hadoop.ozone.container.common.ContainerTestUtils; +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; +import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher; +import org.apache.hadoop.ozone.container.common.interfaces.Handler; +import org.apache.hadoop.ozone.container.common.statemachine.StateContext; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerDomainSocket; +import org.apache.hadoop.ozone.container.common.volume.HddsVolume; +import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet; +import org.apache.hadoop.ozone.container.common.volume.StorageVolume; +import org.apache.hadoop.ozone.container.common.volume.VolumeSet; +import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer; +import org.apache.ozone.test.GenericTestUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.EOFException; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.net.ConnectException; +import java.net.InetSocketAddress; +import java.net.SocketException; +import java.net.SocketTimeoutException; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.hdds.protocol.MockDatanodeDetails.randomDatanodeDetails; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.GetBlock; +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.ReadChunk; +import static org.apache.hadoop.hdds.scm.XceiverClientShortCircuit.vintPrefixed; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests the XceiverServerDomainSocket class. + */ +@Timeout(300) +public class TestXceiverServerDomainSocket { + private final InetSocketAddress localhost = InetSocketAddress.createUnresolved("localhost", 10000); + @TempDir + private File dir; + + private final ThreadPoolExecutor readExecutors = new ThreadPoolExecutor(1, 1, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>()); + + private static OzoneConfiguration conf; + private static ContainerMetrics metrics; + private static int readTimeout; + private static int writeTimeout; + + @BeforeAll + public static void setup() { + // enable short-circuit read + conf = new OzoneConfiguration(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + clientConfig.setShortCircuitReadDisableInterval(1000); + conf.setFromObject(clientConfig); + metrics = ContainerMetrics.create(conf); + readTimeout = 5 * 1000; + writeTimeout = 5 * 1000; + } + + @Test + public void testIllegalDomainPathConfiguration() { + // empty domain path + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, ""); + try { + DomainSocketFactory.getInstance(conf); + fail("Domain path is empty."); + } catch (Throwable e) { + assertTrue(e instanceof IllegalArgumentException); + assertTrue(e.getMessage().contains("ozone.domain.socket.path is not set")); + } + + // Domain path too long + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-datanode-socket-" + System.nanoTime()).getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("Domain path is too long."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof SocketException); + assertTrue(e.getMessage().contains("path too long")); + } finally { + factory.close(); + } + + // non-existing domain parent path + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir.getAbsolutePath() + System.nanoTime(), "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("non-existing domain parent path."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("failed to stat a path component")); + } finally { + factory.close(); + } + + // an existing domain path, the existing regular file will be overwritten and turned into a socket file, + // so configure an existing domain path is disallowed. + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + File file = new File(dir, "ozone-socket"); + try { + assertTrue(file.createNewFile()); + DomainSocketFactory.getInstance(conf); + fail("an existing domain path is not allowed."); + } catch (Throwable e) { + e.printStackTrace(); + assertTrue(e instanceof IllegalArgumentException); + assertTrue(e.getMessage().contains("an existing file")); + } finally { + file.delete(); + } + } + + @Test + public void testDomainPathPermission() { + // write from everyone is not allowed (permission too open) + assertTrue(dir.setWritable(true, false)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("write from everyone is not allowed."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("It is not protected because it is world-writable")); + } finally { + factory.close(); + } + + // write from owner is required + assertTrue(dir.setWritable(false, false)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("write from owner is required."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("Permission denied")); + } finally { + factory.close(); + } + + // write from owner is required + assertTrue(dir.setWritable(true, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket-write").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = null; + try { + server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + } catch (Throwable e) { + fail("write from owner is required."); + } finally { + factory.close(); + if (server != null) { + server.stop(); + } + } + + // execute from owner is required + assertTrue(dir.setExecutable(false, true)); + assertTrue(dir.setWritable(true, true)); + assertTrue(dir.setReadable(true, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket-execute").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + fail("execute from owner is required."); + } catch (Throwable e) { + assertTrue(e.getCause() instanceof IOException); + assertTrue(e.getMessage().contains("Permission denied")); + } finally { + factory.close(); + dir.setExecutable(true, true); + } + + // read from owner is not required + assertTrue(dir.setExecutable(true, true)); + assertTrue(dir.setWritable(true, true)); + assertTrue(dir.setReadable(false, true)); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket-read").getAbsolutePath()); + factory = DomainSocketFactory.getInstance(conf); + try { + server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + } catch (Throwable e) { + fail("read from owner is not required."); + } finally { + factory.close(); + dir.setReadable(true, true); + if (server != null) { + server.stop(); + } + } + } + + /** + * Test connection and read/write. + * On Linux, when there is still open file handle of a deleted file, the file handle remains open and can still + * be used to read and write the file. + */ + @ParameterizedTest + @CsvSource({ + "true, true", + "true, false", + "false, true", + "false, false", + }) + public void testReadWrite(boolean deleteFileBeforeRead, boolean deleteFileDuringRead) throws IOException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + ContainerMetrics containerMetrics = ContainerMetrics.create(conf); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, containerMetrics, factory); + DomainSocket sock = null; + try { + File volume = new File(dir, "dn-volume"); + server.setContainerDispatcher(createDispatcherAndPrepareData(volume, server, containerMetrics)); + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + + // send request + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + + // receive response + final DataInputStream inputStream = new DataInputStream(sock.getInputStream()); + short ret = inputStream.readShort(); + assertEquals(OzoneClientConfig.DATA_TRANSFER_VERSION, ret); + ret = inputStream.readShort(); + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), ret); + ContainerProtos.ContainerCommandResponseProto responseProto = + ContainerProtos.ContainerCommandResponseProto.parseFrom(vintPrefixed(inputStream)); + + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), responseProto.getCmdType().getNumber()); + ContainerProtos.GetBlockResponseProto getBlockResponseProto = responseProto.getGetBlock(); + assertEquals(ContainerProtos.Result.SUCCESS, responseProto.getResult()); + assertTrue(getBlockResponseProto.getShortCircuitAccessGranted()); + + // read FSD from domainSocket + FileInputStream[] fis = new FileInputStream[1]; + byte[] buf = new byte[1]; + sock.recvFileInputStreams(fis, buf, 0, buf.length); + assertNotNull(fis[0]); + + if (deleteFileBeforeRead) { + FileUtils.deleteDirectory(volume); + } + // read file content + FileChannel dataIn = fis[0].getChannel(); + int chunkSize = 1024 * 1024; + dataIn.position(0); + ByteBuffer dataBuf = ByteBuffer.allocate(chunkSize / 2); + // a closed socket doesn't impact file stream + sock.close(); + int readSize = dataIn.read(dataBuf); + assertEquals(chunkSize / 2, readSize); + if (deleteFileDuringRead) { + FileUtils.deleteDirectory(volume); + } + dataBuf.flip(); + readSize = dataIn.read(dataBuf); + assertEquals(chunkSize / 2, readSize); + dataBuf.flip(); + readSize = dataIn.read(dataBuf); + assertEquals(-1, readSize); + + // check metrics + assertEquals(1, containerMetrics.getContainerLocalOpsMetrics(ContainerProtos.Type.GetBlock)); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + /** + * Test server is not listening. + */ + @Test + public void testServerNotListening() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + DomainSocket sock = null; + try { + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + } catch (IOException e) { + assertTrue(e instanceof ConnectException); + assertTrue(e.getMessage().contains("connect(2) error: No such file or directory")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + } + } + + /** + * Test server is not started to accept new connection. + * Although socket can be created, read will fail, write can succeed. + */ + @Test + public void testServerNotStart() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + // send request + try(DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream())) { + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + } + + try(DataInputStream inputStream = new DataInputStream(sock.getInputStream())) { + inputStream.readShort(); + } + } catch (IOException e) { + assertTrue(e instanceof SocketTimeoutException); + assertTrue(e.getMessage().contains("read(2) error: Resource temporarily unavailable")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + public void testReadTimeout() throws InterruptedException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + conf.set(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT, "2s"); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + assertTrue(sock.isOpen()); + + // server will close the DomainSocket if there is no message from client in OZONE_CLIENT_READ_TIMEOUT + Thread.sleep(2 * 1000); + // send request + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(GetBlock.getNumber()); + getBlockRequest().writeDelimitedTo(outputStream); + outputStream.flush(); + } catch (IOException e) { + assertTrue(e instanceof SocketException); + assertTrue(e.getMessage().contains("write(2) error: Broken pipe")); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + public void testMaxXceiverCount() throws IOException, InterruptedException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + List list = new ArrayList<>(); + GenericTestUtils.LogCapturer logCapturer = + GenericTestUtils.LogCapturer.captureLogs(XceiverServerDomainSocket.LOG); + try { + server.start(); + // test max allowed xceiver count(default 10 * 5) + int count = 51; + for (int i = 1; i <= count; i++) { + DomainSocket sock = factory.createSocket(readTimeout, writeTimeout, localhost); + list.add(sock); + } + + assertTrue(logCapturer.getOutput().contains("Xceiver count exceeds the limit" + (count - 1))); + DomainSocket lastSock = list.get(list.size() - 1); + // although remote peer is already closed due to limit exhausted, sock.isOpen() is still true. + // Only when client read/write socket stream, there will be exception or -1 returned. + assertTrue(lastSock.isOpen()); + + // write to first 50 sockets should be OK + for (int i = 0; i < count - 2; i++) { + DomainSocket sock = list.get(i); + assertTrue(sock.isOpen()); + sock.getOutputStream().write(1); + sock.getOutputStream().flush(); + sock.close(); + assertFalse(sock.isOpen()); + } + + Thread.sleep(5000); + // read a broken pipe will return -1 + int data = lastSock.getInputStream().read(); + assertEquals(-1, data); + + // write the last socket should fail + try { + lastSock.getOutputStream().write(1); + lastSock.getOutputStream().flush(); + fail("Write to a peer closed socket should fail"); + } catch (Exception e) { + assertTrue(e instanceof SocketException); + assertTrue(e.getMessage().contains("write(2) error: Broken pipe")); + } + lastSock.close(); + assertFalse(lastSock.isOpen()); + } finally { + factory.close(); + server.stop(); + } + } + + /** + * When server receives any message which doesn't follow the version, request type, request body sequence, server + * will treat it as a critical error, close the connection. + */ + @Test + public void testSendIrrelevantMessage() { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + String data = "hello world"; + try { + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + try(DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream())) { + outputStream.write(data.getBytes()); + outputStream.flush(); + } + sock.getInputStream().read(); + } catch (IOException e) { + assertTrue(e instanceof EOFException); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + @Test + public void testSendUnsupportedRequest() throws IOException { + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + DomainSocketFactory factory = DomainSocketFactory.getInstance(conf); + XceiverServerDomainSocket server = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, factory); + DomainSocket sock = null; + try { + File volume = new File(dir, "dn-volume"); + server.setContainerDispatcher(createDispatcherAndPrepareData(volume, server, metrics)); + server.start(); + sock = factory.createSocket(readTimeout, writeTimeout, localhost); + final DataOutputStream outputStream = new DataOutputStream(sock.getOutputStream()); + outputStream.writeShort(OzoneClientConfig.DATA_TRANSFER_VERSION); + outputStream.writeShort(ReadChunk.getNumber()); + ContainerTestHelper.getDummyCommandRequestProto(ReadChunk).writeDelimitedTo(outputStream); + outputStream.flush(); + + // receive response + final DataInputStream inputStream = new DataInputStream(sock.getInputStream()); + short ret = inputStream.readShort(); + assertEquals(OzoneClientConfig.DATA_TRANSFER_VERSION, ret); + ret = inputStream.readShort(); + assertEquals(ContainerProtos.Type.ReadChunk.getNumber(), ret); + ContainerProtos.ContainerCommandResponseProto responseProto = + ContainerProtos.ContainerCommandResponseProto.parseFrom(vintPrefixed(inputStream)); + assertTrue(responseProto.getResult() == ContainerProtos.Result.UNSUPPORTED_REQUEST); + } finally { + factory.close(); + if (sock != null) { + try { + sock.close(); + } catch (IOException e) { + } + } + server.stop(); + } + } + + private ContainerProtos.ContainerCommandRequestProto getBlockRequest() { + long value = 1; + String datanodeUUID = UUID.randomUUID().toString(); + ContainerProtos.GetBlockRequestProto.Builder getBlock = + ContainerProtos.GetBlockRequestProto.newBuilder() + .setBlockID(new BlockID(value, value).getDatanodeBlockIDProtobuf()) + .setRequestShortCircuitAccess(true); + return ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(GetBlock) + .setContainerID(value) + .setGetBlock(getBlock) + .setDatanodeUuid(datanodeUUID) + .build(); + } + + private ContainerProtos.ContainerCommandRequestProto getReadChunkRequest() { + long value = 1; + String datanodeUUID = UUID.randomUUID().toString(); + ContainerProtos.GetBlockRequestProto.Builder getBlock = + ContainerProtos.GetBlockRequestProto.newBuilder() + .setBlockID(new BlockID(value, value).getDatanodeBlockIDProtobuf()) + .setRequestShortCircuitAccess(true); + return ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(GetBlock) + .setContainerID(value) + .setGetBlock(getBlock) + .setDatanodeUuid(datanodeUUID) + .build(); + } + + private ContainerDispatcher createDispatcherAndPrepareData(File volume, + XceiverServerDomainSocket domainSocketServer, ContainerMetrics containerMetrics) throws IOException { + DatanodeDetails datanodeDetails = randomDatanodeDetails(); + conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, volume.getAbsolutePath()); + conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, volume.getAbsolutePath()); + VolumeSet volumeSet = new MutableVolumeSet(datanodeDetails.getUuidString(), conf, + null, StorageVolume.VolumeType.DATA_VOLUME, null); + String cID = UUID.randomUUID().toString(); + HddsVolume dataVolume = (HddsVolume) volumeSet.getVolumesList().get(0); + dataVolume.format(cID); + dataVolume.setDbParentDir(volume); + assertTrue(dataVolume.getDbParentDir() != null); + ContainerSet containerSet = new ContainerSet(1000); + + // create HddsDispatcher + StateContext context = ContainerTestUtils.getMockContext(datanodeDetails, conf); + Map handlers = Maps.newHashMap(); + OzoneContainer ozoneContainer = mock(OzoneContainer.class); + when(ozoneContainer.getReadDomainSocketChannel()).thenReturn(domainSocketServer); + for (ContainerProtos.ContainerType containerType : + ContainerProtos.ContainerType.values()) { + handlers.put(containerType, + Handler.getHandlerForContainerType(containerType, conf, + context.getParent().getDatanodeDetails().getUuidString(), + containerSet, volumeSet, metrics, + c -> { }, ozoneContainer)); + } + HddsDispatcher dispatcher = + new HddsDispatcher(conf, containerSet, volumeSet, handlers, context, containerMetrics, null); + dispatcher.setClusterId(cID); + // create container + long value = 1L; + String pipelineID = UUID.randomUUID().toString(); + final ContainerProtos.ContainerCommandRequestProto createContainer = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.CreateContainer) + .setDatanodeUuid(datanodeDetails.getUuidString()).setCreateContainer( + ContainerProtos.CreateContainerRequestProto.newBuilder() + .setContainerType(ContainerProtos.ContainerType.KeyValueContainer).build()) + .setContainerID(value).setPipelineID(pipelineID) + .build(); + dispatcher.dispatch(createContainer, null); + + // write chunk + long id = 1; + int chunkSize = 1024 * 1024; + byte[] rawData = RandomStringUtils.randomAscii(chunkSize).getBytes(StandardCharsets.UTF_8); + Checksum checksum = new Checksum(ContainerProtos.ChecksumType.CRC32, chunkSize); + ContainerProtos.ChecksumData checksumProtobuf = checksum.computeChecksum(rawData).getProtoBufMessage(); + ContainerProtos.DatanodeBlockID blockId = ContainerProtos.DatanodeBlockID.newBuilder() + .setContainerID(id).setLocalID(id).setBlockCommitSequenceId(id).build(); + ContainerProtos.BlockData.Builder blockData = ContainerProtos.BlockData.newBuilder().setBlockID(blockId); + ContainerProtos.ChunkInfo.Builder chunkInfo = ContainerProtos.ChunkInfo.newBuilder() + .setChunkName("chunk_" + value).setOffset(0).setLen(chunkSize).setChecksumData(checksumProtobuf); + blockData.addChunks(chunkInfo); + Pipeline pipeline = MockPipeline.createSingleNodePipeline(); + ContainerProtos.WriteChunkRequestProto.Builder writeChunk = + ContainerProtos.WriteChunkRequestProto.newBuilder() + .setBlockID(blockId).setChunkData(chunkInfo) + .setData(ChunkBuffer.wrap(ByteBuffer.wrap(rawData)).toByteString()); + + ContainerProtos.ContainerCommandRequestProto writeChunkRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.WriteChunk) + .setContainerID(blockId.getContainerID()) + .setWriteChunk(writeChunk) + .setDatanodeUuid(pipeline.getFirstNode().getUuidString()).build(); + dispatcher.dispatch(writeChunkRequest, null); + + ContainerProtos.PutBlockRequestProto.Builder putBlock = ContainerProtos.PutBlockRequestProto + .newBuilder().setBlockData(blockData); + ContainerProtos.ContainerCommandRequestProto putBlockRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.PutBlock) + .setContainerID(blockId.getContainerID()) + .setDatanodeUuid(datanodeDetails.getUuidString()) + .setPutBlock(putBlock) + .build(); + + dispatcher.dispatch(putBlockRequest, null); + return dispatcher; + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java index ff55ee83c17..6c690f987e2 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/MiniOzoneCluster.java @@ -306,7 +306,7 @@ abstract class Builder { protected boolean includeS3G = false; protected int dnInitialVersion = DatanodeVersion.FUTURE_VERSION.toProtoValue(); - protected int dnCurrentVersion = DatanodeVersion.COMBINED_PUTBLOCK_WRITECHUNK_RPC.toProtoValue(); + protected int dnCurrentVersion = DatanodeVersion.SHORT_CIRCUIT_READS.toProtoValue(); protected int numOfDatanodes = 3; protected boolean startDataNodes = true; diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java index c5301ba4194..0c323b61bb5 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestChunkInputStream.java @@ -62,10 +62,10 @@ void testAll(ContainerLayoutVersion layout) throws Exception { * Test to verify that data read from chunks is stored in a list of buffers * with max capacity equal to the bytes per checksum. */ - private void testChunkReadBuffers(TestBucket bucket) throws Exception { + protected void testChunkReadBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); - byte[] inputData = bucket.writeRandomBytes(keyName, dataLength); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), dataLength); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { @@ -124,9 +124,9 @@ private void testChunkReadBuffers(TestBucket bucket) throws Exception { } } - private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { + protected void testCloseReleasesBuffers(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); - bucket.writeRandomBytes(keyName, CHUNK_SIZE); + bucket.writeRandomBytes(keyName, getRepConfig(), CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { BlockInputStream block0Stream = @@ -147,9 +147,9 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { * Test that ChunkInputStream buffers are released as soon as the last byte * of the buffer is read. */ - private void testBufferRelease(TestBucket bucket) throws Exception { + protected void testBufferRelease(TestBucket bucket) throws Exception { String keyName = getNewKeyName(); - byte[] inputData = bucket.writeRandomBytes(keyName, CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), CHUNK_SIZE); try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { @@ -205,7 +205,7 @@ private void testBufferRelease(TestBucket bucket) throws Exception { } } - private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, + protected byte[] readDataFromChunk(ChunkInputStream chunkInputStream, int offset, int readDataLength) throws IOException { byte[] readData = new byte[readDataLength]; chunkInputStream.seek(offset); @@ -229,7 +229,7 @@ private byte[] readDataFromChunk(ChunkInputStream chunkInputStream, * @param expectedBufferCapacity expected buffer capacity of unreleased * buffers */ - private void checkBufferSizeAndCapacity(ByteBuffer[] buffers, + protected void checkBufferSizeAndCapacity(ByteBuffer[] buffers, int expectedNumBuffers, int numReleasedBuffers, long expectedBufferCapacity) { assertEquals(expectedNumBuffers, buffers.length, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java new file mode 100644 index 00000000000..677e2c8ca58 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestDomainSocketFactory.java @@ -0,0 +1,77 @@ +/* + * 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.client.rpc.read; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +import java.io.File; +import java.net.InetSocketAddress; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for {@link DomainSocketFactory}'s functionality. + */ +public class TestDomainSocketFactory { + + private final InetSocketAddress localhost = InetSocketAddress.createUnresolved("localhost", 10000); + + @TempDir + private File dir; + + private DomainSocketFactory getDomainSocketFactory() { + // enable short-circuit read + OzoneConfiguration conf = new OzoneConfiguration(); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + clientConfig.setShortCircuitReadDisableInterval(1); + conf.setFromObject(clientConfig); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-socket").getAbsolutePath()); + + // create DomainSocketFactory + DomainSocketFactory domainSocketFactory = DomainSocketFactory.getInstance(conf); + assertTrue(domainSocketFactory.isServiceEnabled()); + assertTrue(domainSocketFactory.isServiceReady()); + return domainSocketFactory; + } + + @Test + public void testShortCircuitDisableTemporary() { + DomainSocketFactory factory = getDomainSocketFactory(); + try { + // temporary disable short-circuit read + long pathExpireDuration = factory.getPathExpireMills(); + factory.disableShortCircuit(); + DomainSocketFactory.PathInfo pathInfo = factory.getPathInfo(localhost); + assertEquals(DomainSocketFactory.PathState.DISABLED, pathInfo.getPathState()); + try { + Thread.sleep(pathExpireDuration + 100); + } catch (InterruptedException e) { + } + pathInfo = factory.getPathInfo(localhost); + assertEquals(DomainSocketFactory.PathState.VALID, pathInfo.getPathState()); + } finally { + factory.close(); + } + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java index 256148dfb8d..65490a678af 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestInputStreamBase.java @@ -20,6 +20,8 @@ import java.util.UUID; import java.util.concurrent.TimeUnit; +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.scm.OzoneClientConfig; @@ -30,6 +32,7 @@ import org.apache.hadoop.ozone.OzoneConfigKeys; import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DEADNODE_INTERVAL; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_STALENODE_INTERVAL; @@ -42,7 +45,7 @@ abstract class TestInputStreamBase { static final int BLOCK_SIZE = 2 * MAX_FLUSH_SIZE; // 8MB static final int BYTES_PER_CHECKSUM = 256 * 1024; // 256KB - protected static MiniOzoneCluster newCluster( + protected MiniOzoneCluster newCluster( ContainerLayoutVersion containerLayout) throws Exception { OzoneConfiguration conf = new OzoneConfiguration(); @@ -64,6 +67,7 @@ protected static MiniOzoneCluster newCluster( conf.getObject(ReplicationManagerConfiguration.class); repConf.setInterval(Duration.ofSeconds(1)); conf.setFromObject(repConf); + setCustomizedProperties(conf); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) .setBlockSize(BLOCK_SIZE) @@ -73,12 +77,22 @@ protected static MiniOzoneCluster newCluster( .applyTo(conf); return MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(5) + .setNumDatanodes(getDatanodeCount()) .build(); } - static String getNewKeyName() { + String getNewKeyName() { return UUID.randomUUID().toString(); } + int getDatanodeCount() { + return 5; + } + + void setCustomizedProperties(OzoneConfiguration configuration) { + } + + ReplicationConfig getRepConfig() { + return RatisReplicationConfig.getInstance(THREE); + } } diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java new file mode 100644 index 00000000000..e8f2129d598 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java @@ -0,0 +1,160 @@ +/** + * 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.client.rpc.read; + +import org.apache.hadoop.hdds.client.RatisReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationConfig; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientGrpc; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; +import org.apache.hadoop.hdds.scm.storage.BlockInputStream; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.scm.storage.ShortCircuitChunkInputStream; +import org.apache.hadoop.ozone.MiniOzoneCluster; +import org.apache.hadoop.ozone.client.OzoneClient; +import org.apache.hadoop.ozone.client.io.KeyInputStream; +import org.apache.hadoop.ozone.container.common.impl.ContainerLayoutVersion; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; +import org.apache.hadoop.ozone.container.keyvalue.ContainerLayoutTestInfo; +import org.apache.hadoop.ozone.om.TestBucket; +import org.apache.ozone.test.GenericTestUtils; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assumptions.assumeTrue; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.event.Level; + +import java.io.File; +import java.io.IOException; + +/** + * Tests {@link ShortCircuitChunkInputStream}. + */ +public class TestShortCircuitChunkInputStream extends TestChunkInputStream { + + @TempDir + private File dir; + + @Override + int getDatanodeCount() { + return 1; + } + + @Override + void setCustomizedProperties(OzoneConfiguration configuration) { + OzoneClientConfig clientConfig = configuration.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuit(true); + configuration.setFromObject(clientConfig); + configuration.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-socket").getAbsolutePath()); + GenericTestUtils.setLogLevel(XceiverClientShortCircuit.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(XceiverClientGrpc.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(ShortCircuitChunkInputStream.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(BlockInputStream.LOG, Level.DEBUG); + } + + @Override + ReplicationConfig getRepConfig() { + return RatisReplicationConfig.getInstance(ONE); + } + + + /** + * Run the tests as a single test method to avoid needing a new mini-cluster + * for each test. + */ + @ContainerLayoutTestInfo.ContainerTest + @Override + void testAll(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout)) { + cluster.waitForClusterToBeReady(); + assumeTrue(DomainSocketFactory.getInstance(cluster.getConf()).isServiceReady()); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + GenericTestUtils.LogCapturer logCapturer1 = + GenericTestUtils.LogCapturer.captureLogs(ShortCircuitChunkInputStream.LOG); + GenericTestUtils.LogCapturer logCapturer2 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientShortCircuit.LOG); + GenericTestUtils.LogCapturer logCapturer3 = + GenericTestUtils.LogCapturer.captureLogs(BlockInputStream.LOG); + GenericTestUtils.LogCapturer logCapturer4 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.LOG); + testChunkReadBuffers(bucket); + testBufferRelease(bucket); + testCloseReleasesBuffers(bucket); + assertTrue(logCapturer1.getOutput().contains("ShortCircuitChunkInputStream is created")); + assertTrue(logCapturer2.getOutput().contains("XceiverClientShortCircuit is created")); + assertTrue((logCapturer3.getOutput().contains("Get the FileInputStream of block"))); + assertFalse(logCapturer4.getOutput().contains("XceiverClientGrpc is created")); + } + } + } + + @ContainerLayoutTestInfo.ContainerTest + void testFallbackToGrpc(ContainerLayoutVersion layout) throws Exception { + try (MiniOzoneCluster cluster = newCluster(layout)) { + cluster.waitForClusterToBeReady(); + assumeTrue(DomainSocketFactory.getInstance(cluster.getConf()).isServiceReady()); + + try (OzoneClient client = cluster.newClient()) { + TestBucket bucket = TestBucket.newBuilder(client).build(); + GenericTestUtils.LogCapturer logCapturer1 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientShortCircuit.LOG); + GenericTestUtils.LogCapturer logCapturer2 = + GenericTestUtils.LogCapturer.captureLogs(XceiverClientGrpc.LOG); + + // create key + String keyName = getNewKeyName(); + int dataLength = (2 * BLOCK_SIZE) + (CHUNK_SIZE); + byte[] inputData = bucket.writeRandomBytes(keyName, getRepConfig(), dataLength); + try (KeyInputStream keyInputStream = bucket.getKeyInputStream(keyName)) { + BlockInputStream block0Stream = + (BlockInputStream)keyInputStream.getPartStreams().get(0); + block0Stream.initialize(); + assertNotNull(block0Stream.getBlockInputStream()); + assertTrue(logCapturer1.getOutput().contains("XceiverClientShortCircuit is created")); + // stop XceiverServerDomainSocket + XceiverServerSpi server = cluster.getHddsDatanodes().get(0) + .getDatanodeStateMachine().getContainer().getReadDomainSocketChannel(); + server.stop(); + BlockInputStream block1Stream = (BlockInputStream)keyInputStream.getPartStreams().get(1); + try { + block1Stream.initialize(); + } catch (IOException e) { + assertTrue(e.getMessage().contains("DomainSocket stream is not open")); + assertTrue(logCapturer1.getOutput().contains("ReceiveResponseTask is closed due to java.io.EOFException")); + } + // read whole key + byte[] data = new byte[dataLength]; + int readLen = keyInputStream.read(data); + assertEquals(dataLength, readLen); + assertArrayEquals(inputData, data); + assertTrue(logCapturer2.getOutput().contains("XceiverClientGrpc is created")); + } + } + } + } +} diff --git a/pom.xml b/pom.xml index 9d48f929f69..2e0b3038546 100644 --- a/pom.xml +++ b/pom.xml @@ -67,7 +67,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs 2023-01-01T00:00:00Z 2.10.2 - 3.3.6 + 3.3.7-20241127-SNAPSHOT ${ozone.version}