ashishkumar50 commented on code in PR #7456:
URL: https://github.com/apache/ozone/pull/7456#discussion_r1853266770
##########
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 =
Review Comment:
```suggestion
private static final Logger LOG =
```
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java:
##########
@@ -263,7 +342,7 @@ public static class ScmClientConfig {
@Config(key = "idle.threshold",
type = ConfigType.TIME, timeUnit = MILLISECONDS,
- defaultValue = "10s",
+ defaultValue = "300s",
Review Comment:
Why we are changing default from 10s to 300s?
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java:
##########
@@ -197,12 +270,16 @@ private String getPipelineCacheKey(Pipeline pipeline,
// Append user short name to key to prevent a different user
// from using same instance of xceiverClient.
try {
- key += UserGroupInformation.getCurrentUser().getShortUserName();
+ key = UserGroupInformation.getCurrentUser().getShortUserName() + "@" +
key;
} catch (IOException e) {
LOG.error("Failed to get current user to create pipeline cache key:" +
e.getMessage());
}
}
+ LOG.info("cache key {} for pipeline {}", key, pipeline);
+ if (localDN != null) {
+ localDNCache.put(key, localDN);
Review Comment:
Move above log inside if condition.
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java:
##########
@@ -0,0 +1,629 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.LimitInputStream;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.io.grpc.Status;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.ClosedChannelException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.HddsUtils.processForDebug;
+import static
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE;
+import static
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION;
+
+/**
+ * {@link XceiverClientSpi} implementation, the client to read local replica
through short circuit.
+ */
+public class XceiverClientShortCircuit extends XceiverClientSpi {
+ public static final Logger LOG =
+ LoggerFactory.getLogger(XceiverClientShortCircuit.class);
+ private final Pipeline pipeline;
+ private final ConfigurationSource config;
+ private final XceiverClientMetrics metrics;
+ private int readTimeoutMs;
+ private int writeTimeoutMs;
+ // Cache the stream of blocks
+ private final Map<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 String prefix;
+
+ /**
+ * Constructs a client that can communicate with the Container framework on
local datanode through DomainSocket.
+ */
+ public XceiverClientShortCircuit(Pipeline pipeline, ConfigurationSource
config, DatanodeDetails dn) {
+ super();
+ Preconditions.checkNotNull(config);
+ this.readTimeoutMs = (int)
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT,
+ OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
+ this.writeTimeoutMs = (int)
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT,
+ OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
+
+ this.pipeline = pipeline;
+ this.dn = dn;
+ this.domainSocketFactory = DomainSocketFactory.getInstance(config);
+ this.config = config;
+ this.metrics = XceiverClientManager.getXceiverClientMetrics();
+ this.blockStreamCache = new ConcurrentHashMap<>();
+ this.sentRequests = new ConcurrentHashMap<>();
+ int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+ this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port);
+ this.bufferSize =
config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize();
+ this.readDaemon = new Daemon(new ReceiveResponseTask());
+ LOG.info("{} is created for pipeline {}",
XceiverClientShortCircuit.class.getSimpleName(), pipeline);
+ }
+
+ /**
+ * Create the DomainSocket to connect to the local DataNode.
+ */
+ @Override
+ public void connect() throws IOException {
+ // Even the in & out stream has returned EOFException,
domainSocket.isOpen() is still true.
+ if (domainSocket != null && domainSocket.isOpen() &&
isDomainSocketOpen.get()) {
+ return;
+ }
+ domainSocket = domainSocketFactory.createSocket(readTimeoutMs,
writeTimeoutMs, dnAddr);
+ isDomainSocketOpen.set(true);
+ prefix = XceiverClientShortCircuit.class.getSimpleName() + "-" +
domainSocket.toString();
+ timer = new Timer(prefix + "-Timer");
+ readDaemon.start();
+ LOG.info("{} is started", prefix);
+ }
+
+ /**
+ * Close the DomainSocket.
+ */
+ @Override
+ public synchronized void close() {
+ closed = true;
+ timer.cancel();
+ if (domainSocket != null) {
+ try {
+ isDomainSocketOpen.set(false);
+ domainSocket.close();
+ LOG.info("{} is closed for {}", domainSocket.toString(), dn);
+ } catch (IOException e) {
+ LOG.warn("Failed to close domain socket for datanode {}", dn, e);
+ }
+ }
+ readDaemon.interrupt();
+ try {
+ readDaemon.join();
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ }
+ }
+
+ public boolean isClosed() {
+ return closed;
+ }
+
+ @Override
+ public Pipeline getPipeline() {
+ return pipeline;
+ }
+
+ public DatanodeDetails getDn() {
+ return this.dn;
+ }
+
+ public ByteString getClientId() {
+ return clientId;
+ }
+
+ public long getCallId() {
+ return callId.incrementAndGet();
+ }
+
+ @Override
+ public ContainerCommandResponseProto
sendCommand(ContainerCommandRequestProto request) throws IOException {
+ try {
+ return sendCommandWithTraceID(request, null).getResponse().get();
+ } catch (ExecutionException e) {
+ throw getIOExceptionForSendCommand(request, e);
+ } catch (InterruptedException e) {
+ LOG.error("Command execution was interrupted.");
+ Thread.currentThread().interrupt();
+ throw (IOException) new InterruptedIOException(
+ "Command " + processForDebug(request) + " was interrupted.")
+ .initCause(e);
+ }
+ }
+
+ @Override
+ public Map<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 = "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;
Review Comment:
As per above comment `// In case of an exception or an error, we will try to
read from the
// datanodes in the pipeline in a round-robin fashion.` So from
other datanode it will read without short-circuit, is it handled in caller?
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientManager.java:
##########
@@ -150,24 +197,50 @@ 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) {
+ DatanodeDetails localDN = null;
+
+ if ((!isEC) && allowShortCircuit && isShortCircuitEnabled()) {
+ int port = 0;
+ InetSocketAddress localAddr = null;
+ for (DatanodeDetails dn : pipeline.getNodes()) {
+ // read port from the data node, on failure use default configured
port.
+ port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+ InetSocketAddress addr = NetUtils.createSocketAddr(dn.getIpAddress(),
port);
+ if (OzoneNetUtils.isAddressLocal(addr) &&
+ dn.getCurrentVersion() >= SHORT_CIRCUIT_READS.toProtoValue()) {
+ localAddr = addr;
+ localDN = dn;
+ break;
+ }
+ }
+ if (localAddr != null) {
+ // Find a local DN and short circuit read is enabled
+ key += "@" + localAddr.getHostName() + ":" + port + "/" +
DomainSocketFactory.FEATURE_FLAG;
+ }
+ } else if (topologyAware || isEC) {
Review Comment:
In previous condition if `localDN` is not found, i think we should use
`topologyAware` in that case and use the closest node.
##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientShortCircuit.java:
##########
@@ -0,0 +1,629 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdds.conf.ConfigurationSource;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
+import
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.storage.DomainSocketFactory;
+import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
+import org.apache.hadoop.hdds.tracing.TracingUtil;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.net.unix.DomainSocket;
+import org.apache.hadoop.ozone.OzoneConfigKeys;
+import org.apache.hadoop.thirdparty.protobuf.CodedInputStream;
+import org.apache.hadoop.util.Daemon;
+import org.apache.hadoop.util.LimitInputStream;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.apache.ratis.thirdparty.io.grpc.Status;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InterruptedIOException;
+import java.net.InetSocketAddress;
+import java.nio.channels.ClosedChannelException;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.UUID;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.hadoop.hdds.HddsUtils.processForDebug;
+import static
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_MAGIC_CODE;
+import static
org.apache.hadoop.hdds.scm.OzoneClientConfig.DATA_TRANSFER_VERSION;
+
+/**
+ * {@link XceiverClientSpi} implementation, the client to read local replica
through short circuit.
+ */
+public class XceiverClientShortCircuit extends XceiverClientSpi {
+ public static final Logger LOG =
+ LoggerFactory.getLogger(XceiverClientShortCircuit.class);
+ private final Pipeline pipeline;
+ private final ConfigurationSource config;
+ private final XceiverClientMetrics metrics;
+ private int readTimeoutMs;
+ private int writeTimeoutMs;
+ // Cache the stream of blocks
+ private final Map<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 String prefix;
+
+ /**
+ * Constructs a client that can communicate with the Container framework on
local datanode through DomainSocket.
+ */
+ public XceiverClientShortCircuit(Pipeline pipeline, ConfigurationSource
config, DatanodeDetails dn) {
+ super();
+ Preconditions.checkNotNull(config);
+ this.readTimeoutMs = (int)
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT,
+ OzoneConfigKeys.OZONE_CLIENT_READ_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
+ this.writeTimeoutMs = (int)
config.getTimeDuration(OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT,
+ OzoneConfigKeys.OZONE_CLIENT_WRITE_TIMEOUT_DEFAULT,
TimeUnit.MILLISECONDS);
+
+ this.pipeline = pipeline;
+ this.dn = dn;
+ this.domainSocketFactory = DomainSocketFactory.getInstance(config);
+ this.config = config;
+ this.metrics = XceiverClientManager.getXceiverClientMetrics();
+ this.blockStreamCache = new ConcurrentHashMap<>();
+ this.sentRequests = new ConcurrentHashMap<>();
+ int port = dn.getPort(DatanodeDetails.Port.Name.STANDALONE).getValue();
+ this.dnAddr = NetUtils.createSocketAddr(dn.getIpAddress(), port);
+ this.bufferSize =
config.getObject(OzoneClientConfig.class).getShortCircuitBufferSize();
+ this.readDaemon = new Daemon(new ReceiveResponseTask());
+ LOG.info("{} is created for pipeline {}",
XceiverClientShortCircuit.class.getSimpleName(), pipeline);
+ }
+
+ /**
+ * Create the DomainSocket to connect to the local DataNode.
+ */
+ @Override
+ public void connect() throws IOException {
+ // Even the in & out stream has returned EOFException,
domainSocket.isOpen() is still true.
+ if (domainSocket != null && domainSocket.isOpen() &&
isDomainSocketOpen.get()) {
+ return;
+ }
+ domainSocket = domainSocketFactory.createSocket(readTimeoutMs,
writeTimeoutMs, dnAddr);
+ isDomainSocketOpen.set(true);
+ prefix = XceiverClientShortCircuit.class.getSimpleName() + "-" +
domainSocket.toString();
+ timer = new Timer(prefix + "-Timer");
+ readDaemon.start();
+ LOG.info("{} is started", prefix);
+ }
+
+ /**
+ * Close the DomainSocket.
+ */
+ @Override
+ public synchronized void close() {
+ closed = true;
+ timer.cancel();
+ if (domainSocket != null) {
+ try {
+ isDomainSocketOpen.set(false);
+ domainSocket.close();
+ LOG.info("{} is closed for {}", domainSocket.toString(), dn);
+ } catch (IOException e) {
+ LOG.warn("Failed to close domain socket for datanode {}", dn, e);
Review Comment:
Should we retry here in case of failure?
--
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]