|
@@ -35,8 +35,8 @@ import java.util.concurrent.ConcurrentHashMap;
|
|
|
|
|
|
import org.apache.commons.io.IOUtils;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
-import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.ByteBufferReadable;
|
|
|
+import org.apache.hadoop.fs.ChecksumException;
|
|
|
import org.apache.hadoop.fs.FSInputStream;
|
|
|
import org.apache.hadoop.fs.UnresolvedLinkException;
|
|
|
import org.apache.hadoop.hdfs.net.DomainPeer;
|
|
@@ -56,7 +56,8 @@ import org.apache.hadoop.ipc.RemoteException;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.net.unix.DomainSocket;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
-import org.apache.hadoop.hdfs.FileInputStreamCache;
|
|
|
+
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
|
|
|
/****************************************************************
|
|
|
* DFSInputStream provides bytes from a named file. It handles
|
|
@@ -64,11 +65,11 @@ import org.apache.hadoop.hdfs.FileInputStreamCache;
|
|
|
****************************************************************/
|
|
|
@InterfaceAudience.Private
|
|
|
public class DFSInputStream extends FSInputStream implements ByteBufferReadable {
|
|
|
+ @VisibleForTesting
|
|
|
+ static boolean tcpReadsDisabledForTesting = false;
|
|
|
private final PeerCache peerCache;
|
|
|
-
|
|
|
private final DFSClient dfsClient;
|
|
|
private boolean closed = false;
|
|
|
-
|
|
|
private final String src;
|
|
|
private final long prefetchSize;
|
|
|
private BlockReader blockReader = null;
|
|
@@ -853,33 +854,23 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private Peer newPeer(InetSocketAddress addr) throws IOException {
|
|
|
+ private Peer newTcpPeer(InetSocketAddress addr) throws IOException {
|
|
|
Peer peer = null;
|
|
|
boolean success = false;
|
|
|
Socket sock = null;
|
|
|
- DomainSocket domSock = null;
|
|
|
-
|
|
|
try {
|
|
|
- domSock = dfsClient.getDomainSocketFactory().create(addr, this);
|
|
|
- if (domSock != null) {
|
|
|
- // Create a UNIX Domain peer.
|
|
|
- peer = new DomainPeer(domSock);
|
|
|
- } else {
|
|
|
- // Create a conventional TCP-based Peer.
|
|
|
- sock = dfsClient.socketFactory.createSocket();
|
|
|
- NetUtils.connect(sock, addr,
|
|
|
- dfsClient.getRandomLocalInterfaceAddr(),
|
|
|
- dfsClient.getConf().socketTimeout);
|
|
|
- peer = TcpPeerServer.peerFromSocketAndKey(sock,
|
|
|
- dfsClient.getDataEncryptionKey());
|
|
|
- }
|
|
|
+ sock = dfsClient.socketFactory.createSocket();
|
|
|
+ NetUtils.connect(sock, addr,
|
|
|
+ dfsClient.getRandomLocalInterfaceAddr(),
|
|
|
+ dfsClient.getConf().socketTimeout);
|
|
|
+ peer = TcpPeerServer.peerFromSocketAndKey(sock,
|
|
|
+ dfsClient.getDataEncryptionKey());
|
|
|
success = true;
|
|
|
return peer;
|
|
|
} finally {
|
|
|
if (!success) {
|
|
|
IOUtils.closeQuietly(peer);
|
|
|
IOUtils.closeQuietly(sock);
|
|
|
- IOUtils.closeQuietly(domSock);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -888,6 +879,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|
|
* Retrieve a BlockReader suitable for reading.
|
|
|
* This method will reuse the cached connection to the DN if appropriate.
|
|
|
* Otherwise, it will create a new connection.
|
|
|
+ * Throwing an IOException from this method is basically equivalent to
|
|
|
+ * declaring the DataNode bad, so we try to connect a lot of different ways
|
|
|
+ * before doing that.
|
|
|
*
|
|
|
* @param dnAddr Address of the datanode
|
|
|
* @param chosenNode Chosen datanode information
|
|
@@ -912,9 +906,6 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|
|
boolean verifyChecksum,
|
|
|
String clientName)
|
|
|
throws IOException {
|
|
|
-
|
|
|
- IOException err = null;
|
|
|
-
|
|
|
// Firstly, we check to see if we have cached any file descriptors for
|
|
|
// local blocks. If so, we can just re-use those file descriptors.
|
|
|
FileInputStream fis[] = fileInputStreamCache.get(chosenNode, block);
|
|
@@ -927,67 +918,84 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable
|
|
|
block, startOffset, len, fis[0], fis[1], chosenNode, verifyChecksum);
|
|
|
}
|
|
|
|
|
|
- // We retry several times here.
|
|
|
- // On the first nCachedConnRetry times, we try to fetch a socket from
|
|
|
- // the socketCache and use it. This may fail, since the old socket may
|
|
|
- // have been closed by the peer.
|
|
|
- // After that, we try to create a new socket using newPeer().
|
|
|
- // This may create either a TCP socket or a UNIX domain socket, depending
|
|
|
- // on the configuration and whether the peer is remote.
|
|
|
- // If we try to create a UNIX domain socket and fail, we will not try that
|
|
|
- // again. Instead, we'll try to create a TCP socket. Only after we've
|
|
|
- // failed to create a TCP-based BlockReader will we throw an IOException
|
|
|
- // from this function. Throwing an IOException from here is basically
|
|
|
- // equivalent to declaring the DataNode bad.
|
|
|
- boolean triedNonDomainSocketReader = false;
|
|
|
- for (int retries = 0;
|
|
|
- retries < nCachedConnRetry || (!triedNonDomainSocketReader);
|
|
|
- ++retries) {
|
|
|
- Peer peer = null;
|
|
|
- if (retries < nCachedConnRetry) {
|
|
|
- peer = peerCache.get(chosenNode);
|
|
|
- }
|
|
|
- if (peer == null) {
|
|
|
- peer = newPeer(dnAddr);
|
|
|
- if (peer.getDomainSocket() == null) {
|
|
|
- triedNonDomainSocketReader = true;
|
|
|
+ // Look for cached domain peers.
|
|
|
+ int cacheTries = 0;
|
|
|
+ DomainSocketFactory dsFactory = dfsClient.getDomainSocketFactory();
|
|
|
+ BlockReader reader = null;
|
|
|
+ for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
|
|
+ Peer peer = peerCache.get(chosenNode, true);
|
|
|
+ if (peer == null) break;
|
|
|
+ try {
|
|
|
+ boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
|
|
+ shortCircuitLocalReads && (!shortCircuitForbidden());
|
|
|
+ reader = BlockReaderFactory.newBlockReader(
|
|
|
+ dfsClient.conf, file, block, blockToken, startOffset,
|
|
|
+ len, verifyChecksum, clientName, peer, chosenNode,
|
|
|
+ dsFactory, allowShortCircuitLocalReads);
|
|
|
+ return reader;
|
|
|
+ } catch (IOException ex) {
|
|
|
+ DFSClient.LOG.debug("Error making BlockReader with DomainSocket. " +
|
|
|
+ "Closing stale " + peer, ex);
|
|
|
+ } finally {
|
|
|
+ if (reader == null) {
|
|
|
+ IOUtils.closeQuietly(peer);
|
|
|
}
|
|
|
}
|
|
|
- boolean success = false;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Try to create a DomainPeer.
|
|
|
+ DomainSocket domSock = dsFactory.create(dnAddr, this);
|
|
|
+ if (domSock != null) {
|
|
|
+ Peer peer = new DomainPeer(domSock);
|
|
|
try {
|
|
|
- boolean allowShortCircuitLocalReads =
|
|
|
- (peer.getDomainSocket() != null) &&
|
|
|
- dfsClient.getConf().shortCircuitLocalReads &&
|
|
|
- (!shortCircuitForbidden());
|
|
|
- // Here we will try to send either an OP_READ_BLOCK request or an
|
|
|
- // OP_REQUEST_SHORT_CIRCUIT_FDS, depending on what kind of block reader
|
|
|
- // we're trying to create.
|
|
|
- BlockReader blockReader = BlockReaderFactory.newBlockReader(
|
|
|
+ boolean allowShortCircuitLocalReads = dfsClient.getConf().
|
|
|
+ shortCircuitLocalReads && (!shortCircuitForbidden());
|
|
|
+ reader = BlockReaderFactory.newBlockReader(
|
|
|
dfsClient.conf, file, block, blockToken, startOffset,
|
|
|
len, verifyChecksum, clientName, peer, chosenNode,
|
|
|
- dfsClient.getDomainSocketFactory(), allowShortCircuitLocalReads);
|
|
|
- success = true;
|
|
|
- return blockReader;
|
|
|
- } catch (IOException ex) {
|
|
|
- // Our socket is no good.
|
|
|
- DFSClient.LOG.debug("Error making BlockReader. " +
|
|
|
- "Closing stale " + peer, ex);
|
|
|
- if (peer.getDomainSocket() != null) {
|
|
|
- // If the Peer that we got the error from was a DomainPeer,
|
|
|
- // mark the socket path as bad, so that newDataSocket will not try
|
|
|
- // to re-open this socket for a while.
|
|
|
- dfsClient.getDomainSocketFactory().
|
|
|
- disableDomainSocketPath(peer.getDomainSocket().getPath());
|
|
|
+ dsFactory, allowShortCircuitLocalReads);
|
|
|
+ return reader;
|
|
|
+ } catch (IOException e) {
|
|
|
+ DFSClient.LOG.warn("failed to connect to " + domSock, e);
|
|
|
+ } finally {
|
|
|
+ if (reader == null) {
|
|
|
+ // If the Peer that we got the error from was a DomainPeer,
|
|
|
+ // mark the socket path as bad, so that newDataSocket will not try
|
|
|
+ // to re-open this socket for a while.
|
|
|
+ dsFactory.disableDomainSocketPath(domSock.getPath());
|
|
|
+ IOUtils.closeQuietly(peer);
|
|
|
}
|
|
|
- err = ex;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ // Look for cached peers.
|
|
|
+ for (; cacheTries < nCachedConnRetry; ++cacheTries) {
|
|
|
+ Peer peer = peerCache.get(chosenNode, false);
|
|
|
+ if (peer == null) break;
|
|
|
+ try {
|
|
|
+ reader = BlockReaderFactory.newBlockReader(
|
|
|
+ dfsClient.conf, file, block, blockToken, startOffset,
|
|
|
+ len, verifyChecksum, clientName, peer, chosenNode,
|
|
|
+ dsFactory, false);
|
|
|
+ return reader;
|
|
|
+ } catch (IOException ex) {
|
|
|
+ DFSClient.LOG.debug("Error making BlockReader. Closing stale " +
|
|
|
+ peer, ex);
|
|
|
} finally {
|
|
|
- if (!success) {
|
|
|
+ if (reader == null) {
|
|
|
IOUtils.closeQuietly(peer);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- throw err;
|
|
|
+ if (tcpReadsDisabledForTesting) {
|
|
|
+ throw new IOException("TCP reads are disabled.");
|
|
|
+ }
|
|
|
+ // Try to create a new remote peer.
|
|
|
+ Peer peer = newTcpPeer(dnAddr);
|
|
|
+ return BlockReaderFactory.newBlockReader(
|
|
|
+ dfsClient.conf, file, block, blockToken, startOffset,
|
|
|
+ len, verifyChecksum, clientName, peer, chosenNode,
|
|
|
+ dsFactory, false);
|
|
|
}
|
|
|
|
|
|
|