You are viewing a plain text version of this content. The canonical link for it is here.
Posted to hdfs-commits@hadoop.apache.org by da...@apache.org on 2012/07/09 17:28:27 UTC
svn commit: r1359221 - in
/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs:
CHANGES.txt src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
Author: daryn
Date: Mon Jul 9 15:28:27 2012
New Revision: 1359221
URL: http://svn.apache.org/viewvc?rev=1359221&view=rev
Log:
HDFS-3376. DFSClient fails to make connection to DN if there are many unusable cached sockets. Contributed by Todd Lipcon.
Modified:
hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt?rev=1359221&r1=1359220&r2=1359221&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt Mon Jul 9 15:28:27 2012
@@ -50,6 +50,9 @@ Release 0.23.3 - UNRELEASED
HDFS-3357. DataXceiver reads from client socket with incorrect/no timeout
(todd)
+ HDFS-3376. DFSClient fails to make connection to DN if there are many
+ unusable cached sockets (todd)
+
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES
Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java?rev=1359221&r1=1359220&r2=1359221&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java Mon Jul 9 15:28:27 2012
@@ -759,7 +759,13 @@ public class DFSInputStream extends FSIn
// Allow retry since there is no way of knowing whether the cached socket
// is good until we actually use it.
for (int retries = 0; retries <= nCachedConnRetry && fromCache; ++retries) {
- Socket sock = socketCache.get(dnAddr);
+ Socket sock = null;
+ // Don't use the cache on the last attempt - it's possible that there
+ // are arbitrarily many unusable sockets in the cache, but we don't
+ // want to fail the read.
+ if (retries < nCachedConnRetry) {
+ sock = socketCache.get(dnAddr);
+ }
if (sock == null) {
fromCache = false;
Modified: hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java?rev=1359221&r1=1359220&r2=1359221&view=diff
==============================================================================
--- hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java (original)
+++ hadoop/common/branches/branch-0.23/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferKeepalive.java Mon Jul 9 15:28:27 2012
@@ -17,10 +17,12 @@
*/
package org.apache.hadoop.hdfs;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
import static org.junit.Assert.*;
+import java.io.InputStream;
import java.io.PrintWriter;
import java.net.InetSocketAddress;
import java.net.Socket;
@@ -40,6 +42,8 @@ import org.junit.After;
import org.junit.Before;
import org.junit.Test;
+import com.google.common.io.NullOutputStream;
+
public class TestDataTransferKeepalive {
Configuration conf = new HdfsConfiguration();
private MiniDFSCluster cluster;
@@ -56,6 +60,8 @@ public class TestDataTransferKeepalive {
public void setup() throws Exception {
conf.setInt(DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
KEEPALIVE_TIMEOUT);
+ conf.setInt(DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_KEY,
+ 0);
cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(1).build();
@@ -143,6 +149,40 @@ public class TestDataTransferKeepalive {
IOUtils.closeStream(stm);
}
}
+
+ @Test(timeout=30000)
+ public void testManyClosedSocketsInCache() throws Exception {
+ // Make a small file
+ DFSTestUtil.createFile(fs, TEST_FILE, 1L, (short)1, 0L);
+
+ // Insert a bunch of dead sockets in the cache, by opening
+ // many streams concurrently, reading all of the data,
+ // and then closing them.
+ InputStream[] stms = new InputStream[5];
+ try {
+ for (int i = 0; i < stms.length; i++) {
+ stms[i] = fs.open(TEST_FILE);
+ }
+ for (InputStream stm : stms) {
+ IOUtils.copyBytes(stm, new NullOutputStream(), 1024);
+ }
+ } finally {
+ IOUtils.cleanup(null, stms);
+ }
+
+ DFSClient client = ((DistributedFileSystem)fs).dfs;
+ assertEquals(5, client.socketCache.size());
+
+ // Let all the xceivers timeout
+ Thread.sleep(1500);
+ assertXceiverCount(0);
+
+ // Client side still has the sockets cached
+ assertEquals(5, client.socketCache.size());
+
+ // Reading should not throw an exception.
+ DFSTestUtil.readFile(fs, TEST_FILE);
+ }
private void assertXceiverCount(int expected) {
// Subtract 1, since the DataXceiverServer