diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java index 5426bbc49817..3ecc4fa73ac1 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/OzoneClientConfig.java @@ -39,6 +39,51 @@ public class OzoneClientConfig { private static final Logger LOG = LoggerFactory.getLogger(OzoneClientConfig.class); + public static final String OZONE_READ_SHORT_CIRCUIT = "ozone.client.read.short-circuit"; + public static final boolean OZONE_READ_SHORT_CIRCUIT_DEFAULT = false; + public static final String OZONE_DOMAIN_SOCKET_PATH = "ozone.domain.socket.path"; + public static final String OZONE_DOMAIN_SOCKET_PATH_DEFAULT = "/var/lib/ozone/dn_socket"; + public static final String SHORT_CIRCUIT_PREFIX = "read.short-circuit."; + public static final int DATA_TRANSFER_VERSION = 28; + + @Config(key = "read.short-circuit", + defaultValue = "false", + type = ConfigType.BOOLEAN, + description = "Whether read short-circuit is enabled or not", + tags = { ConfigTag.CLIENT, ConfigTag.DATANODE }) + private boolean shortCircuitEnabled = OZONE_READ_SHORT_CIRCUIT_DEFAULT; + + @Config(key = SHORT_CIRCUIT_PREFIX + "buffer.size", + defaultValue = "128KB", + type = ConfigType.SIZE, + description = "Buffer size of reader/writer.", + tags = { ConfigTag.CLIENT, ConfigTag.DATANODE }) + private int shortCircuitBufferSize = 128 * 1024; + + @Config(key = SHORT_CIRCUIT_PREFIX + "disable.interval", + defaultValue = "600", + type = ConfigType.LONG, + description = "If some unknown IO error happens on Domain socket read, short circuit read will be disabled " + + "temporary for this period of time(seconds).", + tags = { ConfigTag.CLIENT }) + private long shortCircuitReadDisableInterval = 60 * 10; + + public long getShortCircuitReadDisableInterval() { + return shortCircuitReadDisableInterval; + } + + public void setShortCircuitReadDisableInterval(long value) { + shortCircuitReadDisableInterval = value; + } + + public int getShortCircuitBufferSize() { + return shortCircuitBufferSize; + } + + public void setShortCircuitBufferSize(int size) { + this.shortCircuitBufferSize = size; + } + /** * Enum for indicating what mode to use when combining chunk and block * checksums to define an aggregate FileChecksum. This should be considered @@ -558,4 +603,12 @@ public void setMaxConcurrentWritePerKey(int maxConcurrentWritePerKey) { public int getMaxConcurrentWritePerKey() { return this.maxConcurrentWritePerKey; } + + public boolean isShortCircuitEnabled() { + return shortCircuitEnabled; + } + + public void setShortCircuit(boolean enabled) { + shortCircuitEnabled = enabled; + } } 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 75ae01c10058..4491e26f9434 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,13 +60,26 @@ public XceiverClientCreator(ConfigurationSource conf, ClientTrustManager trustMa if (securityEnabled) { Preconditions.checkNotNull(trustManager); } + shortCircuitEnabled = conf.getBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, + OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT_DEFAULT); + 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: @@ -74,6 +91,9 @@ protected XceiverClientSpi newClient(Pipeline pipeline) throws IOException { case EC: client = new ECXceiverClientGrpc(pipeline, conf, trustManager); break; + case SHORT_CIRCUIT: + client = new XceiverClientShortCircuit(pipeline, conf, dn); + break; case CHAINED: default: throw new IOException("not implemented " + pipeline.getType()); @@ -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); } @@ -108,7 +135,7 @@ public void releaseClientForReadData(XceiverClientSpi xceiverClient, boolean inv @Override public XceiverClientSpi acquireClient(Pipeline pipeline, boolean topologyAware) throws IOException { - return newClient(pipeline); + return newClient(pipeline, null); } @Override @@ -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 b7276d645b44..c47321b815d8 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 c02306f8af8b..a2cdfbbf602a 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", XceiverClientGrpc.class.getSimpleName()); } /** @@ -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 07b704417216..68178dad112b 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,8 +19,13 @@ 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.client.ReplicationConfig; +import org.apache.hadoop.hdds.client.ReplicationFactor; +import org.apache.hadoop.hdds.client.ReplicationType; import org.apache.hadoop.hdds.conf.Config; import org.apache.hadoop.hdds.conf.ConfigGroup; import org.apache.hadoop.hdds.conf.ConfigType; @@ -29,6 +34,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 +46,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 +73,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 +114,7 @@ public void onRemoval( }).build(); cacheMetrics = CacheMetrics.create(clientCache, this); + this.localDNCache = new ConcurrentHashMap<>(); } @VisibleForTesting @@ -117,17 +127,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 +188,8 @@ public void releaseClient(XceiverClientSpi client, boolean invalidateClient, client.decrementReference(); if (invalidateClient) { Pipeline pipeline = client.getPipeline(); - String key = getPipelineCacheKey(pipeline, topologyAware); + // allowShortCircuit = false, which means XceiverClientShortCircuit will never be released currently. + String key = getPipelineCacheKey(pipeline, topologyAware, false); XceiverClientSpi cachedClient = clientCache.getIfPresent(key); if (cachedClient == client) { clientCache.invalidate(key); @@ -150,26 +198,49 @@ 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)); + String key = getPipelineCacheKey(pipeline, topologyAware, allowShortCircuit); + if (key.endsWith(DomainSocketFactory.FEATURE_FLAG)) { + final Pipeline newPipeline = Pipeline.newBuilder(pipeline).setReplicationConfig( + ReplicationConfig.fromTypeAndFactor(ReplicationType.SHORT_CIRCUIT, + ReplicationFactor.valueOf(pipeline.getReplicationConfig().getReplication()))).build(); + return clientCache.get(key, () -> newClient(newPipeline, localDNCache.get(key))); + } else { + return clientCache.get(key, () -> newClient(pipeline)); + } } 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) { + if ((!isEC) && allowShortCircuit && isShortCircuitEnabled()) { + int port = 0; + 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()) { + // Find a local DN and short circuit read is enabled + key += "@" + addr.getHostName() + ":" + port + "/" + DomainSocketFactory.FEATURE_FLAG; + localDNCache.put(key, dn); + break; + } + } + } + + if ((!allowShortCircuit && topologyAware) || isEC) { try { DatanodeDetails closestNode = pipeline.getClosestNode(); + // Pipeline cache key uses host:port suffix to handle // both EC, Ratis, and Standalone client. // @@ -185,7 +256,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,7 +268,7 @@ 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()); @@ -211,12 +282,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) { @@ -263,7 +336,7 @@ public static class ScmClientConfig { @Config(key = "idle.threshold", type = ConfigType.TIME, timeUnit = MILLISECONDS, - defaultValue = "10s", + defaultValue = "300s", tags = {OZONE, PERFORMANCE}, description = "In the standalone pipelines, the SCM clients use netty to " 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 000000000000..f43b6739413c --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java @@ -0,0 +1,645 @@ +/* + * 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.client.ReplicationType; +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.util.Daemon; +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.BufferedInputStream; +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.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.BlockingDeque; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicLong; + +import static org.apache.hadoop.hdds.HddsUtils.processForDebug; +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 BlockingDeque pendingRequests; + private final Daemon writeDaemon; + private final Daemon readDaemon; + private final Timer timer; + + private boolean closed = false; + private final DatanodeDetails dn; + private final InetSocketAddress dnAddr; + private final DomainSocketFactory domainSocketFactory; + private DomainSocket domainSocket; + private DataOutputStream dataOut; + private DataInputStream dataIn; + private final int bufferSize; + private final ByteString clientId = ByteString.copyFrom(UUID.randomUUID().toString().getBytes()); + private final AtomicLong callId = new AtomicLong(0); + private final 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<>(); + this.pendingRequests = new LinkedBlockingDeque<>(); + int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue(); + this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port); + this.bufferSize = config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize(); + this.prefix = "Pipeline-" + pipeline.getId().getId() + "-" + XceiverClientShortCircuit.class.getSimpleName(); + this.timer = new Timer(prefix + "-Timer"); + this.writeDaemon = new Daemon(new SendRequestTask()); + this.readDaemon = new Daemon(new ReceiveResponseTask()); + LOG.info("{} is created", prefix); + } + + /** + * Create the DomainSocket to connect to the local DataNode. + */ + @Override + public void connect() throws IOException { + if (domainSocket != null && domainSocket.isOpen()) { + return; + } + domainSocket = domainSocketFactory.createSocket(readTimeoutMs, writeTimeoutMs, dnAddr); + dataOut = new DataOutputStream(new BufferedOutputStream(domainSocket.getOutputStream(), bufferSize)); + dataIn = new DataInputStream(new BufferedInputStream(domainSocket.getInputStream(),bufferSize)); + writeDaemon.start(); + readDaemon.start(); + } + + /** + * Close the DomainSocket. + */ + @Override + public synchronized void close() { + closed = true; + timer.cancel(); + if (domainSocket != null) { + try { + dataOut.close(); + dataIn.close(); + domainSocket.close(); + dataOut = null; + dataIn = null; + LOG.info("{} is closed for {}", domainSocket.toString(), dn); + } catch (IOException e) { + LOG.warn("Failed to close domain socket for datanode {}", dn, e); + } + } + writeDaemon.interrupt(); + readDaemon.interrupt(); + try { + writeDaemon.join(); + } catch (InterruptedException e) { + Thread.currentThread().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 pipeline {}.", request, pipeline); + } else { + LOG.error(message + " on the pipeline {}.", request.getCmdType(), pipeline); + } + throw ioException; + } + }); + } + + @VisibleForTesting + public XceiverClientReply sendCommandInternal(ContainerCommandRequestProto request) + throws IOException, InterruptedException { + checkOpen(); + final CompletableFuture replyFuture = + new CompletableFuture<>(); + RequestEntry entry = new RequestEntry(request, replyFuture); + // pendingRequests.add(entry); + // CompletableFuture.runAsync(() -> sendRequest(entry), poolExecutor); + 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("This DomainSocket is not connected."); + } + + // try to connect again + if (!domainSocket.isOpen()) { + connect(); + } + } + + @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 callId, DatanodeBlockID blockID) { + String mapKey = callId + 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 class SendRequestTask implements Runnable { + @Override + public void run() { + do { + Thread.currentThread().setName(prefix + "-SendRequest"); + try { + RequestEntry entry = pendingRequests.take(); + ContainerCommandRequestProto request = entry.getRequest(); + long requestTime = System.nanoTime(); + try { + String key = getRequestUniqueID(request); + entry.setSentTimeNs(System.nanoTime()); + 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()); + } + // send version number + dataOut.writeShort(DATA_TRANSFER_VERSION); + // send command type + dataOut.writeShort(type.getNumber()); + // send request body size + dataOut.writeInt(bytes.length); + // send request body + dataOut.write(bytes); + dataOut.flush(); + // LOG.info("send {} {} bytes with id {}", type, bytes.length, key); + } 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() - requestTime); + } + } catch (InterruptedException ex) { + LOG.info("sendRequestTask is interrupted"); + Thread.currentThread().interrupt(); + } + } while (!isClosed()); + } + } + + public void sendRequest(RequestEntry entry) { + ContainerCommandRequestProto request = entry.getRequest(); + long requestTime = System.nanoTime(); + try { + String key = getRequestUniqueID(request); + entry.setSentTimeNs(System.nanoTime()); + 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()); + } + synchronized (dataOut) { + // send version number + dataOut.writeShort(DATA_TRANSFER_VERSION); + // send command type + dataOut.writeShort(type.getNumber()); + // send request body size + dataOut.writeInt(bytes.length); + // send request body + dataOut.write(bytes); + dataOut.flush(); + } + // LOG.info("send {} {} bytes with id {}", type, bytes.length, key); + } 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() - requestTime); + } + } + + public class ReceiveResponseTask implements Runnable { + @Override + public void run() { + long timerTaskCancelledCount = 0; + do { + Thread.currentThread().setName(prefix + "-ReceiveResponse"); + RequestEntry entry = null; + try { + short version = dataIn.readShort(); + if (version != DATA_TRANSFER_VERSION) { + throw new IOException("Version Mismatch (Expected: " + + DATA_TRANSFER_VERSION + ", Received: " + version + " )"); + } + final short typeNumber = dataIn.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + final int size = dataIn.readInt(); + byte[] responseBytes = new byte[size]; + int totalReadSize = dataIn.read(responseBytes); + while (totalReadSize < size) { + int readSize = dataIn.read(responseBytes, totalReadSize, size - totalReadSize); + totalReadSize += readSize; + } + ContainerCommandResponseProto responseProto = ContainerCommandResponseProto.parseFrom(responseBytes); + 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()); + } + // LOG.info("received type {} bytes {} id {}", type, size, key); + // cancel timeout timer task + if (entry.getTimerTask().cancel()) { + timerTaskCancelledCount++; + // purge timer every 1000 cancels + if (timerTaskCancelledCount == 1000) { + timer.purge(); + timerTaskCancelledCount = 0; + } + } + + 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]; + domainSocket.recvFileInputStreams(fis, buf, 0, buf.length); + if (fis[0] == null) { + throw new IOException("the datanode " + dn + " failed to " + + "pass a file descriptor (might have reached open file limit)."); + } + DatanodeBlockID blockID = getBlockResponse.getBlockData().getBlockID(); + String mapKey = responseProto.getCallId() + blockID.toString(); + blockStreamCache.put(mapKey, fis[0]); + metrics.decrPendingContainerOpsMetrics(type); + long endToEndCost = System.nanoTime() - entry.getCreateTimeNs(); + long sentCost = entry.getSentTimeNs() - entry.getCreateTimeNs(); + metrics.addContainerOpsLatency(type, endToEndCost); + if (LOG.isDebugEnabled()) { + LOG.debug("Executed command {} on datanode {}, cost = {}ns {}ns, cmdType = {}", + entry.getRequest(), dn, endToEndCost, sentCost, type); + } + } 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); + } + } catch (EOFException | ClosedChannelException e) { + LOG.info("receiveResponseTask is closed with {}", e.getClass().getName()); + // fail all requests pending responses + sentRequests.values().forEach(i -> + i.getFuture().completeExceptionally(new IOException("DomainSocket InputStream is closed"))); + } catch (Throwable e) { + LOG.error("Failed to receive response", e); + if (entry != null) { + entry.getFuture().completeExceptionally(e); + } + sentRequests.values().forEach(i -> + i.getFuture().completeExceptionally(new IOException("Unexpected failure", e))); + break; + } + } while (!isClosed()); + } + } + + public 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(long nanoSecond) { + sentTimeNs = nanoSecond; + } + + public void setTimerTask(TimerTask task) { + timerTask = task; + } + + public TimerTask getTimerTask() { + return timerTask; + } + } +} 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 f792a678dad4..a6fb34224064 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,6 +276,59 @@ protected BlockData getBlockData() throws IOException { */ protected BlockData getBlockDataUsingClient() throws IOException { Pipeline pipeline = pipelineRef.get(); + if (xceiverClientShortCircuit != null) { + return getBlockDataUsingSCClient(); + } else { + 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); + } + 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); @@ -285,7 +345,7 @@ protected BlockData getBlockDataUsingClient() throws IOException { } GetBlockResponseProto response = ContainerProtocolCalls.getBlock( - xceiverClient, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); + xceiverClientGrpc, VALIDATORS, blockID, tokenRef.get(), pipeline.getReplicaIndexes()); return response.getBlockData(); } @@ -341,13 +401,27 @@ 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); + xceiverClientShortCircuit = + (XceiverClientShortCircuit) xceiverClientFactory.acquireClientForReadData(pipeline, 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(pipelineRef.get(), 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; } } @@ -363,8 +437,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 @@ -536,12 +615,19 @@ 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; } } 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 983bb74989ad..615c582c3857 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 @@ -64,15 +64,15 @@ public class ChunkInputStream extends InputStream private final BlockID blockID; private ContainerProtos.DatanodeBlockID datanodeBlockID; private final XceiverClientFactory xceiverClientFactory; - private XceiverClientSpi xceiverClient; + protected XceiverClientSpi xceiverClient; private final Supplier pipelineSupplier; - private final boolean verifyChecksum; - private boolean allocated = false; + protected final boolean verifyChecksum; + protected boolean allocated = false; // Buffers to store the chunk data read from the DN container - private ByteBuffer[] buffers; + protected ByteBuffer[] buffers; // Index of the buffers corresponding to the current position of the buffers - private int bufferIndex; + protected int bufferIndex; // bufferOffsets[i] stores the index of the first data byte in buffer i // (buffers.get(i)) w.r.t first byte in the buffers. // Let's say each buffer has a capacity of 40 bytes. The bufferOffset for @@ -80,27 +80,27 @@ public class ChunkInputStream extends InputStream // in buffers. BufferOffset for the 2nd buffer would be 40 as bytes 0-39 // would be stored in buffer 0. Hence, bufferOffsets[0] = 0, // bufferOffsets[1] = 40, bufferOffsets[2] = 80, etc. - private long[] bufferOffsets = null; + protected long[] bufferOffsets = null; // The offset of the current data residing in the buffers w.r.t the start // of chunk data - private long bufferOffsetWrtChunkData; + protected long bufferOffsetWrtChunkData; // Index of the first buffer which has not been released - private int firstUnreleasedBufferIndex = 0; + protected int firstUnreleasedBufferIndex = 0; // The number of bytes of chunk data residing in the buffers currently - private long buffersSize; + protected long buffersSize; // Position of the ChunkInputStream is maintained by this variable (if a // seek is performed. This position is w.r.t to the chunk only and not the // block or key. This variable is also set before attempting a read to enable // retry. Once the chunk is read, this variable is reset. - private long chunkPosition = -1; + protected long chunkPosition = -1; - private final Supplier> tokenSupplier; + protected final Supplier> tokenSupplier; - private static final int EOF = -1; + protected static final int EOF = -1; private final List validators; ChunkInputStream(ChunkInfo chunkInfo, BlockID blockId, @@ -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/DomainPeer.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainPeer.java new file mode 100644 index 000000000000..a39a13c0d4b6 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainPeer.java @@ -0,0 +1,111 @@ +/** + * 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 org.apache.hadoop.net.unix.DomainSocket; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.channels.ReadableByteChannel; + +/** + * Represents a peer that we communicate with by using blocking I/O + * on a UNIX domain socket. + */ +public class DomainPeer implements Closeable { + private final DomainSocket socket; + private final OutputStream out; + private final InputStream in; + private final ReadableByteChannel channel; + public static final Logger LOG = LoggerFactory.getLogger(DomainPeer.class); + + public DomainPeer(DomainSocket socket) { + this.socket = socket; + this.out = socket.getOutputStream(); + this.in = socket.getInputStream(); + this.channel = socket.getChannel(); + } + + public ReadableByteChannel getInputStreamChannel() { + return channel; + } + + public void setReadTimeout(int timeoutMs) throws IOException { + socket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, timeoutMs); + } + + public int getReceiveBufferSize() throws IOException { + return socket.getAttribute(DomainSocket.RECEIVE_BUFFER_SIZE); + } + + public void setWriteTimeout(int timeoutMs) throws IOException { + socket.setAttribute(DomainSocket.SEND_TIMEOUT, timeoutMs); + } + + public boolean isClosed() { + return !socket.isOpen(); + } + + public void close() throws IOException { + socket.close(); + LOG.info("{} is closed", socket); + } + + public String getRemoteAddressString() { + return "unix:{" + socket.toString() + "}"; + } + + public String getLocalAddressString() { + return ""; + } + + public InputStream getInputStream() throws IOException { + return in; + } + + public OutputStream getOutputStream() throws IOException { + return out; + } + + @Override + public String toString() { + return "DomainPeer(" + getRemoteAddressString() + ")"; + } + + public DomainSocket getDomainSocket() { + return socket; + } + + public boolean hasSecureChannel() { + // + // Communication over domain sockets is assumed to be secure, since it + // doesn't pass over any network. We also carefully control the privileges + // that can be used on the domain socket inode and its parent directories. + // See #{java.org.apache.hadoop.net.unix.DomainSocket#validateSocketPathSecurity0} + // for details. + // + // So unless you are running as root or the hdfs superuser, you cannot + // launch a man-in-the-middle attach on UNIX domain socket traffic. + // + return true; + } +} 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 new file mode 100644 index 000000000000..cfd3753c24d7 --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DomainSocketFactory.java @@ -0,0 +1,262 @@ +/** + * 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.commons.io.IOUtils; +import org.apache.commons.lang3.SystemUtils; +import org.apache.hadoop.hdds.conf.ConfigurationSource; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.net.unix.DomainSocket; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Closeable; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.util.Timer; +import java.util.TimerTask; +import java.util.concurrent.ConcurrentHashMap; + +public class DomainSocketFactory implements Closeable { + private static final Logger LOG = LoggerFactory.getLogger( + DomainSocketFactory.class); + + public enum PathState { + NOT_CONFIGURED(false), + DISABLED(false), + VALID(true); + + PathState(boolean usableForShortCircuit) { + this.usableForShortCircuit = usableForShortCircuit; + } + + public boolean getUsableForShortCircuit() { + return usableForShortCircuit; + } + private final boolean usableForShortCircuit; + } + + public static class PathInfo { + private final static PathInfo NOT_CONFIGURED = new PathInfo("", PathState.NOT_CONFIGURED); + private final static PathInfo DISABLED = new PathInfo("", PathState.DISABLED); + private final static PathInfo VALID = new PathInfo("", PathState.VALID); + + final private String path; + final private PathState state; + + PathInfo(String path, PathState state) { + this.path = path; + this.state = state; + } + + public String getPath() { + return path; + } + + public PathState getPathState() { + return state; + } + + @Override + public String toString() { + return "PathInfo{path=" + path + ", state=" + state + "}"; + } + } + + /** + * Information about domain socket paths. + */ + public static String FEATURE = "short-circuit local reads"; + public static String FEATURE_FLAG = "SC"; + public static boolean nativeCodeLoaded = false; + public static String nativeLibraryLoadFailureReason; + private long pathExpireMills; + private ConcurrentHashMap pathMap; + public Timer timer; + private boolean isEnabled = false; + private String domainSocketPath; + + static { + // Try to load native hadoop library and set fallback flag appropriately + if (SystemUtils.IS_OS_WINDOWS) { + nativeLibraryLoadFailureReason = "UNIX Domain sockets are not available on Windows."; + } else { + LOG.info("Trying to load the custom-built native-hadoop library..."); + try { + System.loadLibrary("hadoop"); + LOG.info("Loaded the native-hadoop library"); + nativeCodeLoaded = true; + } catch (Throwable t) { + // Ignore failure to load + LOG.info("Failed to load native-hadoop with error: " + t); + LOG.info("java.library.path=" + System.getProperty("java.library.path")); + nativeLibraryLoadFailureReason = "libhadoop cannot be loaded."; + } + + if (!nativeCodeLoaded) { + LOG.warn("Unable to load native-hadoop library for your platform... " + + "using builtin-java classes where applicable"); + } + } + } + + private static volatile DomainSocketFactory instance = null; + + public static synchronized DomainSocketFactory getInstance(ConfigurationSource conf) { + if (instance == null) { + instance = new DomainSocketFactory(conf); + } + return instance; + } + + private DomainSocketFactory(ConfigurationSource conf) { + PathInfo pathInfo; + domainSocketPath = conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH_DEFAULT); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + boolean shortCircuitEnabled = clientConfig.isShortCircuitEnabled(); + long startTime = System.nanoTime(); + if (!shortCircuitEnabled) { + LOG.info(FEATURE + " is disabled."); + pathInfo = PathInfo.NOT_CONFIGURED; + } else { + if (domainSocketPath.isEmpty()) { + throw new IllegalArgumentException(FEATURE + " is enabled but " + + OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH + " is not set."); + } else if (!nativeCodeLoaded) { + LOG.warn(FEATURE + " cannot be used because " + nativeLibraryLoadFailureReason); + pathInfo = PathInfo.DISABLED; + } else { + pathInfo = PathInfo.VALID; + isEnabled = true; + timer = new Timer(DomainSocketFactory.class.getSimpleName()); + LOG.info(FEATURE + " is enabled within {} ns.", System.nanoTime() - startTime); + } + } + pathExpireMills = clientConfig.getShortCircuitReadDisableInterval() * 1000; + pathMap = new ConcurrentHashMap<>(); + pathMap.put(domainSocketPath, pathInfo); + } + + public boolean isServiceEnabled() { + return isEnabled; + } + + public boolean isServiceReady() { + if (isEnabled) { + PathInfo status = pathMap.get(domainSocketPath); + return status.getPathState() == PathState.VALID; + } else { + return false; + } + } + + /** + * Get information about a domain socket path. Caller must make sure that addr is a local address. + * + * @param addr The local inet address to use. + * @return Information about the socket path. + */ + public PathInfo getPathInfo(InetSocketAddress addr) { + if (!isEnabled) { + return PathInfo.NOT_CONFIGURED; + } + + if (!isServiceReady()) { + return PathInfo.DISABLED; + } + + String escapedPath = DomainSocket.getEffectivePath(domainSocketPath, addr.getPort()); + PathInfo status = pathMap.get(escapedPath); + if (status == null) { + PathInfo pathInfo = new PathInfo(escapedPath, PathState.VALID); + pathMap.put(escapedPath, pathInfo); + return pathInfo; + } else { + return status; + } + } + + /** + * Create DomainSocket for addr. Caller must make sure that addr is a local address. + */ + public DomainSocket createSocket(int readTimeoutMs, int writeTimeoutMs, InetSocketAddress addr) throws IOException { + if (!isEnabled || !isServiceReady()) { + return null; + } + boolean success = false; + DomainSocket sock = null; + String escapedPath = null; + long startTime = System.nanoTime(); + try { + escapedPath = DomainSocket.getEffectivePath(domainSocketPath, addr.getPort()); + sock = DomainSocket.connect(escapedPath); + sock.setAttribute(DomainSocket.RECEIVE_TIMEOUT, readTimeoutMs); + sock.setAttribute(DomainSocket.SEND_TIMEOUT, writeTimeoutMs); + success = true; + LOG.info("{} is created within {} ns", sock, System.nanoTime() - startTime); + } catch (IOException e) { + LOG.error("Failed to create DomainSocket", e); + throw e; + } finally { + if (!success) { + if (sock != null) { + IOUtils.closeQuietly(sock); + } + if (escapedPath != null) { + pathMap.put(escapedPath, PathInfo.DISABLED); + LOG.error("{} is disabled for {} ms due to current failure", escapedPath, pathExpireMills); + schedulePathEnable(escapedPath, pathExpireMills); + } + sock = null; + } + } + return sock; + } + + public void disableShortCircuit() { + pathMap.put(domainSocketPath, PathInfo.DISABLED); + schedulePathEnable(domainSocketPath, pathExpireMills); + } + + private void schedulePathEnable(String path, long delayMills) { + timer.schedule(new TimerTask() { + @Override + public void run() { + pathMap.put(path, PathInfo.VALID); + } + }, delayMills); + } + + @VisibleForTesting + public void clearPathMap() { + pathMap.clear(); + } + + public long getPathExpireMills() { + return pathExpireMills; + } + + @Override + public void close() { + if (timer != null) { + timer.cancel(); + } + } +} 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 000000000000..f9ed391aa87d --- /dev/null +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ShortCircuitChunkInputStream.java @@ -0,0 +1,140 @@ +/* + * 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; + public static final Logger LOG = + LoggerFactory.getLogger(ShortCircuitChunkInputStream.class); + + 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; + } + + @Override + protected void readChunkDataIntoBuffers(ChunkInfo readChunkInfo) + throws IOException { + buffers = readChunk(readChunkInfo); + buffersSize = readChunkInfo.getLen(); + + bufferOffsets = new long[buffers.length]; + int tempOffset = 0; + for (int i = 0; i < buffers.length; i++) { + bufferOffsets[i] = tempOffset; + tempOffset += buffers[i].limit(); + } + + bufferIndex = 0; + firstUnreleasedBufferIndex = 0; + allocated = true; + } + + /** + * 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 { + if (xceiverClientShortCircuit != null && !xceiverClientShortCircuit.isClosed()) { + xceiverClient = xceiverClientShortCircuit; + } else { + throw new IOException(DomainSocketFactory.FEATURE + " client for " + + xceiverClientShortCircuit.getDn().getHostName() + " is already closed"); + } + } +} diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestDomainSocketFactory.java b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestDomainSocketFactory.java new file mode 100644 index 000000000000..6c61252509a9 --- /dev/null +++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestDomainSocketFactory.java @@ -0,0 +1,259 @@ +/* + * 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 jdk.nashorn.internal.ir.annotations.Ignore; +import org.apache.hadoop.hdds.client.BlockID; +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.net.unix.DomainSocket; +import org.junit.jupiter.api.Test; + +import java.io.BufferedInputStream; +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FileInputStream; +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.UUID; + +import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Type.GetBlock; +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; + +/** + * Tests for {@link DomainSocketFactory}'s functionality. + */ +public class TestDomainSocketFactory { + + InetSocketAddress localhost = InetSocketAddress.createUnresolved("localhost", 10000); + // Add "-Djava.library.path=${native_lib_path}" to intellij run configuration to run it locally + // Dynamically set the java.library.path in java code doesn't affect the library loading + @Test + @Ignore + public void test() throws IOException { + // enable short-circuit read + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, true); + OzoneClientConfig clientConfig = conf.getObject(OzoneClientConfig.class); + clientConfig.setShortCircuitReadDisableInterval(1); + conf.setFromObject(clientConfig); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, "/Users/sammi/ozone_dn_socket"); + + // create DomainSocketFactory + DomainSocketFactory domainSocketFactory = DomainSocketFactory.getInstance(conf); + assertTrue(conf.getBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, + OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT_DEFAULT)); + assertTrue(domainSocketFactory.isServiceEnabled()); + assertTrue(domainSocketFactory.isServiceReady()); + + try { + // testShortCircuitDisableTemporary(domainSocketFactory); + // testMaxXceiverCount(domainSocketFactory); + // testSendIrrelevantMessage(domainSocketFactory); + testSendCorrectMessage(domainSocketFactory); + } finally { + domainSocketFactory.close(); + } + } + + private void testSendCorrectMessage(DomainSocketFactory factory) throws IOException { + // test send irrelevant message to server + DomainSocket sock = factory.createSocket(1000, 1000, localhost); + final DataInputStream inputStream = new DataInputStream(new BufferedInputStream(sock.getInputStream())); + final DataOutputStream outputStream = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream())); + byte version = 28; + try { + outputStream.writeShort(version); + outputStream.writeShort(GetBlock.getNumber()); + long value = 1; + String datanodeUUID = UUID.randomUUID().toString(); + ContainerProtos.GetBlockRequestProto.Builder getBlock = + ContainerProtos.GetBlockRequestProto.newBuilder() + .setBlockID(new BlockID(value, value).getDatanodeBlockIDProtobuf()).setRequestShortCircuitAccess(true); + ContainerProtos.ContainerCommandRequestProto getBlockRequest = + ContainerProtos.ContainerCommandRequestProto.newBuilder() + .setCmdType(ContainerProtos.Type.GetBlock) + .setContainerID(value) + .setGetBlock(getBlock) + .setDatanodeUuid(datanodeUUID) + .build(); + byte[] requestBytes = getBlockRequest.toByteArray(); + outputStream.writeInt(requestBytes.length); + System.out.println("send request with size " + requestBytes.length); + outputStream.write(requestBytes); + outputStream.flush(); + } catch (IOException e) { + e.printStackTrace(); + } + short ret = 0; + try { + ret = inputStream.readShort(); + assertEquals(version, ret); + ret = inputStream.readShort(); + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), ret); + int size = inputStream.readInt(); + byte[] response = new byte[size]; + inputStream.read(response); + ContainerProtos.ContainerCommandResponseProto responseProto = + ContainerProtos.ContainerCommandResponseProto.parseFrom(response); + assertEquals(ContainerProtos.Type.GetBlock.getNumber(), responseProto.getCmdType().getNumber()); + ContainerProtos.GetBlockResponseProto getBlockResponseProto = responseProto.getGetBlock(); + assertEquals(ContainerProtos.Result.SUCCESS, responseProto.getResult()); + assertTrue(getBlockResponseProto.getShortCircuitAccessGranted()); + + // read FSD + // read FS from domainSocket + FileInputStream[] fis = new FileInputStream[1]; + byte buf[] = new byte[1]; + sock.recvFileInputStreams(fis, buf, 0, buf.length); + assertNotNull(fis[0]); + FileChannel dataIn = fis[0].getChannel(); + int chunkSize = 1024 * 1024; + dataIn.position(chunkSize/2); + ByteBuffer dataBuf = ByteBuffer.allocate(chunkSize); + int round = 10; + sock.close(); + System.out.println("DomainSocket is closed " + new Date()); + Thread.sleep(30000); + for (int i = 0; i < round; i++) { + dataIn.position(0); + int readSize = dataIn.read(dataBuf); + assertEquals(chunkSize, readSize); + System.out.println("received " + readSize + " bytes data"); + dataBuf.flip(); + readSize = dataIn.read(dataBuf); + assertEquals(-1, readSize); + } + dataIn.close(); + fis[0].close(); + System.out.println("File InputStream is closed"); + Thread.sleep(30000); + } catch (IOException | InterruptedException e) { + e.printStackTrace(); + } + } + + private void testSendIrrelevantMessage(DomainSocketFactory factory) throws IOException { + // test send irrelevant message to server + DomainSocket sock = factory.createSocket(1000, 1000, localhost); + DomainSocket.DomainOutputStream outputStream = sock.getOutputStream(); + DomainSocket.DomainInputStream inputStream = sock.getInputStream(); + String data = "hello world"; + try { + outputStream.write(data.getBytes()); + } catch (IOException e) { + fail("should not fail"); + } + int ret = 0; + try { + ret = inputStream.read(); + } catch (IOException e) { + fail("should not fail"); + } + assertEquals(-1, ret); + try { + outputStream.write(data.getBytes()); + } catch (Exception e) { + e.printStackTrace(); + } + assertTrue(sock.isOpen()); + try { + sock.close(); + } catch (IOException e) { + fail("should not fail"); + } + } + + private void testShortCircuitDisableTemporary(DomainSocketFactory factory) { + // 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()); + } + + private void testMaxXceiverCount(DomainSocketFactory factory) throws IOException { + // test max allowed xceiver count(default 10) + int count = 11; + List list = new ArrayList<>(); + for (int i = 1; i <= count; i++) { + DomainSocket sock = factory.createSocket(1000, 1000, localhost); + list.add(sock); + System.out.println("Created DomainSocket " + sock.toString()); + } + try { + Thread.sleep(5000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + // write to first 10 sockets should be OK + for (int i = 0; i < count - 2; i++) { + DomainSocket sock = list.get(i); + assertTrue(sock.isOpen()); + try { + sock.getOutputStream().write(1); + sock.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + assertFalse(sock.isOpen()); + } + + // read/write the last socket which remote peer is closed by server already + DomainSocket sock = list.get(list.size() - 1); + assertTrue(sock.isOpen()); + try { + sock.getOutputStream().write(1); + fail("Write to a peer closed socket should fail"); + } catch (Exception e) { + e.printStackTrace(); + } + try { + int data = sock.getInputStream().read(); + System.out.println("Read from a peer closed socket returns " + data); + assertEquals(-1, data); + } catch (Exception e) { + e.printStackTrace(); + } + try { + sock.close(); + } catch (IOException e) { + throw new RuntimeException(e); + } + assertFalse(sock.isOpen()); + } +} 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 e35d20d53e15..0b1d538fb184 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, "Support short-circuit reads."), + 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/ReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java index 7542409679b0..57db149047e4 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationConfig.java @@ -49,6 +49,8 @@ static ReplicationConfig fromProtoTypeAndFactor( return RatisReplicationConfig.getInstance(factor); case STAND_ALONE: return StandaloneReplicationConfig.getInstance(factor); + case SHORT_CIRCUIT: + return ShortCircuitReplicationConfig.getInstance(factor); default: throw new UnsupportedOperationException( "Not supported replication: " + type); @@ -102,6 +104,7 @@ static ReplicationConfig fromProto( return new ECReplicationConfig(ecConfig); case RATIS: case STAND_ALONE: + case SHORT_CIRCUIT: return fromProtoTypeAndFactor(type, factor); default: throw new UnsupportedOperationException( diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java index 64969eac4226..f4dc891f47a7 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ReplicationType.java @@ -27,7 +27,8 @@ public enum ReplicationType { RATIS, STAND_ALONE, CHAINED, - EC; + EC, + SHORT_CIRCUIT; public static ReplicationType fromProto( HddsProtos.ReplicationType replicationType) { @@ -43,6 +44,8 @@ public static ReplicationType fromProto( return ReplicationType.CHAINED; case EC: return ReplicationType.EC; + case SHORT_CIRCUIT: + return ReplicationType.SHORT_CIRCUIT; default: throw new IllegalArgumentException( "Unsupported ProtoBuf replication type: " + replicationType); @@ -63,6 +66,8 @@ public static HddsProtos.ReplicationType toProto( return HddsProtos.ReplicationType.CHAINED; case EC: return HddsProtos.ReplicationType.EC; + case SHORT_CIRCUIT: + return HddsProtos.ReplicationType.SHORT_CIRCUIT; default: throw new IllegalArgumentException( "Unsupported replication type: " + replicationType); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java new file mode 100644 index 000000000000..aa72e8cacf3e --- /dev/null +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/ShortCircuitReplicationConfig.java @@ -0,0 +1,129 @@ +/** + * 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.client; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import net.jcip.annotations.Immutable; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType; + +import java.util.Objects; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.THREE; + +/** + * Replication configuration for SHORT_CIRCUIT replication. + */ +@Immutable +public final class ShortCircuitReplicationConfig implements + ReplicatedReplicationConfig { + + private final ReplicationFactor replicationFactor; + private static final String REPLICATION_TYPE = "SHORT_CIRCUIT"; + + private static final ShortCircuitReplicationConfig SHORT_CIRCUIT_ONE_CONFIG = + new ShortCircuitReplicationConfig(ONE); + + private static final ShortCircuitReplicationConfig SHORT_CIRCUIT_THREE_CONFIG = + new ShortCircuitReplicationConfig(THREE); + + /** + * Get an instance of Short-circuit Replication Config with the requested factor. + * The same static instance will be returned for all requests for the same + * factor. + * @param factor Replication Factor requested + * @return ShortCircuitReplicationConfig object of the requested factor + */ + public static ShortCircuitReplicationConfig getInstance( + ReplicationFactor factor) { + if (factor == ONE) { + return SHORT_CIRCUIT_ONE_CONFIG; + } else if (factor == THREE) { + return SHORT_CIRCUIT_THREE_CONFIG; + } + return new ShortCircuitReplicationConfig(factor); + } + + /** + * Use the static getInstance method instead of the private constructor. + * @param replicationFactor + */ + private ShortCircuitReplicationConfig(ReplicationFactor replicationFactor) { + this.replicationFactor = replicationFactor; + } + + @Override + public ReplicationFactor getReplicationFactor() { + return replicationFactor; + } + + @Override + public int getRequiredNodes() { + return replicationFactor.getNumber(); + } + + @Override + @JsonIgnore + public String getReplication() { + return String.valueOf(this.replicationFactor); + } + + @Override + public ReplicationType getReplicationType() { + return ReplicationType.SHORT_CIRCUIT; + } + + /** + * This method is here only to allow the string value for replicationType to + * be output in JSON. + */ + @JsonProperty("replicationType") + public String replicationType() { + return REPLICATION_TYPE; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShortCircuitReplicationConfig that = (ShortCircuitReplicationConfig) o; + return replicationFactor == that.replicationFactor; + } + + @Override + public int hashCode() { + return Objects.hash(replicationFactor); + } + + @Override + public String toString() { + return REPLICATION_TYPE + "/" + replicationFactor; + } + + @Override + public String configFormat() { + return toString(); + } +} 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 9ac32c469ca4..f15df161a8b3 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/protocolPB/ContainerCommandResponseBuilders.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/ContainerCommandResponseBuilders.java index d3f39c023b73..5dbfdc8fd7bb 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 cb41479b5f3e..a8b0c49a11df 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,18 @@ 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 + */ + 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 +779,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 +791,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 f4efe1790bf7..d598aea4ca98 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 df0fdc59a4ae..9d5484173b8a 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 @@ -474,8 +474,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 f8b3febfeca8..190b7a6cfaa4 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 855153f39ed9..95fd356486ea 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 @@ -21,9 +21,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; 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 +93,32 @@ private static String getHostNameWithoutDomain(final String fqdn) { } + private static final Map localAddrMap = Collections + .synchronizedMap(new HashMap()); /** * Match input address to local address. * Return true if it matches, false otherwsie. */ public static boolean isAddressLocal(InetSocketAddress addr) { InetAddress inetAddress = addr.getAddress(); - return inetAddress != null && NetUtils.isLocalAddress(inetAddress); + if (inetAddress == null) { + return false; + } + Boolean cached = localAddrMap.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")); + } + localAddrMap.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 c1b65e10bc9e..b56ea59e08e9 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,40 @@ public void testUuidConversion() { UUID deserialized = fromProtobuf(protobuf); assertEquals(original, deserialized); } + + @Test + public void testContainerCommandRequestProtoConversion() { + 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(); + + try { + 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(bcsid, request.getGetBlock().getBlockID().getBlockCommitSequenceId()); + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } } 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 2b7592e1c356..952396084aae 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 883f6cd851e3..63e8ea93a58a 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 @@ -31,7 +31,9 @@ import org.apache.hadoop.metrics2.lib.MutableQuantiles; import org.apache.hadoop.metrics2.lib.MutableRate; +import java.util.ArrayList; import java.util.EnumMap; +import java.util.List; /** * @@ -57,20 +59,32 @@ public class ContainerMetrics { @Metric private MutableCounterLong numReadStateMachine; @Metric private MutableCounterLong bytesReadStateMachine; - + // for remote request private final EnumMap numOpsArray; private final EnumMap opsBytesArray; private final EnumMap opsLatency; private final EnumMap opsLatQuantiles; + // for local short-circuit request + private final EnumMap numLocalOpsArray; + private final EnumMap opsLocalBytesArray; + private final EnumMap opsLocalLatency; + private final EnumMap opsLocalLatencyQuantiles; + private final EnumMap opsLocalInQueueLatency; 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); this.opsLatQuantiles = new EnumMap<>(ContainerProtos.Type.class); + + this.numLocalOpsArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalBytesArray = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalLatency = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalLatencyQuantiles = new EnumMap<>(ContainerProtos.Type.class); + this.opsLocalInQueueLatency = new EnumMap<>(ContainerProtos.Type.class); + this.registry = new MetricsRegistry("StorageContainerMetrics"); for (ContainerProtos.Type type : ContainerProtos.Type.values()) { @@ -80,6 +94,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"; @@ -88,6 +103,25 @@ public ContainerMetrics(int[] intervals) { } opsLatQuantiles.put(type, latQuantiles); } + 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)); + opsLocalLatency.put(type, registry.newRate("localLatency" + type, type + " op")); + opsLocalInQueueLatency.put(type, registry.newRate("localInQueueLatency" + type, type + " op")); + MutableQuantiles[] latQuantiles = new MutableQuantiles[len]; + for (int j = 0; j < len; j++) { + int interval = intervals[j]; + String quantileName = "local" + type + "Nanos" + interval + "s"; + latQuantiles[j] = registry.newQuantiles(quantileName, + "latency of Container ops", "ops", "latency", interval); + } + opsLocalLatencyQuantiles.put(type, latQuantiles); + } } public static ContainerMetrics create(ConfigurationSource conf) { @@ -117,6 +151,22 @@ public void incContainerOpsLatencies(ContainerProtos.Type type, long latencyNs) } } + public void incContainerLocalOpsMetrics(ContainerProtos.Type type) { + numOps.incr(); + numLocalOpsArray.get(type).incr(); + } + + public void incContainerLocalOpsLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalLatency.get(type).add(nanoSeconds); + for (MutableQuantiles q: opsLocalLatencyQuantiles.get(type)) { + q.add(nanoSeconds); + } + } + + public void incContainerLocalOpsInQueueLatencies(ContainerProtos.Type type, long nanoSeconds) { + opsLocalInQueueLatency.get(type).add(nanoSeconds); + } + public void incContainerBytesStats(ContainerProtos.Type type, long bytes) { opsBytesArray.get(type).incr(bytes); } 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 bfdff69be46f..c08495344c24 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; @@ -33,10 +34,12 @@ import org.apache.hadoop.ozone.container.common.impl.ContainerData; import org.apache.hadoop.ozone.container.common.impl.ContainerSet; import org.apache.hadoop.ozone.container.common.report.IncrementalReportSender; +import org.apache.hadoop.ozone.container.common.transport.server.XceiverServerSpi; import org.apache.hadoop.ozone.container.common.transport.server.ratis.DispatcherContext; 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; @@ -55,6 +58,7 @@ public abstract class Handler { protected final ContainerMetrics metrics; protected String datanodeId; private IncrementalReportSender icrSender; + private OzoneContainer ozoneContainer; protected Handler(ConfigurationSource config, String datanodeId, ContainerSet contSet, VolumeSet volumeSet, @@ -73,14 +77,21 @@ 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); + } + + 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); - default: - throw new IllegalArgumentException("Handler for ContainerType: " + - containerType + "doesn't exist."); + case KeyValueContainer: + return new KeyValueHandler(config, datanodeId, contSet, volumeSet, metrics, + icrSender, ozoneContainer); + default: + throw new IllegalArgumentException("Handler for ContainerType: " + + containerType + "doesn't exist."); } } @@ -221,4 +232,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 000000000000..93375b2a1663 --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/Receiver.java @@ -0,0 +1,298 @@ +/** + * 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.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_VERSION; + +/** + * Class for processing incoming/outgoing requests. + */ +class Receiver implements XceiverClientProtocol, 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; + private final AtomicLong opsHandled = new AtomicLong(0); + + 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; + ContainerCommandRequestProto requestProto = null; + try { + domainSocketServer.addPeer(peer, Thread.currentThread(), this); + // "java.net.SocketException: setsockopt(SO_SNDTIMEO) error: Invalid argument" on MacOS + // peer.setWriteTimeout(domainSocketServer.writeTimeout); + // peer.setReadTimeout(domainSocketServer.readTimeout); + 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 { + requestProto = 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(requestProto, output, opsHandled)); + ++opsReceived; + // reset request variable + requestProto = null; + } while (peer != null && !peer.isClosed()); + } catch (Throwable t) { + String s = "Receiver error processing " + + ((requestProto == null) ? "unknown" : requestProto.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 final ContainerCommandRequestProto 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 + " )"); + } + // second short is ContainerProtos#Type + final short typeNumber = in.readShort(); + ContainerProtos.Type type = ContainerProtos.Type.forNumber(typeNumber); + // first 4 bytes indicates the serialized ContainerCommandRequestProto size + final int size = in.readInt(); + // LOG.info("received {} {} bytes", type, size); + byte[] bytes = new byte[size]; + int totalReadSize = in.read(bytes); + while (totalReadSize < size) { + int readSize = in.read(bytes, totalReadSize, size - totalReadSize); + totalReadSize += readSize; + } + ContainerCommandRequestProto requestProto = ContainerCommandRequestProto.parseFrom(bytes); + if (requestProto.getCmdType() != type) { + throw new IOException("Type mismatch, " + type + " in header while " + requestProto. getCmdType() + + " in request body"); + } + return requestProto; + } + + /** Process the request **/ + public class processRequestTask implements Runnable { + private final ContainerCommandRequestProto request; + private final DataOutputStream out; + private final long createTime; + private final AtomicLong counter; + + public processRequestTask(ContainerCommandRequestProto request, DataOutputStream out, AtomicLong counter) { + this.request = request; + this.out = out; + this.counter = counter; + this.createTime = System.nanoTime(); + } + + @Override + public void run() { + ContainerProtos.Type type = request.getCmdType(); + Span span = TracingUtil.importAndCreateSpan("XceiverServerDomainSocket." + type.name(), + request.getTraceID()); + try (Scope scope = GlobalTracer.get().activateSpan(span)) { + metrics.incContainerLocalOpsMetrics(type); + metrics.incContainerLocalOpsInQueueLatencies(type, System.nanoTime() - createTime); + long startTime = System.nanoTime(); + ContainerCommandResponseProto responseProto = dispatcher.dispatch(request, null); + FileInputStream fis = null; + if (responseProto.getResult() == ContainerProtos.Result.SUCCESS && type == ContainerProtos.Type.GetBlock) { + // get FileDescriptor + Handler handler = dispatcher.getHandler(ContainerProtos.ContainerType.KeyValueContainer); + fis = handler.getBlockInputStream(request); + Preconditions.checkNotNull(fis, + "Failed to get block InputStream for block " + request.getGetBlock().getBlockID()); + } + ResponseEntry responseEntry = new ResponseEntry(responseProto, fis, System.nanoTime() - startTime); + sendResponse(responseEntry); + } catch (Throwable e) { + LOG.error("Failed to processRequest {} {} {}", type, request.getClientId(), request.getCallId(), e); + } finally { + span.finish(); + counter.incrementAndGet(); + } + } + } + + void sendResponse(ResponseEntry entry) { + byte buf[] = new byte[1]; + buf[0] = (byte) 99; + ContainerCommandResponseProto responseProto = entry.getResponse(); + ContainerProtos.Type type = responseProto.getCmdType(); + synchronized (output) { + long cost = 0; + FileInputStream fis = entry.getFis(); + try { + byte[] bytes = responseProto.toByteArray(); + output.writeShort(DATA_TRANSFER_VERSION); + output.writeShort(type.getNumber()); + output.writeInt(bytes.length); + // send response proto + output.write(bytes); + long startTime = System.nanoTime(); + output.flush(); + cost = System.nanoTime() - startTime; + 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 for buf. + sock.sendFileDescriptors(fds, buf, 0, buf.length); + } + } catch (Throwable e) { + LOG.error("Failed to send response {}", responseProto.getCmdType(), e); + } finally { + // metrics.incContainerLocalOpsLatencies(type, entry.getProcessTimeNs()); + 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); + } + } + metrics.incContainerLocalOpsLatencies(type, cost); + opsHandled.incrementAndGet(); + } + } + } + + @Override + public ContainerCommandResponseProto send(ContainerCommandRequestProto request) throws IOException { + throw new UnsupportedOperationException("Operation is not supported for " + this.getClass().getSimpleName()); + } + + class ResponseEntry { + ContainerCommandResponseProto response; + FileInputStream fis; + long processTimeNs; + + ResponseEntry(ContainerCommandResponseProto responseProto, FileInputStream fis, long processTimeNs) { + this.response = responseProto; + this.fis = fis; + this.processTimeNs = processTimeNs; + } + + public ContainerCommandResponseProto getResponse() { + return response; + } + + public FileInputStream getFis() { + return fis; + } + + public long getProcessTimeNs() { + return processTimeNs; + } + } +} diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverClientProtocol.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverClientProtocol.java new file mode 100644 index 000000000000..497ab5a0541a --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverClientProtocol.java @@ -0,0 +1,44 @@ +/** + * 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 org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto; +import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto; +import org.apache.hadoop.security.KerberosInfo; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_KERBEROS_PRINCIPAL_KEY; + +/** + * Security protocol for a secure OzoneManager. + */ +@KerberosInfo( + serverPrincipal = DFS_DATANODE_KERBEROS_PRINCIPAL_KEY +) +public interface XceiverClientProtocol { + + /** + * Send the client request, and receive datanode response. + * + * @param request the request sent by client + * @return Token + * @throws IOException + */ + ContainerCommandResponseProto send(ContainerCommandRequestProto request) throws IOException; +} 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 000000000000..c15c48204add --- /dev/null +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java @@ -0,0 +1,317 @@ +/* + * 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 { + private 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. + */ + 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<>(); + protected int readTimeoutMs; + protected 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()) { + int port = conf.getInt(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT, OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT); + 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; + } + } + } + + @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(); + 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() { + DomainPeer peer = null; + while (isRunning.get()) { + try { + DomainSocket connSock = domainSocket.accept(); + peer = new DomainPeer(connSock); + peer.setReadTimeout(readTimeoutMs); + peer.setWriteTimeout(writeTimeoutMs); + LOG.debug("Accepted a new connection. xceriverCount {}", xceriverCount.get()); + + // Make sure the xceiver count is not exceeded + if (xceriverCount.get() >= maxXceiverCount) { + throw new IOException("Xceiver count " + xceriverCount.get() + + " exceeds the limit of concurrent xceivers: " + 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", 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 dispatcher) { + this.dispatcher = dispatcher; + } +} 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 ad9c5c9d9ca0..c1e970dcc95a 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 @@ -65,6 +65,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; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED; import static org.apache.hadoop.ozone.OzoneConfigKeys.OZONE_EC_GRPC_ZERO_COPY_ENABLED_DEFAULT; @@ -83,7 +84,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; @@ -92,9 +92,8 @@ public final class XceiverServerGrpc implements XceiverServerSpi { * * @param conf - Configuration */ - public XceiverServerGrpc(DatanodeDetails datanodeDetails, - ConfigurationSource conf, - ContainerDispatcher dispatcher, CertificateClient caClient) { + public XceiverServerGrpc(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, ThreadPoolExecutor executor, CertificateClient caClient) { Preconditions.checkNotNull(conf); this.id = datanodeDetails.getUuid(); @@ -107,19 +106,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) @@ -128,10 +131,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; } final boolean zeroCopyEnabled = conf.getBoolean( @@ -167,6 +170,12 @@ public XceiverServerGrpc(DatanodeDetails datanodeDetails, storageContainer = dispatcher; } + @VisibleForTesting + public XceiverServerGrpc(DatanodeDetails datanodeDetails, ConfigurationSource conf, + ContainerDispatcher dispatcher, CertificateClient caClient) { + this(datanodeDetails, conf, dispatcher, null, caClient); + } + @Override public int getIPCPort() { return this.port; @@ -207,8 +216,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(); @@ -220,6 +227,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 480561270ec4..7cafeaecc53f 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 7899cdcc0e67..e7f34a712c87 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 1bcb64200b22..de108391c641 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; @@ -148,6 +151,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, @@ -155,7 +161,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(); @@ -658,16 +675,29 @@ ContainerCommandResponseProto handleGetBlock( } ContainerProtos.BlockData responseData; + boolean shortCircuitGranted = false; try { - BlockID blockID = BlockID.getFromProtobuf( - request.getGetBlock().getBlockID()); - if (replicaIndexCheckRequired(request)) { - BlockUtils.verifyReplicaIdx(kvContainer, blockID); - } + ContainerProtos.GetBlockRequestProto getBlock = request.getGetBlock(); + BlockID blockID = BlockID.getFromProtobuf(getBlock.getBlockID()); 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); - } catch (StorageContainerException ex) { return ContainerUtils.logAndReturnError(LOG, ex, request); } catch (IOException ex) { @@ -676,7 +706,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/BlockManagerImpl.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java index 6232b843567c..6a39304e25c6 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/BlockManagerImpl.java @@ -18,6 +18,7 @@ package org.apache.hadoop.ozone.container.keyvalue.impl; +import java.io.FileInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -29,6 +30,10 @@ import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; import org.apache.hadoop.hdds.utils.db.BatchOperation; import org.apache.hadoop.hdds.utils.db.Table; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil; import org.apache.hadoop.ozone.container.common.helpers.BlockData; import org.apache.hadoop.ozone.container.common.interfaces.Container; import org.apache.hadoop.ozone.container.common.interfaces.DBHandle; @@ -43,7 +48,6 @@ import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.BCSID_MISMATCH; import static org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result.UNSUPPORTED_REQUEST; import static org.apache.hadoop.ozone.OzoneConsts.INCREMENTAL_CHUNK_LIST; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6a1d5533cf2c..0df62ad8e0f1 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 4ca578d7717a..a92a896015d3 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,10 @@ 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.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DatanodeUtil; +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 +50,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 +60,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 +214,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 +396,4 @@ public void close() { } } } - } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerChunkStrategy.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerChunkStrategy.java index 6ac88cad7f5c..839af4472b87 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerChunkStrategy.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/impl/FilePerChunkStrategy.java @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileInputStream; import java.io.IOException; import java.nio.file.Files; import java.nio.file.StandardCopyOption; diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java index 256d357a31dc..873a2d8bbb97 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/keyvalue/interfaces/BlockManager.java @@ -61,6 +61,7 @@ long putBlock(Container container, BlockData data, boolean endOfBlock) BlockData getBlock(Container container, BlockID blockID) throws IOException; + /** * Deletes an existing block. * 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 7751dba429de..39862918ecb5 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 e40fa635c121..4f9779ae716a 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.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics; @@ -46,6 +48,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; @@ -60,6 +63,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; @@ -77,7 +81,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; @@ -113,6 +119,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; @@ -207,7 +216,7 @@ 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); @@ -235,15 +244,39 @@ public OzoneContainer(HddsDatanodeService hddsDatanodeService, volumeSet), datanodeDetails.threadNamePrefix()); + 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, hddsDispatcher, certClient); - Duration blockDeletingSvcInterval = dnConf.getBlockDeletionInterval(); + datanodeDetails, config, hddsDispatcher, readExecutors, 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); @@ -482,6 +515,9 @@ public void start(String clusterId) throws IOException { hddsDispatcher.setClusterId(clusterId); writeChannel.start(); readChannel.start(); + if (readDomainSocketChannel != null) { + readDomainSocketChannel.start(); + } blockDeletingService.start(); recoveringContainerScrubbingService.start(); @@ -498,7 +534,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); @@ -547,6 +595,10 @@ public XceiverServerSpi getReadChannel() { return readChannel; } + public XceiverServerSpi getReadDomainSocketChannel() { + return readDomainSocketChannel; + } + public ContainerController getController() { return controller; } diff --git a/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestXceiverServerDomainSocket.java b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestXceiverServerDomainSocket.java new file mode 100644 index 000000000000..38a8e302f1b2 --- /dev/null +++ b/hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/common/TestXceiverServerDomainSocket.java @@ -0,0 +1,204 @@ +/* + * 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; + +import com.google.common.collect.Maps; +import jdk.nashorn.internal.ir.annotations.Ignore; +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.RandomStringUtils; +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.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.ScmConfigKeys; +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.ozone.OzoneConfigKeys; +import org.apache.hadoop.ozone.common.Checksum; +import org.apache.hadoop.ozone.common.ChunkBuffer; +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.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Paths; +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.junit.jupiter.api.Assertions.assertTrue; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +/** + * Tests the XceiverServerDomainSocket class. + */ +public class TestXceiverServerDomainSocket { + // Add "-Djava.library.path=${native_lib_path}" to intellij run configuration to run it locally + // Dynamically set the java.library.path in java code doesn't affect the library loading + @Test + @Ignore + public void test() { + // enable short-circuit read + OzoneConfiguration conf = new OzoneConfiguration(); + conf.setBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, true); + OzoneClientConfig ozoneClientConfig = conf.getObject(OzoneClientConfig.class); + ozoneClientConfig.setShortCircuitReadDisableInterval(1); + conf.setFromObject(ozoneClientConfig); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, "/Users/sammi/ozone_dn_socket"); + + // create DomainSocketFactory + DomainSocketFactory domainSocketFactory = DomainSocketFactory.getInstance(conf); + assertTrue(conf.getBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, + OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT_DEFAULT)); + assertTrue(domainSocketFactory.isServiceEnabled()); + assertTrue(domainSocketFactory.isServiceReady()); + + XceiverServerDomainSocket serverDomainSocket = null; + String path = GenericTestUtils.getRandomizedTempPath(); + System.out.println("Randomized temp path " + path); + try { + ThreadPoolExecutor readExecutors = new ThreadPoolExecutor(1, 1, + 60, TimeUnit.SECONDS, + new LinkedBlockingQueue<>()); + ContainerMetrics metrics = ContainerMetrics.create(conf); + serverDomainSocket = new XceiverServerDomainSocket(MockDatanodeDetails.randomDatanodeDetails(), + conf, null, readExecutors, metrics, domainSocketFactory); + serverDomainSocket.setContainerDispatcher( + createDispatcherAndPrepareData(conf, path, serverDomainSocket, metrics)); + serverDomainSocket.start(); + Thread.sleep(600 * 1000); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + if (domainSocketFactory != null) { + domainSocketFactory.close(); + } + if (serverDomainSocket != null) { + serverDomainSocket.stop(); + } + try { + FileUtils.deleteDirectory(new File(path)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + private ContainerDispatcher createDispatcherAndPrepareData(OzoneConfiguration conf, String path, + XceiverServerDomainSocket domainSocketServer, ContainerMetrics metrics) throws IOException { + DatanodeDetails datanodeDetails = randomDatanodeDetails(); + conf.set(ScmConfigKeys.HDDS_DATANODE_DIR_KEY, path); + conf.set(OzoneConfigKeys.OZONE_METADATA_DIRS, path); + 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(Paths.get(path).toFile()); + 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, metrics, 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-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto b/hadoop-hdds/interface-client/src/main/proto/DatanodeClientProtocol.proto index 6cfae24d41e7..6df204b94704 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,13 +369,14 @@ message FinalizeBlockResponseProto { message GetBlockRequestProto { required DatanodeBlockID blockID = 1; + optional bool requestShortCircuitAccess = 2 [default = false]; } message GetBlockResponseProto { required BlockData blockData = 1; + optional bool shortCircuitAccessGranted = 2; } - message DeleteBlockRequestProto { required DatanodeBlockID blockID = 1; } @@ -405,7 +415,6 @@ message EchoResponseProto { } // Chunk Operations - message ChunkInfo { required string chunkName = 1; required uint64 offset = 2; diff --git a/hadoop-hdds/interface-client/src/main/proto/hdds.proto b/hadoop-hdds/interface-client/src/main/proto/hdds.proto index 6cd4f6235ce7..0aecdaf7b966 100644 --- a/hadoop-hdds/interface-client/src/main/proto/hdds.proto +++ b/hadoop-hdds/interface-client/src/main/proto/hdds.proto @@ -286,6 +286,7 @@ enum ReplicationType { STAND_ALONE = 2; CHAINED = 3; EC = 4; + SHORT_CIRCUIT = 5; NONE = -1; // Invalid Type } 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 6edcca65f926..251871451891 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/dist/src/main/compose/ozone/docker-config b/hadoop-ozone/dist/src/main/compose/ozone/docker-config index a657f22340e8..8430aa45587c 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozone/docker-config @@ -64,3 +64,5 @@ no_proxy=om,scm,s3g,recon,kdc,localhost,127.0.0.1 # Explicitly enable filesystem snapshot feature for this Docker compose cluster OZONE-SITE.XML_ozone.filesystem.snapshot.enabled=true + +OZONE-SITE.XML_ozone.client.read.short-circuit=true \ No newline at end of file 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 95a0b0e17fd9..3555863078d7 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 @@ -21,6 +21,7 @@ import org.apache.hadoop.hdds.scm.XceiverClientManager.ScmClientConfig; import org.apache.hadoop.hdds.scm.client.ClientTrustManager; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -48,7 +49,9 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Test for XceiverClientManager caching and eviction. @@ -278,4 +281,37 @@ public void testFreeByRetryFailure() throws IOException { clientManager.releaseClient(client2, false); } } + + @Test + public void testShortCircuitClient() throws IOException { + OzoneConfiguration conf = new OzoneConfiguration(); + String metaDir = GenericTestUtils.getTempPath( + TestXceiverClientManager.class.getName() + UUID.randomUUID()); + DomainSocketFactory domainSocketFactory = mock(DomainSocketFactory.class); + when(domainSocketFactory.isServiceReady()).thenReturn(true); + conf.set(HDDS_METADATA_DIR_NAME, metaDir); + + ClientTrustManager trustManager = mock(ClientTrustManager.class); + try (XceiverClientManager clientManager = new XceiverClientManager(conf, + conf.getObject(ScmClientConfig.class), trustManager)) { + + ContainerWithPipeline container1 = storageContainerLocationClient + .allocateContainer( + SCMTestUtils.getReplicationType(conf), + HddsProtos.ReplicationFactor.ONE, + 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); + // client is still kept in the cache, for create a domain socket connection is expensive. + assertEquals(1, clientManager.getClientCache().size()); + } + } } 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 c5301ba4194f..a00a5959f3af 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 @@ -65,7 +65,7 @@ void testAll(ContainerLayoutVersion layout) throws Exception { private 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)) { @@ -126,7 +126,7 @@ private void testChunkReadBuffers(TestBucket bucket) throws Exception { private 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 = @@ -149,7 +149,7 @@ private void testCloseReleasesBuffers(TestBucket bucket) throws Exception { */ private 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)) { 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 256148dfb8de..65490a678af1 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 000000000000..bd37ebe78206 --- /dev/null +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestShortCircuitChunkInputStream.java @@ -0,0 +1,63 @@ +/** + * 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.ShortCircuitChunkInputStream; +import org.apache.ozone.test.GenericTestUtils; + +import static org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor.ONE; +import static org.apache.hadoop.hdds.scm.OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT; + +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.event.Level; + +import java.io.File; + +/** + * Tests {@link ShortCircuitChunkInputStream}. + */ +public class TestShortCircuitChunkInputStream extends TestChunkInputStream { + + @TempDir + private File dir; + + @Override + int getDatanodeCount() { + return 1; + } + + @Override + void setCustomizedProperties(OzoneConfiguration configuration) { + configuration.setBoolean(OZONE_READ_SHORT_CIRCUIT, true); + configuration.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, + new File(dir, "ozone-dn-socket").getAbsolutePath()); + GenericTestUtils.setLogLevel(XceiverClientShortCircuit.LOG, Level.DEBUG); + GenericTestUtils.setLogLevel(XceiverClientGrpc.LOG, Level.DEBUG); + } + + @Override + ReplicationConfig getRepConfig() { + return RatisReplicationConfig.getInstance(ONE); + } +} diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java index 33d59f101ebc..800d07688b10 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/freon/TestDNRPCLoadGenerator.java @@ -24,6 +24,8 @@ import org.apache.hadoop.hdds.ratis.conf.RatisClientConfig; import org.apache.hadoop.hdds.scm.XceiverClientCreator; import org.apache.hadoop.hdds.scm.XceiverClientFactory; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientManager; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; import org.apache.hadoop.hdds.scm.protocolPB.StorageContainerLocationProtocolClientSideTranslatorPB; @@ -33,17 +35,25 @@ 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.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; import picocli.CommandLine; +import java.io.File; +import java.io.IOException; import java.time.Duration; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.stream.Stream; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_BLOCK_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_DATANODE_HANDLER_COUNT_KEY; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; +import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HANDLER_COUNT_KEY; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -53,6 +63,8 @@ public class TestDNRPCLoadGenerator { private static MiniOzoneCluster cluster = null; private static ContainerWithPipeline container; + @TempDir + private static File dir; private static void startCluster(OzoneConfiguration conf) throws Exception { DatanodeRatisServerConfig ratisServerConfig = @@ -67,11 +79,19 @@ private static void startCluster(OzoneConfiguration conf) throws Exception { raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(10)); conf.setFromObject(raftClientConfig); + conf.setBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, true); + conf.set(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH, new File(dir, "ozone-dn-socket").getAbsolutePath()); + conf.setInt(OZONE_OM_HANDLER_COUNT_KEY, 5); + conf.setInt(OZONE_SCM_HANDLER_COUNT_KEY, 2); + conf.setInt(OZONE_SCM_CLIENT_HANDLER_COUNT_KEY, 2); + conf.setInt(OZONE_SCM_BLOCK_HANDLER_COUNT_KEY, 2); + conf.setInt(OZONE_SCM_DATANODE_HANDLER_COUNT_KEY, 2); + cluster = MiniOzoneCluster.newBuilder(conf) - .setNumDatanodes(5).build(); + .setNumDatanodes(1).build(); cluster.waitForClusterToBeReady(); - cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.THREE, - 180000); +// cluster.waitForPipelineTobeReady(HddsProtos.ReplicationFactor.THREE, +// 180000); StorageContainerLocationProtocolClientSideTranslatorPB storageContainerLocationClient = cluster @@ -106,29 +126,55 @@ public static void shutdown() { private static Stream provideParameters() { return Stream.of( - Arguments.of(true, true), - Arguments.of(true, false), - Arguments.of(false, true), - Arguments.of(false, false) +// Arguments.of(true, "ratis", "0", "0"), +// Arguments.of(false, "ratis", "0", "0"), +// Arguments.of(true, "grpc", "0", "0"), +// Arguments.of(false, "grpc", "0", "0"), +// Arguments.of(true, "grpc", "1", "1"), +// Arguments.of(true, "grpc", "4", "4"), +// Arguments.of(true, "grpc", "16", "16"), +// Arguments.of(true, "grpc", "64", "64"), +// Arguments.of(true, "grpc", "128", "128"), +// Arguments.of(true, "grpc", "1024", "1024"), +// Arguments.of(false, "short-circuit", "0", "0"), +// Arguments.of(true, "short-circuit", "0", "1"), +// Arguments.of(true, "short-circuit", "1", "0"), +// Arguments.of(true, "short-circuit", "1", "1"), +// Arguments.of(true, "short-circuit", "4", "4"), +// Arguments.of(true, "short-circuit", , "64", "64"), +//// Arguments.of(true, "short-cir"16", "16"), +// Arguments.of(true, "short-circuit"cuit", "128", "128"), + Arguments.of(true, "grpc", "1", "128"), + Arguments.of(true, "short-circuit", "1", "128"), + Arguments.of(true, "grpc", "1", "64"), + Arguments.of(true, "short-circuit", "1", "64"), + Arguments.of(true, "grpc", "1", "32"), + Arguments.of(true, "short-circuit", "1", "32") ); } @ParameterizedTest @MethodSource("provideParameters") - public void test(boolean readOnly, boolean ratis) { + public void test(boolean readOnly, String type, String requestSizeKB, String responseSizeKB) throws Exception { DNRPCLoadGenerator randomKeyGenerator = new DNRPCLoadGenerator(cluster.getConf()); CommandLine cmd = new CommandLine(randomKeyGenerator); List cmdArgs = new ArrayList<>(Arrays.asList( "--container-id", Long.toString(container.getContainerInfo().getContainerID()), - "--clients", "5", - "-t", "10")); + "--channel-type", type, + "--clients", "1", + "--payload-req", requestSizeKB, + "--payload-resp", responseSizeKB, + "-n", "1000000", + "-t", "100")); if (readOnly) { cmdArgs.add("--read-only"); - } - if (ratis) { - cmdArgs.add("--ratis"); + try (XceiverClientFactory factory = new XceiverClientCreator(cluster.getConf()); + XceiverClientSpi client = factory.acquireClient(container.getPipeline())) { + ContainerProtocolCalls.closeContainer(client, + container.getContainerInfo().getContainerID(), null); + } } int exitCode = cmd.execute(cmdArgs.toArray(new String[0])); diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java index a7527952ca35..016f67691ab3 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/DNRPCLoadGenerator.java @@ -20,32 +20,37 @@ import com.codahale.metrics.Timer; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.client.StandaloneReplicationConfig; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; +import org.apache.hadoop.hdds.scm.XceiverClientManager; +import org.apache.hadoop.hdds.scm.XceiverClientShortCircuit; +import org.apache.hadoop.hdds.scm.client.ClientTrustManager; +import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory; +import org.apache.hadoop.hdds.security.x509.certificate.client.CACertificateProvider; +import org.apache.hadoop.ozone.OzoneSecurityUtil; +import org.apache.hadoop.ozone.util.PayloadUtils; +import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; +import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos; -import org.apache.hadoop.hdds.scm.XceiverClientCreator; import org.apache.hadoop.hdds.scm.XceiverClientFactory; import org.apache.hadoop.hdds.scm.XceiverClientSpi; import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; -import org.apache.hadoop.hdds.scm.client.ClientTrustManager; -import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; import org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls; -import org.apache.hadoop.hdds.security.x509.certificate.client.CACertificateProvider; -import org.apache.hadoop.ozone.OzoneSecurityUtil; import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB; -import org.apache.hadoop.ozone.util.PayloadUtils; -import org.apache.ratis.thirdparty.com.google.protobuf.ByteString; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import picocli.CommandLine; import picocli.CommandLine.Command; import picocli.CommandLine.Option; +import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicLong; import static org.apache.hadoop.hdds.client.ReplicationConfig.getLegacyFactor; @@ -71,6 +76,9 @@ public class DNRPCLoadGenerator extends BaseFreonGenerator private int payloadRespSize; private List clients; private String encodedContainerToken; + private final AtomicLong callId = new AtomicLong(0); + private final ByteString clientId = ByteString.copyFrom(this.getClass().getSimpleName().getBytes()); + private boolean shortCircuitRead = false; @Option(names = {"--payload-req"}, description = "Specifies the size of payload in KB in RPC request. ", @@ -102,15 +110,15 @@ public class DNRPCLoadGenerator extends BaseFreonGenerator defaultValue = "false") private boolean readOnly = false; - @Option(names = {"--ratis"}, - description = "if Ratis or grpc", - defaultValue = "false") - private boolean ratis = false; + @Option(names = {"--channel-type"}, + description = "if Ratis, grpc or short-circuit", + defaultValue = "grpc") + private String type = "grpc"; @CommandLine.ParentCommand private Freon freon; - // empy constructor for picocli + // empty constructor for picocli DNRPCLoadGenerator() { } @@ -122,47 +130,58 @@ public class DNRPCLoadGenerator extends BaseFreonGenerator @Override public Void call() throws Exception { Preconditions.checkArgument(payloadReqSizeKB >= 0, - "OM echo request payload size should be positive value or zero."); + "OM echo request payload size should be positive value or zero."); Preconditions.checkArgument(payloadRespSizeKB >= 0, - "OM echo response payload size should be positive value or zero."); + "OM echo response payload size should be positive value or zero."); if (configuration == null) { configuration = freon.createOzoneConfiguration(); } - ContainerOperationClient scmClient = new ContainerOperationClient(configuration); - ContainerInfo containerInfo = scmClient.getContainer(containerID); - List pipelineList = scmClient.listPipelines(); - Pipeline pipeline = pipelineList.stream() - .filter(p -> p.getId().equals(containerInfo.getPipelineID())) - .findFirst() - .orElse(null); - // If GRPC, use STANDALONE pipeline - if (!ratis) { - if (!readOnly) { - LOG.warn("Read only is not set to true for GRPC, setting it to true"); - readOnly = true; + if (type.equalsIgnoreCase("short-circuit")) { + boolean shortCircuit = configuration.getBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, + OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT_DEFAULT); + if (!shortCircuit) { + LOG.error("Short-circuit is not enabled"); + return null; + } + DomainSocketFactory domainSocketFactory = DomainSocketFactory.getInstance(configuration); + if (domainSocketFactory != null && domainSocketFactory.isServiceReady()) { + shortCircuitRead = true; + } else { + LOG.error("Short-circuit read is enabled but service is not ready"); + return null; } - pipeline = Pipeline.newBuilder(pipeline) - .setReplicationConfig(StandaloneReplicationConfig.getInstance( - getLegacyFactor(pipeline.getReplicationConfig()))) - .build(); } + ContainerOperationClient scmClient = new ContainerOperationClient(configuration); encodedContainerToken = scmClient.getEncodedContainerToken(containerID); XceiverClientFactory xceiverClientManager; OzoneManagerProtocolClientSideTranslatorPB omClient; if (OzoneSecurityUtil.isSecurityEnabled(configuration)) { omClient = createOmClient(configuration, null); CACertificateProvider caCerts = () -> omClient.getServiceInfo().provideCACerts(); - xceiverClientManager = new XceiverClientCreator(configuration, + xceiverClientManager = new XceiverClientManager(configuration, + configuration.getObject(XceiverClientManager.ScmClientConfig.class), new ClientTrustManager(caCerts, null)); } else { omClient = null; - xceiverClientManager = new XceiverClientCreator(configuration); + xceiverClientManager = new XceiverClientManager(configuration); } clients = new ArrayList<>(numClients); for (int i = 0; i < numClients; i++) { - clients.add(xceiverClientManager.acquireClient(pipeline)); + Pipeline pipeline = getPipeline(scmClient, containerID); + XceiverClientSpi client = shortCircuitRead ? xceiverClientManager.acquireClientForReadData(pipeline, true) : + xceiverClientManager.acquireClient(pipeline); + if (shortCircuitRead) { + if (!(client instanceof XceiverClientShortCircuit)) { + LOG.error("Short-circuit is enabled while client is of type {}", client.getClass().getSimpleName()); + clients.forEach(c -> xceiverClientManager.releaseClient(c, false)); + xceiverClientManager.close(); + scmClient.close(); + return null; + } + } + clients.add(client); } init(); @@ -176,14 +195,40 @@ public Void call() throws Exception { omClient.close(); } for (XceiverClientSpi client : clients) { - xceiverClientManager.releaseClient(client, false); + xceiverClientManager.releaseClient(client, true); } + clients = null; xceiverClientManager.close(); scmClient.close(); } return null; } + private Pipeline getPipeline(ContainerOperationClient scmClient, long containerId) throws IOException { + ContainerWithPipeline containerInfo = scmClient.getContainerWithPipeline(containerId); + + List pipelineList = scmClient.listPipelines(); + Pipeline pipeline = pipelineList.stream() + .filter(p -> p.getId().equals(containerInfo.getPipeline().getId())) + .findFirst() + .orElse(null); + // If GRPC or Short-circuit, use STANDALONE pipeline + if (!type.equalsIgnoreCase("ratis")) { + if (!readOnly) { + LOG.warn("Read only is not set to true for grpc/short-circuit, setting it to true"); + readOnly = true; + } + if (pipeline == null) { + pipeline = containerInfo.getPipeline(); + } + pipeline = Pipeline.newBuilder(pipeline) + .setReplicationConfig(StandaloneReplicationConfig.getInstance( + getLegacyFactor(pipeline.getReplicationConfig()))) + .build(); + } + return pipeline; + } + private int calculateMaxPayloadSize(int payloadSizeKB) { if (payloadSizeKB > 0) { return Math.min( @@ -201,7 +246,10 @@ private int payloadSizeInBytes(int payloadSizeKB) { private void sendRPCReq(long l) throws Exception { timer.time(() -> { int clientIndex = (numClients == 1) ? 0 : (int)l % numClients; - ContainerProtos.EchoResponseProto response = + ContainerProtos.EchoResponseProto response = shortCircuitRead ? + ContainerProtocolCalls.echo(clients.get(clientIndex), encodedContainerToken, + containerID, payloadReqBytes, payloadRespSize, sleepTimeMs, readOnly, + clientId, callId.incrementAndGet(), false) : ContainerProtocolCalls.echo(clients.get(clientIndex), encodedContainerToken, containerID, payloadReqBytes, payloadRespSize, sleepTimeMs, readOnly); return null; diff --git a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java index 58b62d22b985..cc637fcae9c8 100644 --- a/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java +++ b/hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/freon/RandomKeyGenerator.java @@ -44,6 +44,7 @@ import org.apache.hadoop.hdds.client.ReplicationConfig; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageSize; +import org.apache.hadoop.hdds.scm.OzoneClientConfig; import org.apache.hadoop.hdds.tracing.TracingUtil; import org.apache.hadoop.ozone.OzoneConsts; import org.apache.hadoop.ozone.client.ObjectStore; @@ -166,6 +167,11 @@ enum FreonOps { defaultValue = "1") private int numOfValidateThreads = 1; + @Option(names = {"--validate-channel"}, + description = "grpc or short-circuit.", + defaultValue = "grpc") + private String validateChannel = "grpc"; + @Option( names = {"--buffer-size", "--bufferSize"}, description = "Specifies the buffer size while writing. Full name " + @@ -294,6 +300,17 @@ public Void call() throws Exception { + HddsConfigKeys.HDDS_CONTAINER_PERSISTDATA + " is set to false."); validateWrites = false; } + if (validateChannel.equalsIgnoreCase("grpc")) { + ozoneConfiguration.setBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, false); + } else if (validateChannel.equalsIgnoreCase("short-circuit")){ + boolean shortCircuit = ozoneConfiguration.getBoolean(OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT, + OzoneClientConfig.OZONE_READ_SHORT_CIRCUIT_DEFAULT); + if (!shortCircuit) { + LOG.error("Short-circuit read is not enabled"); + return null; + } + } + init(ozoneConfiguration); replicationConfig = replication.fromParamsOrConfig(ozoneConfiguration); @@ -366,11 +383,13 @@ public Void call() throws Exception { } else { progressbar.shutdown(); } + LOG.info("Data generation is completed"); if (validateExecutor != null) { while (!validationQueue.isEmpty()) { Thread.sleep(CHECK_INTERVAL_MILLIS); } + LOG.info("Data validation is completed"); validateExecutor.shutdown(); validateExecutor.awaitTermination(Integer.MAX_VALUE, TimeUnit.MILLISECONDS);