Skip to content

Commit 2b2a796

Browse files
authored
HDDS-12969. Use DatanodeID in XceiverClientGrpc (#8430)
1 parent 3c0bafa commit 2b2a796

File tree

1 file changed

+9
-9
lines changed

1 file changed

+9
-9
lines changed

hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java

Lines changed: 9 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,6 @@
3333
import java.util.List;
3434
import java.util.Map;
3535
import java.util.Objects;
36-
import java.util.UUID;
3736
import java.util.concurrent.CompletableFuture;
3837
import java.util.concurrent.ConcurrentHashMap;
3938
import java.util.concurrent.ExecutionException;
@@ -42,6 +41,7 @@
4241
import org.apache.hadoop.hdds.HddsUtils;
4342
import org.apache.hadoop.hdds.conf.ConfigurationSource;
4443
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
44+
import org.apache.hadoop.hdds.protocol.DatanodeID;
4545
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
4646
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandRequestProto;
4747
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ContainerCommandResponseProto;
@@ -88,9 +88,9 @@ public class XceiverClientGrpc extends XceiverClientSpi {
8888
LoggerFactory.getLogger(XceiverClientGrpc.class);
8989
private final Pipeline pipeline;
9090
private final ConfigurationSource config;
91-
private final Map<UUID, XceiverClientProtocolServiceStub> asyncStubs;
91+
private final Map<DatanodeID, XceiverClientProtocolServiceStub> asyncStubs;
9292
private final XceiverClientMetrics metrics;
93-
private final Map<UUID, ManagedChannel> channels;
93+
private final Map<DatanodeID, ManagedChannel> channels;
9494
private final Semaphore semaphore;
9595
private long timeout;
9696
private final SecurityConfig secConfig;
@@ -178,8 +178,8 @@ private synchronized void connectToDatanode(DatanodeDetails dn)
178178
ManagedChannel channel = createChannel(dn, port).build();
179179
XceiverClientProtocolServiceStub asyncStub =
180180
XceiverClientProtocolServiceGrpc.newStub(channel);
181-
asyncStubs.put(dn.getUuid(), asyncStub);
182-
channels.put(dn.getUuid(), channel);
181+
asyncStubs.put(dn.getID(), asyncStub);
182+
channels.put(dn.getID(), channel);
183183
}
184184

185185
protected NettyChannelBuilder createChannel(DatanodeDetails dn, int port)
@@ -213,7 +213,7 @@ protected NettyChannelBuilder createChannel(DatanodeDetails dn, int port)
213213
*/
214214
@VisibleForTesting
215215
public boolean isConnected(DatanodeDetails details) {
216-
return isConnected(channels.get(details.getUuid()));
216+
return isConnected(channels.get(details.getID()));
217217
}
218218

219219
private boolean isConnected(ManagedChannel channel) {
@@ -567,7 +567,7 @@ public XceiverClientReply sendCommandAsync(
567567
ContainerCommandRequestProto request, DatanodeDetails dn)
568568
throws IOException, InterruptedException {
569569
checkOpen(dn);
570-
UUID dnId = dn.getUuid();
570+
DatanodeID dnId = dn.getID();
571571
if (LOG.isDebugEnabled()) {
572572
LOG.debug("Send command {} to datanode {}",
573573
request.getCmdType(), dn.getIpAddress());
@@ -625,7 +625,7 @@ private synchronized void checkOpen(DatanodeDetails dn)
625625
throw new IOException("This channel is not connected.");
626626
}
627627

628-
ManagedChannel channel = channels.get(dn.getUuid());
628+
ManagedChannel channel = channels.get(dn.getID());
629629
// If the channel doesn't exist for this specific datanode or the channel
630630
// is closed, just reconnect
631631
if (!isConnected(channel)) {
@@ -639,7 +639,7 @@ private void reconnect(DatanodeDetails dn)
639639
ManagedChannel channel;
640640
try {
641641
connectToDatanode(dn);
642-
channel = channels.get(dn.getUuid());
642+
channel = channels.get(dn.getID());
643643
} catch (Exception e) {
644644
throw new IOException("Error while connecting", e);
645645
}

0 commit comments

Comments
 (0)