|
@@ -85,6 +85,7 @@ public class TestDistributedFileSystem {
|
|
|
/**
|
|
|
* Tests DFSClient.close throws no ConcurrentModificationException if
|
|
|
* multiple files are open.
|
|
|
+ * Also tests that any cached sockets are closed. (HDFS-3359)
|
|
|
*/
|
|
|
@Test
|
|
|
public void testDFSClose() throws Exception {
|
|
@@ -94,11 +95,23 @@ public class TestDistributedFileSystem {
|
|
|
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(2).build();
|
|
|
FileSystem fileSys = cluster.getFileSystem();
|
|
|
|
|
|
- // create two files
|
|
|
+ // create two files, leaving them open
|
|
|
fileSys.create(new Path("/test/dfsclose/file-0"));
|
|
|
fileSys.create(new Path("/test/dfsclose/file-1"));
|
|
|
+
|
|
|
+ // create another file, close it, and read it, so
|
|
|
+ // the client gets a socket in its SocketCache
|
|
|
+ Path p = new Path("/non-empty-file");
|
|
|
+ DFSTestUtil.createFile(fileSys, p, 1L, (short)1, 0L);
|
|
|
+ DFSTestUtil.readFile(fileSys, p);
|
|
|
+
|
|
|
+ DFSClient client = ((DistributedFileSystem)fileSys).dfs;
|
|
|
+ SocketCache cache = client.socketCache;
|
|
|
+ assertEquals(1, cache.size());
|
|
|
|
|
|
fileSys.close();
|
|
|
+
|
|
|
+ assertEquals(0, cache.size());
|
|
|
} finally {
|
|
|
if (cluster != null) {cluster.shutdown();}
|
|
|
}
|