jojochuang commented on code in PR #7456:
URL: https://github.com/apache/ozone/pull/7456#discussion_r1913958682


##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java:
##########
@@ -117,17 +124,54 @@ public Cache<String, XceiverClientSpi> 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

Review Comment:
   ```suggestion
      * @param pipeline the container pipeline for the client connection
      * @param allowShortCircuit create a short-circuit read based client if 
applicable.
   ```



##########
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/XceiverServerDomainSocket.java:
##########
@@ -0,0 +1,338 @@
+/*
+ * 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.FaultInjector;
+import org.apache.hadoop.hdds.utils.HddsServerUtil;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
+import org.apache.hadoop.ozone.container.common.interfaces.ContainerDispatcher;
+import 
org.apache.hadoop.ozone.container.common.statemachine.DatanodeConfiguration;
+import org.apache.hadoop.util.Daemon;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.SocketTimeoutException;
+import java.nio.channels.AsynchronousCloseException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.concurrent.ThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Creates a DomainSocket server endpoint that acts as the communication layer 
for Ozone containers.
+ */
+public final class XceiverServerDomainSocket implements XceiverServerSpi, 
Runnable {
+  public static final Logger LOG = 
LoggerFactory.getLogger(XceiverServerDomainSocket.class);
+  private int port;
+  private Daemon server;
+  private ContainerDispatcher dispatcher;
+  private ContainerMetrics metrics;
+  private final AtomicBoolean isRunning = new AtomicBoolean(false);
+
+  /**
+   * Maximal number of concurrent readers per node.
+   * Enforcing the limit is required in order to avoid data-node
+   * running out of memory.
+   */
+  private final int maxXceiverCount;
+  private final AtomicInteger xceriverCount;
+  private DomainSocket domainSocket;
+  private final ConfigurationSource config;
+  private final String threadPrefix;
+  private final HashMap<DomainPeer, Thread> peers = new HashMap<>();
+  private final HashMap<DomainPeer, Receiver> peersReceiver = new HashMap<>();
+  private int readTimeoutMs;
+  private int writeTimeoutMs;
+  private final ThreadPoolExecutor readExecutors;
+  private FaultInjector injector;
+
+  /**
+   * Constructs a DomainSocket server class, used to listen for requests from 
local clients.
+   */
+  public XceiverServerDomainSocket(DatanodeDetails datanodeDetails, 
ConfigurationSource conf,
+      ContainerDispatcher dispatcher, ThreadPoolExecutor executor,
+      ContainerMetrics metrics, DomainSocketFactory domainSocketFactory) {
+    Preconditions.checkNotNull(conf);
+    this.port = conf.getInt(OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT,
+        OzoneConfigKeys.HDDS_CONTAINER_IPC_PORT_DEFAULT);
+    if (conf.getBoolean(OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT,
+        OzoneConfigKeys.HDDS_CONTAINER_IPC_RANDOM_PORT_DEFAULT)) {
+      this.port = 0;
+    }
+    this.config = conf;
+    final int threadCountPerDisk =
+        
conf.getObject(DatanodeConfiguration.class).getNumReadThreadPerVolume();
+    final int numberOfDisks = 
HddsServerUtil.getDatanodeStorageDirs(conf).size();
+    this.maxXceiverCount = threadCountPerDisk * numberOfDisks * 5;
+    this.xceriverCount = new AtomicInteger(0);
+    this.dispatcher = dispatcher;
+    this.readExecutors = executor;
+    this.metrics = metrics;
+    LOG.info("Max allowed {} xceiver", maxXceiverCount);
+    this.threadPrefix = datanodeDetails.threadNamePrefix() + 
XceiverServerDomainSocket.class.getSimpleName();
+
+    if (domainSocketFactory.isServiceEnabled() && 
domainSocketFactory.isServiceReady()) {
+      this.readTimeoutMs = (int) 
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT,
+          OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, 
TimeUnit.MILLISECONDS);
+      this.writeTimeoutMs = (int) 
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT,
+          OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, 
TimeUnit.MILLISECONDS);
+      try {
+        domainSocket = DomainSocket.bindAndListen(
+            
DomainSocket.getEffectivePath(conf.get(OzoneClientConfig.OZONE_DOMAIN_SOCKET_PATH),
 port));
+        OzoneClientConfig ozoneClientConfig = 
conf.getObject(OzoneClientConfig.class);
+        domainSocket.setAttribute(DomainSocket.RECEIVE_TIMEOUT, readTimeoutMs);
+        domainSocket.setAttribute(DomainSocket.SEND_TIMEOUT, writeTimeoutMs);
+        LOG.info("UNIX domain socket {} is created: {}, timeout for read {} 
ms, timeout for write {} ms, " +
+                "send/receive buffer {} bytes", domainSocket, 
domainSocket.getPath(), readTimeoutMs, writeTimeoutMs,
+            ozoneClientConfig.getShortCircuitBufferSize());
+      } catch (IOException e) {
+        LOG.warn("Although short-circuit local reads are configured, we cannot 
" +
+            "enable the short circuit read because DomainSocket operation 
failed", e);
+        domainSocket = null;
+        throw new IllegalArgumentException(e);
+      }
+    }
+  }
+
+  @Override
+  public int getIPCPort() {
+    return this.port;
+  }
+
+  /**
+   * Returns the Replication type supported by this end-point.
+   *
+   * @return enum -- {Stand_Alone, Ratis, Grpc, Chained}

Review Comment:
   ```suggestion
      * @return enum STAND_ALONE
   ```



##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientCreator.java:
##########
@@ -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

Review Comment:
   ```suggestion
       // clients are not tracked, closing each client is the responsibility of 
users of this class
   ```



##########
hadoop-hdds/common/src/main/java/org/apache/hadoop/ozone/OzoneConfigKeys.java:
##########
@@ -471,8 +471,12 @@ public final class OzoneConfigKeys {
   public static final int OZONE_CLIENT_BYTES_PER_CHECKSUM_MIN_SIZE = 8 * 1024;
 
   public static final String OZONE_CLIENT_READ_TIMEOUT
-          = "ozone.client.read.timeout";
+      = "ozone.client.read.timeout";
   public static final String OZONE_CLIENT_READ_TIMEOUT_DEFAULT = "30s";
+  public static final String OZONE_CLIENT_WRITE_TIMEOUT
+      = "ozone.client.write.timeout";

Review Comment:
   Since it's specifically used by short circuit read, how about make it 
"ozone.client.domainsocket.write.timeout"
   
   Also it would be better move to OzoneClientConfig



##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java:
##########
@@ -0,0 +1,626 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
+import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.LimitInputStream;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.com.google.protobuf.CodedInputStream;
+import org.apache.ratis.thirdparty.io.grpc.Status;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.net.InetSocketAddress;
+import java.net.SocketTimeoutException;
+import java.nio.channels.ClosedChannelException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.HddsUtils.processForDebug;
+import static 
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE;
+import static 
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION;
+
+/**
+ * {@link XceiverClientSpi} implementation, the client to read local replica 
through short circuit.
+ */
+public class XceiverClientShortCircuit extends XceiverClientSpi {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(XceiverClientShortCircuit.class);
+  private final Pipeline pipeline;
+  private final ConfigurationSource config;
+  private final XceiverClientMetrics metrics;
+  private int readTimeoutMs;
+  private int writeTimeoutMs;
+  // Cache the stream of blocks
+  private final Map<String, FileInputStream> blockStreamCache;
+  private final Map<String, RequestEntry> sentRequests;
+  private final Daemon readDaemon;
+  private Timer timer;
+
+  private boolean closed = false;
+  private final DatanodeDetails dn;
+  private final InetSocketAddress dnAddr;
+  private final DomainSocketFactory domainSocketFactory;
+  private DomainSocket domainSocket;
+  private AtomicBoolean isDomainSocketOpen = new AtomicBoolean(false);
+  private Lock lock = new ReentrantLock();
+  private final int bufferSize;
+  private final ByteString clientId = 
ByteString.copyFrom(UUID.randomUUID().toString().getBytes(UTF_8));
+  private final AtomicLong callId = new AtomicLong(0);
+  private long requestSent = 0;
+  private long responseReceived = 0;
+  private String prefix;
+
+  /**
+   * Constructs a client that can communicate with the Container framework on 
local datanode through DomainSocket.
+   */
+  public XceiverClientShortCircuit(Pipeline pipeline, ConfigurationSource 
config, DatanodeDetails dn) {
+    super();
+    Preconditions.checkNotNull(config);
+    this.readTimeoutMs = (int) 
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT,
+        OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT, 
TimeUnit.MILLISECONDS);
+    this.writeTimeoutMs = (int) 
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT,
+        OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT, 
TimeUnit.MILLISECONDS);
+
+    this.pipeline = pipeline;
+    this.dn = dn;
+    this.domainSocketFactory = DomainSocketFactory.getInstance(config);
+    this.config = config;
+    this.metrics = XceiverClientManager.getXceiverClientMetrics();
+    this.blockStreamCache = new ConcurrentHashMap<>();
+    this.sentRequests = new ConcurrentHashMap<>();
+    int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+    this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port);
+    this.bufferSize = 
config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize();
+    this.readDaemon = new Daemon(new ReceiveResponseTask());
+    LOG.info("{} is created for pipeline {}", 
XceiverClientShortCircuit.class.getSimpleName(), pipeline);
+  }
+
+  /**
+   * Create the DomainSocket to connect to the local DataNode.
+   */
+  @Override
+  public void connect() throws IOException {
+    // Even the in & out stream has returned EOFException, 
domainSocket.isOpen() is still true.
+    if (domainSocket != null && domainSocket.isOpen() && 
isDomainSocketOpen.get()) {
+      return;
+    }
+    domainSocket = domainSocketFactory.createSocket(readTimeoutMs, 
writeTimeoutMs, dnAddr);
+    isDomainSocketOpen.set(true);
+    prefix = XceiverClientShortCircuit.class.getSimpleName() + "-" + 
domainSocket.toString();
+    timer = new Timer(prefix + "-Timer");
+    readDaemon.start();
+    LOG.info("{} is started", prefix);
+  }
+
+  /**
+   * Close the DomainSocket.
+   */
+  @Override
+  public synchronized void close() {
+    closed = true;
+    timer.cancel();
+    if (domainSocket != null) {
+      try {
+        isDomainSocketOpen.set(false);
+        domainSocket.close();
+        LOG.info("{} is closed for {} with {} requests sent and {} responses 
received",
+            domainSocket.toString(), dn, requestSent, responseReceived);
+      } catch (IOException e) {
+        LOG.warn("Failed to close domain socket for datanode {}", dn, e);
+      }
+    }
+    readDaemon.interrupt();
+    try {
+      readDaemon.join();
+    } catch (InterruptedException e) {
+      Thread.currentThread().interrupt();
+    }
+  }
+
+  public boolean isClosed() {
+    return closed;
+  }
+
+  @Override
+  public Pipeline getPipeline() {
+    return pipeline;
+  }
+
+  public DatanodeDetails getDn() {
+    return this.dn;
+  }
+
+  public ByteString getClientId() {
+    return clientId;
+  }
+
+  public long getCallId() {
+    return callId.incrementAndGet();
+  }
+
+  @Override
+  public ContainerCommandResponseProto 
sendCommand(ContainerCommandRequestProto request) throws IOException {
+    try {
+      return sendCommandWithTraceID(request, null).getResponse().get();
+    } catch (ExecutionException e) {
+      throw getIOExceptionForSendCommand(request, e);
+    } catch (InterruptedException e) {
+      LOG.error("Command execution was interrupted.");
+      Thread.currentThread().interrupt();
+      throw (IOException) new InterruptedIOException(
+          "Command " + processForDebug(request) + " was interrupted.")
+          .initCause(e);
+    }
+  }
+
+  @Override
+  public Map<DatanodeDetails, ContainerCommandResponseProto>
+      sendCommandOnAllNodes(
+      ContainerCommandRequestProto request) throws IOException {
+    throw new UnsupportedOperationException("Operation Not supported for " +
+        DomainSocketFactory.FEATURE + " client");
+  }
+
+  @Override
+  public ContainerCommandResponseProto sendCommand(
+      ContainerCommandRequestProto request, List<Validator> 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<Validator> validators)
+      throws IOException {
+    String spanName = "XceiverClientShortCircuit." + 
request.getCmdType().name();
+    return TracingUtil.executeInNewSpan(spanName,
+        () -> {
+          ContainerCommandRequestProto finalPayload =
+              ContainerCommandRequestProto.newBuilder(request)
+                  .setTraceID(TracingUtil.exportCurrentSpan()).build();
+          ContainerCommandResponseProto responseProto = null;
+          IOException ioException = null;
+          XceiverClientReply reply = new XceiverClientReply(null);
+
+          if (request.getCmdType() != ContainerProtos.Type.GetBlock &&
+              request.getCmdType() != ContainerProtos.Type.Echo) {
+            throw new UnsupportedOperationException("Command " + 
request.getCmdType() +
+                " is not supported for " + DomainSocketFactory.FEATURE + " 
client");
+          }
+
+          try {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Executing command {} on datanode {}", request, dn);
+            }
+            reply.addDatanode(dn);
+            responseProto = 
sendCommandInternal(finalPayload).getResponse().get();
+            if (validators != null && !validators.isEmpty()) {
+              for (Validator validator : validators) {
+                validator.accept(request, responseProto);
+              }
+            }
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("request {} {} {} finished", request.getCmdType(),
+                  request.getClientId().toStringUtf8(), request.getCallId());
+            }
+          } catch (IOException e) {
+            ioException = e;
+            responseProto = null;
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Failed to execute command {} on datanode {}", 
request, dn, e);
+            }
+          } catch (ExecutionException e) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Failed to execute command {} on datanode {}", 
request, dn, e);
+            }
+            if (Status.fromThrowable(e.getCause()).getCode()
+                == Status.UNAUTHENTICATED.getCode()) {
+              throw new SCMSecurityException("Failed to authenticate with "
+                  + "datanode DomainSocket XceiverServer with Ozone block 
token.");
+            }
+            ioException = new IOException(e);
+          } catch (InterruptedException e) {
+            LOG.error("Command execution was interrupted ", e);
+            Thread.currentThread().interrupt();
+          }
+
+          if (responseProto != null) {
+            
reply.setResponse(CompletableFuture.completedFuture(responseProto));
+            return reply;
+          } else {
+            Objects.requireNonNull(ioException);
+            String message = "Failed to execute command {}";
+            if (LOG.isDebugEnabled()) {
+              LOG.debug(message + " on the datanode {} {}.", request, dn, 
domainSocket, ioException);
+            } else {
+              LOG.error(message + " on the datanode {} {}.", request, dn, 
domainSocket, ioException);
+            }
+            throw ioException;
+          }
+        });
+  }
+
+  @VisibleForTesting
+  public XceiverClientReply sendCommandInternal(ContainerCommandRequestProto 
request)
+      throws IOException, InterruptedException {
+    checkOpen();
+    final CompletableFuture<ContainerCommandResponseProto> replyFuture =
+        new CompletableFuture<>();
+    RequestEntry entry = new RequestEntry(request, replyFuture);
+    sendRequest(entry);
+    return new XceiverClientReply(replyFuture);
+  }
+
+  @Override
+  public XceiverClientReply sendCommandAsync(
+      ContainerCommandRequestProto request)
+      throws IOException, ExecutionException, InterruptedException {
+    throw new UnsupportedOperationException("Operation Not supported for " + 
DomainSocketFactory.FEATURE + " client");
+  }
+
+  public synchronized void checkOpen() throws IOException {
+    if (closed) {
+      throw new IOException("DomainSocket is not connected.");
+    }
+
+    if (!isDomainSocketOpen.get()) {
+      throw new IOException(domainSocket.toString() + " is not open.");
+    }
+  }
+
+  @Override
+  public CompletableFuture<XceiverClientReply> watchForCommit(long index) {
+    // there is no notion of watch for commit index in short-circuit local 
reads
+    return null;
+  }
+
+  @Override
+  public long getReplicatedMinCommitIndex() {
+    return 0;
+  }
+
+  public FileInputStream getFileInputStream(long id, DatanodeBlockID blockID) {
+    return blockStreamCache.remove(getFileInputStreamMapKey(id, blockID));
+  }
+
+  private String getFileInputStreamMapKey(long id, DatanodeBlockID blockID) {
+    return id + "-" + blockID.getLocalID();
+  }
+
+  @Override
+  public HddsProtos.ReplicationType getPipelineType() {
+    return HddsProtos.ReplicationType.STAND_ALONE;
+  }
+
+  public ConfigurationSource getConfig() {
+    return config;
+  }
+
+  @VisibleForTesting
+  public static Logger getLogger() {
+    return LOG;
+  }
+
+  public void setReadTimeout(int timeout) {
+    this.readTimeoutMs = timeout;
+  }
+
+  public int getReadTimeout() {
+    return this.readTimeoutMs;
+  }
+
+  String getRequestUniqueID(ContainerCommandRequestProto request) {
+    return request.getClientId().toStringUtf8() + request.getCallId();
+  }
+
+  String getRequestUniqueID(ContainerCommandResponseProto response) {
+    return response.getClientId().toStringUtf8() + response.getCallId();
+  }
+
+  void requestTimeout(String requestId) {
+    final RequestEntry entry = sentRequests.remove(requestId);
+    if (entry != null) {
+      LOG.warn("Timeout to receive response for command {}", 
entry.getRequest());
+      ContainerProtos.Type type = entry.getRequest().getCmdType();
+      metrics.decrPendingContainerOpsMetrics(type);
+      entry.getFuture().completeExceptionally(new TimeoutException("Timeout to 
receive response"));
+    }
+  }
+
+  public void sendRequest(RequestEntry entry) {
+    ContainerCommandRequestProto request = entry.getRequest();
+    try {
+      String key = getRequestUniqueID(request);
+      TimerTask task = new TimerTask() {
+        @Override
+        public void run() {
+          requestTimeout(key);
+        }
+      };
+      entry.setTimerTask(task);
+      timer.schedule(task, readTimeoutMs);
+      sentRequests.put(key, entry);
+      ContainerProtos.Type type = request.getCmdType();
+      metrics.incrPendingContainerOpsMetrics(type);
+      byte[] bytes = request.toByteArray();
+      if (bytes.length != request.getSerializedSize()) {
+        throw new IOException("Serialized request " + request.getCmdType()
+            + " size mismatch, byte array size " + bytes.length +
+            ", serialized size " + request.getSerializedSize());
+      }
+
+      lock.lock();
+      try {
+        DataOutputStream dataOut =
+            new DataOutputStream(new 
BufferedOutputStream(domainSocket.getOutputStream(), bufferSize));

Review Comment:
   Suggest to have some stress test that creates/close many read input streams 
exercise the code path and to make sure there's no buffer heap leak, 
outputstream leak, socket leak... etc.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to