|
@@ -17,13 +17,12 @@
|
|
|
*/
|
|
|
package org.apache.hadoop.hdfs;
|
|
|
|
|
|
-import java.io.FileInputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
-import java.net.Socket;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.AbstractMap;
|
|
|
import java.util.ArrayList;
|
|
|
+import java.util.Collection;
|
|
|
import java.util.EnumSet;
|
|
|
import java.util.HashMap;
|
|
|
import java.util.HashSet;
|
|
@@ -32,9 +31,14 @@ import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Map.Entry;
|
|
|
import java.util.Set;
|
|
|
+import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.ConcurrentHashMap;
|
|
|
+import java.util.concurrent.CountDownLatch;
|
|
|
+import java.util.concurrent.ExecutionException;
|
|
|
+import java.util.concurrent.Future;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
+import java.util.concurrent.TimeoutException;
|
|
|
|
|
|
-import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
|
|
import org.apache.hadoop.fs.ByteBufferUtil;
|
|
@@ -54,15 +58,12 @@ import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
|
|
|
import org.apache.hadoop.hdfs.security.token.block.InvalidBlockTokenException;
|
|
|
-import org.apache.hadoop.hdfs.server.datanode.BlockMetadataHeader;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
|
|
|
import org.apache.hadoop.hdfs.server.datanode.ReplicaNotFoundException;
|
|
|
import org.apache.hadoop.io.ByteBufferPool;
|
|
|
import org.apache.hadoop.ipc.RPC;
|
|
|
import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
-import org.apache.hadoop.net.unix.DomainSocket;
|
|
|
-import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.IdentityHashStore;
|
|
@@ -555,7 +556,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
assert (target==pos) : "Wrong postion " + pos + " expect " + target;
|
|
|
long offsetIntoBlock = target - targetBlock.getStartOffset();
|
|
|
|
|
|
- DNAddrPair retval = chooseDataNode(targetBlock);
|
|
|
+ DNAddrPair retval = chooseDataNode(targetBlock, null);
|
|
|
chosenNode = retval.info;
|
|
|
InetSocketAddress targetAddr = retval.addr;
|
|
|
|
|
@@ -863,32 +864,30 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
corruptedBlockMap.put(blk, dnSet);
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- private DNAddrPair chooseDataNode(LocatedBlock block)
|
|
|
- throws IOException {
|
|
|
+
|
|
|
+ private DNAddrPair chooseDataNode(LocatedBlock block,
|
|
|
+ Collection<DatanodeInfo> ignoredNodes) throws IOException {
|
|
|
while (true) {
|
|
|
DatanodeInfo[] nodes = block.getLocations();
|
|
|
try {
|
|
|
- DatanodeInfo chosenNode = bestNode(nodes, deadNodes);
|
|
|
- final String dnAddr =
|
|
|
- chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
|
|
|
- if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
- DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
|
|
|
- }
|
|
|
- InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
|
|
|
- return new DNAddrPair(chosenNode, targetAddr);
|
|
|
+ return getBestNodeDNAddrPair(nodes, ignoredNodes);
|
|
|
} catch (IOException ie) {
|
|
|
+ String errMsg =
|
|
|
+ getBestNodeDNAddrPairErrorString(nodes, deadNodes, ignoredNodes);
|
|
|
String blockInfo = block.getBlock() + " file=" + src;
|
|
|
if (failures >= dfsClient.getMaxBlockAcquireFailures()) {
|
|
|
- throw new BlockMissingException(src, "Could not obtain block: " + blockInfo,
|
|
|
- block.getStartOffset());
|
|
|
+ String description = "Could not obtain block: " + blockInfo;
|
|
|
+ DFSClient.LOG.warn(description + errMsg
|
|
|
+ + ". Throwing a BlockMissingException");
|
|
|
+ throw new BlockMissingException(src, description,
|
|
|
+ block.getStartOffset());
|
|
|
}
|
|
|
|
|
|
if (nodes == null || nodes.length == 0) {
|
|
|
DFSClient.LOG.info("No node available for " + blockInfo);
|
|
|
}
|
|
|
DFSClient.LOG.info("Could not obtain " + block.getBlock()
|
|
|
- + " from any node: " + ie
|
|
|
+ + " from any node: " + ie + errMsg
|
|
|
+ ". Will get new block locations from namenode and retry...");
|
|
|
try {
|
|
|
// Introducing a random factor to the wait time before another retry.
|
|
@@ -914,21 +913,99 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
continue;
|
|
|
}
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Get the best node.
|
|
|
+ * @param nodes Nodes to choose from.
|
|
|
+ * @param ignoredNodes Do not chose nodes in this array (may be null)
|
|
|
+ * @return The DNAddrPair of the best node.
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private DNAddrPair getBestNodeDNAddrPair(final DatanodeInfo[] nodes,
|
|
|
+ Collection<DatanodeInfo> ignoredNodes) throws IOException {
|
|
|
+ DatanodeInfo chosenNode = bestNode(nodes, deadNodes, ignoredNodes);
|
|
|
+ final String dnAddr =
|
|
|
+ chosenNode.getXferAddr(dfsClient.getConf().connectToDnViaHostname);
|
|
|
+ if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
+ DFSClient.LOG.debug("Connecting to datanode " + dnAddr);
|
|
|
+ }
|
|
|
+ InetSocketAddress targetAddr = NetUtils.createSocketAddr(dnAddr);
|
|
|
+ return new DNAddrPair(chosenNode, targetAddr);
|
|
|
+ }
|
|
|
+
|
|
|
+ private static String getBestNodeDNAddrPairErrorString(
|
|
|
+ DatanodeInfo nodes[], AbstractMap<DatanodeInfo,
|
|
|
+ DatanodeInfo> deadNodes, Collection<DatanodeInfo> ignoredNodes) {
|
|
|
+ StringBuilder errMsgr = new StringBuilder(
|
|
|
+ " No live nodes contain current block ");
|
|
|
+ errMsgr.append("Block locations:");
|
|
|
+ for (DatanodeInfo datanode : nodes) {
|
|
|
+ errMsgr.append(" ");
|
|
|
+ errMsgr.append(datanode.toString());
|
|
|
+ }
|
|
|
+ errMsgr.append(" Dead nodes: ");
|
|
|
+ for (DatanodeInfo datanode : deadNodes.keySet()) {
|
|
|
+ errMsgr.append(" ");
|
|
|
+ errMsgr.append(datanode.toString());
|
|
|
+ }
|
|
|
+ if (ignoredNodes != null) {
|
|
|
+ errMsgr.append(" Ignored nodes: ");
|
|
|
+ for (DatanodeInfo datanode : ignoredNodes) {
|
|
|
+ errMsgr.append(" ");
|
|
|
+ errMsgr.append(datanode.toString());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return errMsgr.toString();
|
|
|
+ }
|
|
|
+
|
|
|
private void fetchBlockByteRange(LocatedBlock block, long start, long end,
|
|
|
byte[] buf, int offset,
|
|
|
Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
throws IOException {
|
|
|
- //
|
|
|
- // Connect to best DataNode for desired Block, with potential offset
|
|
|
- //
|
|
|
+ block = getBlockAt(block.getStartOffset(), false);
|
|
|
+ while (true) {
|
|
|
+ DNAddrPair addressPair = chooseDataNode(block, null);
|
|
|
+ try {
|
|
|
+ actualGetFromOneDataNode(addressPair, block, start, end, buf, offset,
|
|
|
+ corruptedBlockMap);
|
|
|
+ return;
|
|
|
+ } catch (IOException e) {
|
|
|
+ // Ignore. Already processed inside the function.
|
|
|
+ // Loop through to try the next node.
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
|
|
|
+ final LocatedBlock block, final long start, final long end,
|
|
|
+ final ByteBuffer bb,
|
|
|
+ final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
|
|
|
+ final CountDownLatch latch) {
|
|
|
+ return new Callable<ByteBuffer>() {
|
|
|
+ @Override
|
|
|
+ public ByteBuffer call() throws Exception {
|
|
|
+ byte[] buf = bb.array();
|
|
|
+ int offset = bb.position();
|
|
|
+ actualGetFromOneDataNode(datanode, block, start, end, buf, offset,
|
|
|
+ corruptedBlockMap);
|
|
|
+ latch.countDown();
|
|
|
+ return bb;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ }
|
|
|
+
|
|
|
+ private void actualGetFromOneDataNode(final DNAddrPair datanode,
|
|
|
+ LocatedBlock block, final long start, final long end, byte[] buf,
|
|
|
+ int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
+ throws IOException {
|
|
|
+ DFSClientFaultInjector.get().startFetchFromDatanode();
|
|
|
int refetchToken = 1; // only need to get a new access token once
|
|
|
int refetchEncryptionKey = 1; // only need to get a new encryption key once
|
|
|
-
|
|
|
+
|
|
|
while (true) {
|
|
|
// cached block locations may have been updated by chooseDataNode()
|
|
|
- // or fetchBlockAt(). Always get the latest list of locations at the
|
|
|
+ // or fetchBlockAt(). Always get the latest list of locations at the
|
|
|
// start of the loop.
|
|
|
CachingStrategy curCachingStrategy;
|
|
|
boolean allowShortCircuitLocalReads;
|
|
@@ -937,11 +1014,10 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
curCachingStrategy = cachingStrategy;
|
|
|
allowShortCircuitLocalReads = !shortCircuitForbidden();
|
|
|
}
|
|
|
- DNAddrPair retval = chooseDataNode(block);
|
|
|
- DatanodeInfo chosenNode = retval.info;
|
|
|
- InetSocketAddress targetAddr = retval.addr;
|
|
|
+ DatanodeInfo chosenNode = datanode.info;
|
|
|
+ InetSocketAddress targetAddr = datanode.addr;
|
|
|
BlockReader reader = null;
|
|
|
-
|
|
|
+
|
|
|
try {
|
|
|
Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
|
|
|
int len = (int) (end - start + 1);
|
|
@@ -969,11 +1045,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
}
|
|
|
return;
|
|
|
} catch (ChecksumException e) {
|
|
|
- DFSClient.LOG.warn("fetchBlockByteRange(). Got a checksum exception for " +
|
|
|
- src + " at " + block.getBlock() + ":" +
|
|
|
- e.getPos() + " from " + chosenNode);
|
|
|
+ String msg = "fetchBlockByteRange(). Got a checksum exception for "
|
|
|
+ + src + " at " + block.getBlock() + ":" + e.getPos() + " from "
|
|
|
+ + chosenNode;
|
|
|
+ DFSClient.LOG.warn(msg);
|
|
|
// we want to remember what we have tried
|
|
|
addIntoCorruptedBlockMap(block.getBlock(), chosenNode, corruptedBlockMap);
|
|
|
+ addToDeadNodes(chosenNode);
|
|
|
+ throw new IOException(msg);
|
|
|
} catch (IOException e) {
|
|
|
if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) {
|
|
|
DFSClient.LOG.info("Will fetch a new encryption key and retry, "
|
|
@@ -985,22 +1064,164 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
continue;
|
|
|
} else if (refetchToken > 0 && tokenRefetchNeeded(e, targetAddr)) {
|
|
|
refetchToken--;
|
|
|
- fetchBlockAt(block.getStartOffset());
|
|
|
+ try {
|
|
|
+ fetchBlockAt(block.getStartOffset());
|
|
|
+ } catch (IOException fbae) {
|
|
|
+ // ignore IOE, since we can retry it later in a loop
|
|
|
+ }
|
|
|
continue;
|
|
|
} else {
|
|
|
- DFSClient.LOG.warn("Failed to connect to " + targetAddr +
|
|
|
- " for file " + src + " for block " + block.getBlock() + ":" + e);
|
|
|
- if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
- DFSClient.LOG.debug("Connection failure ", e);
|
|
|
- }
|
|
|
+ String msg = "Failed to connect to " + targetAddr + " for file "
|
|
|
+ + src + " for block " + block.getBlock() + ":" + e;
|
|
|
+ DFSClient.LOG.warn("Connection failure: " + msg, e);
|
|
|
+ addToDeadNodes(chosenNode);
|
|
|
+ throw new IOException(msg);
|
|
|
}
|
|
|
} finally {
|
|
|
if (reader != null) {
|
|
|
reader.close();
|
|
|
}
|
|
|
}
|
|
|
- // Put chosen node into dead list, continue
|
|
|
- addToDeadNodes(chosenNode);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
|
|
|
+ * int, Map)} except we start up a second, parallel, 'hedged' read
|
|
|
+ * if the first read is taking longer than configured amount of
|
|
|
+ * time. We then wait on which ever read returns first.
|
|
|
+ *
|
|
|
+ * @param block
|
|
|
+ * @param start
|
|
|
+ * @param end
|
|
|
+ * @param buf
|
|
|
+ * @param offset
|
|
|
+ * @param corruptedBlockMap
|
|
|
+ * @throws IOException
|
|
|
+ */
|
|
|
+ private void hedgedFetchBlockByteRange(LocatedBlock block, long start,
|
|
|
+ long end, byte[] buf, int offset,
|
|
|
+ Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
|
|
|
+ throws IOException {
|
|
|
+ ArrayList<Future<ByteBuffer>> futures = null;
|
|
|
+ ArrayList<DatanodeInfo> ignored = new ArrayList<DatanodeInfo>();
|
|
|
+ ByteBuffer bb = null;
|
|
|
+ int len = (int) (end - start + 1);
|
|
|
+ block = getBlockAt(block.getStartOffset(), false);
|
|
|
+ // Latch shared by all outstanding reads. First to finish closes
|
|
|
+ CountDownLatch hasReceivedResult = new CountDownLatch(1);
|
|
|
+ while (true) {
|
|
|
+ DNAddrPair chosenNode = null;
|
|
|
+ Future<ByteBuffer> future = null;
|
|
|
+ // futures is null if there is no request already executing.
|
|
|
+ if (futures == null) {
|
|
|
+ // chooseDataNode is a commitment. If no node, we go to
|
|
|
+ // the NN to reget block locations. Only go here on first read.
|
|
|
+ chosenNode = chooseDataNode(block, ignored);
|
|
|
+ bb = ByteBuffer.wrap(buf, offset, len);
|
|
|
+ future = getHedgedReadFuture(chosenNode, block, start, end, bb,
|
|
|
+ corruptedBlockMap, hasReceivedResult);
|
|
|
+ try {
|
|
|
+ future.get(dfsClient.getHedgedReadTimeout(), TimeUnit.MILLISECONDS);
|
|
|
+ return;
|
|
|
+ } catch (TimeoutException e) {
|
|
|
+ if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
+ DFSClient.LOG.debug("Waited " + dfsClient.getHedgedReadTimeout() +
|
|
|
+ "ms to read from " + chosenNode.info + "; spawning hedged read");
|
|
|
+ }
|
|
|
+ // Ignore this node on next go around.
|
|
|
+ ignored.add(chosenNode.info);
|
|
|
+ dfsClient.getHedgedReadMetrics().incHedgedReadOps();
|
|
|
+ futures = new ArrayList<Future<ByteBuffer>>();
|
|
|
+ futures.add(future);
|
|
|
+ continue; // no need to refresh block locations
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ // Ignore
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ // Ignore already logged in the call.
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ // We are starting up a 'hedged' read. We have a read already
|
|
|
+ // ongoing. Call getBestNodeDNAddrPair instead of chooseDataNode.
|
|
|
+ // If no nodes to do hedged reads against, pass.
|
|
|
+ try {
|
|
|
+ chosenNode = getBestNodeDNAddrPair(block.getLocations(), ignored);
|
|
|
+ bb = ByteBuffer.allocate(len);
|
|
|
+ future = getHedgedReadFuture(chosenNode, block, start, end, bb,
|
|
|
+ corruptedBlockMap, hasReceivedResult);
|
|
|
+ futures.add(future);
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ if (DFSClient.LOG.isDebugEnabled()) {
|
|
|
+ DFSClient.LOG.debug("Failed getting node for hedged read: " +
|
|
|
+ ioe.getMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+ // if not succeeded. Submit callables for each datanode in a loop, wait
|
|
|
+ // for a fixed interval and get the result from the fastest one.
|
|
|
+ try {
|
|
|
+ ByteBuffer result = getFirstToComplete(futures, hasReceivedResult);
|
|
|
+ // cancel the rest.
|
|
|
+ cancelAll(futures);
|
|
|
+ if (result.array() != buf) { // compare the array pointers
|
|
|
+ dfsClient.getHedgedReadMetrics().incHedgedReadWins();
|
|
|
+ System.arraycopy(result.array(), result.position(), buf, offset,
|
|
|
+ len);
|
|
|
+ } else {
|
|
|
+ dfsClient.getHedgedReadMetrics().incHedgedReadOps();
|
|
|
+ }
|
|
|
+ return;
|
|
|
+ } catch (InterruptedException ie) {
|
|
|
+ // Ignore
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ // exception already handled in the call method. getFirstToComplete
|
|
|
+ // will remove the failing future from the list. nothing more to do.
|
|
|
+ }
|
|
|
+ // We got here if exception. Ignore this node on next go around.
|
|
|
+ ignored.add(chosenNode.info);
|
|
|
+ }
|
|
|
+ // executed if we get an error from a data node
|
|
|
+ block = getBlockAt(block.getStartOffset(), false);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ private Future<ByteBuffer> getHedgedReadFuture(final DNAddrPair chosenNode,
|
|
|
+ final LocatedBlock block, long start,
|
|
|
+ final long end, final ByteBuffer bb,
|
|
|
+ final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
|
|
|
+ final CountDownLatch hasReceivedResult) {
|
|
|
+ Callable<ByteBuffer> getFromDataNodeCallable =
|
|
|
+ getFromOneDataNode(chosenNode, block, start, end, bb,
|
|
|
+ corruptedBlockMap, hasReceivedResult);
|
|
|
+ return dfsClient.getHedgedReadsThreadPool().submit(getFromDataNodeCallable);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ByteBuffer getFirstToComplete(ArrayList<Future<ByteBuffer>> futures,
|
|
|
+ CountDownLatch latch) throws ExecutionException, InterruptedException {
|
|
|
+ latch.await();
|
|
|
+ for (Future<ByteBuffer> future : futures) {
|
|
|
+ if (future.isDone()) {
|
|
|
+ try {
|
|
|
+ return future.get();
|
|
|
+ } catch (ExecutionException e) {
|
|
|
+ // already logged in the Callable
|
|
|
+ futures.remove(future);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ throw new InterruptedException("latch has counted down to zero but no"
|
|
|
+ + "result available yet, for safety try to request another one from"
|
|
|
+ + "outside loop, this should be rare");
|
|
|
+ }
|
|
|
+
|
|
|
+ private void cancelAll(List<Future<ByteBuffer>> futures) {
|
|
|
+ for (Future<ByteBuffer> future : futures) {
|
|
|
+ // Unfortunately, hdfs reads do not take kindly to interruption.
|
|
|
+ // Threads return a variety of interrupted-type exceptions but
|
|
|
+ // also complaints about invalid pbs -- likely because read
|
|
|
+ // is interrupted before gets whole pb. Also verbose WARN
|
|
|
+ // logging. So, for now, do not interrupt running read.
|
|
|
+ future.cancel(false);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1070,8 +1291,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
long targetStart = position - blk.getStartOffset();
|
|
|
long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
|
|
|
try {
|
|
|
- fetchBlockByteRange(blk, targetStart,
|
|
|
- targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
|
|
|
+ if (dfsClient.isHedgedReadsEnabled()) {
|
|
|
+ hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead
|
|
|
+ - 1, buffer, offset, corruptedBlockMap);
|
|
|
+ } else {
|
|
|
+ fetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1,
|
|
|
+ buffer, offset, corruptedBlockMap);
|
|
|
+ }
|
|
|
} finally {
|
|
|
// Check and report if any block replicas are corrupted.
|
|
|
// BlockMissingException may be caught if all block replicas are
|
|
@@ -1265,12 +1491,13 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
|
|
|
* Pick the best node from which to stream the data.
|
|
|
* Entries in <i>nodes</i> are already in the priority order
|
|
|
*/
|
|
|
- static DatanodeInfo bestNode(DatanodeInfo nodes[],
|
|
|
- AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes)
|
|
|
- throws IOException {
|
|
|
- if (nodes != null) {
|
|
|
+ static DatanodeInfo bestNode(DatanodeInfo nodes[],
|
|
|
+ AbstractMap<DatanodeInfo, DatanodeInfo> deadNodes,
|
|
|
+ Collection<DatanodeInfo> ignoredNodes) throws IOException {
|
|
|
+ if (nodes != null) {
|
|
|
for (int i = 0; i < nodes.length; i++) {
|
|
|
- if (!deadNodes.containsKey(nodes[i])) {
|
|
|
+ if (!deadNodes.containsKey(nodes[i])
|
|
|
+ && (ignoredNodes == null || !ignoredNodes.contains(nodes[i]))) {
|
|
|
return nodes[i];
|
|
|
}
|
|
|
}
|