Browse Source

HDFS-5488. Clean up TestHftpURLTimeout. Contributed by Haohui Mai.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1540894 13f79535-47bb-0310-9956-ffa450edef68
Jing Zhao 11 years ago
parent
commit
f10642a8fd

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -524,6 +524,8 @@ Release 2.3.0 - UNRELEASED
 
     HDFS-5325. Remove WebHdfsFileSystem#ConnRunner. (Haohui Mai via jing9) 
 
+    HDFS-5488. Clean up TestHftpURLTimeout. (Haohui Mai via jing9)
+
 Release 2.2.1 - UNRELEASED
 
   INCOMPATIBLE CHANGES

+ 56 - 70
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpFileSystem.java

@@ -28,6 +28,7 @@ import java.net.HttpURLConnection;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.net.URL;
+import java.net.URLConnection;
 import java.util.Random;
 
 import org.apache.commons.logging.impl.Log4JLogger;
@@ -40,15 +41,16 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.MiniDFSCluster.Builder;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.hdfs.web.HftpFileSystem;
-import org.apache.hadoop.hdfs.web.HsftpFileSystem;
 import org.apache.hadoop.util.ServletUtil;
 import org.apache.log4j.Level;
-import org.junit.*;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
 
 public class TestHftpFileSystem {
   private static final Random RAN = new Random();
@@ -65,32 +67,24 @@ public class TestHftpFileSystem {
       new Path("/foo;bar"),
 
       // URI does not encode, Request#getPathInfo returns verbatim
-      new Path("/foo+"),
-      new Path("/foo+bar/foo+bar"),
-      new Path("/foo=bar/foo=bar"),
-      new Path("/foo,bar/foo,bar"),
-      new Path("/foo@bar/foo@bar"),
-      new Path("/foo&bar/foo&bar"),
-      new Path("/foo$bar/foo$bar"),
-      new Path("/foo_bar/foo_bar"),
-      new Path("/foo~bar/foo~bar"),
-      new Path("/foo.bar/foo.bar"),
-      new Path("/foo../bar/foo../bar"),
-      new Path("/foo.../bar/foo.../bar"),
+      new Path("/foo+"), new Path("/foo+bar/foo+bar"),
+      new Path("/foo=bar/foo=bar"), new Path("/foo,bar/foo,bar"),
+      new Path("/foo@bar/foo@bar"), new Path("/foo&bar/foo&bar"),
+      new Path("/foo$bar/foo$bar"), new Path("/foo_bar/foo_bar"),
+      new Path("/foo~bar/foo~bar"), new Path("/foo.bar/foo.bar"),
+      new Path("/foo../bar/foo../bar"), new Path("/foo.../bar/foo.../bar"),
       new Path("/foo'bar/foo'bar"),
       new Path("/foo#bar/foo#bar"),
       new Path("/foo!bar/foo!bar"),
       // HDFS file names may not contain ":"
 
       // URI percent encodes, Request#getPathInfo decodes
-      new Path("/foo bar/foo bar"),
-      new Path("/foo?bar/foo?bar"),
-      new Path("/foo\">bar/foo\">bar"),
-    };
+      new Path("/foo bar/foo bar"), new Path("/foo?bar/foo?bar"),
+      new Path("/foo\">bar/foo\">bar"), };
 
   @BeforeClass
   public static void setUp() throws IOException {
-    ((Log4JLogger)HftpFileSystem.LOG).getLogger().setLevel(Level.ALL);
+    ((Log4JLogger) HftpFileSystem.LOG).getLogger().setLevel(Level.ALL);
 
     final long seed = RAN.nextLong();
     System.out.println("seed=" + seed);
@@ -99,8 +93,8 @@ public class TestHftpFileSystem {
     config = new Configuration();
     cluster = new MiniDFSCluster.Builder(config).numDataNodes(2).build();
     blockPoolId = cluster.getNamesystem().getBlockPoolId();
-    hftpUri =
-      "hftp://" + config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
+    hftpUri = "hftp://"
+        + config.get(DFSConfigKeys.DFS_NAMENODE_HTTP_ADDRESS_KEY);
   }
 
   @AfterClass
@@ -140,7 +134,8 @@ public class TestHftpFileSystem {
 
       // Check the file status matches the path. Hftp returns a FileStatus
       // with the entire URI, extract the path part.
-      assertEquals(p, new Path(hftpFs.getFileStatus(p).getPath().toUri().getPath()));
+      assertEquals(p, new Path(hftpFs.getFileStatus(p).getPath().toUri()
+          .getPath()));
 
       // Test list status (listPath servlet)
       assertEquals(1, hftpFs.listStatus(p).length);
@@ -158,21 +153,20 @@ public class TestHftpFileSystem {
     if (hdfs.exists(path)) {
       hdfs.delete(path, true);
     }
-    FSDataOutputStream out = hdfs.create(path, (short)1);
+    FSDataOutputStream out = hdfs.create(path, (short) 1);
     out.writeBytes("0123456789");
     out.close();
 
     // Get the path's block location so we can determine
     // if we were redirected to the right DN.
-    BlockLocation[] locations =
-        hdfs.getFileBlockLocations(path, 0, 10);
+    BlockLocation[] locations = hdfs.getFileBlockLocations(path, 0, 10);
     String xferAddr = locations[0].getNames()[0];
 
     // Connect to the NN to get redirected
     URL u = hftpFs.getNamenodeURL(
         "/data" + ServletUtil.encodePath(path.toUri().getPath()),
         "ugi=userx,groupy");
-    HttpURLConnection conn = (HttpURLConnection)u.openConnection();
+    HttpURLConnection conn = (HttpURLConnection) u.openConnection();
     HttpURLConnection.setFollowRedirects(true);
     conn.connect();
     conn.getInputStream();
@@ -181,15 +175,15 @@ public class TestHftpFileSystem {
     // Find the datanode that has the block according to locations
     // and check that the URL was redirected to this DN's info port
     for (DataNode node : cluster.getDataNodes()) {
-      DatanodeRegistration dnR =
-        DataNodeTestUtils.getDNRegistrationForBP(node, blockPoolId);
+      DatanodeRegistration dnR = DataNodeTestUtils.getDNRegistrationForBP(node,
+          blockPoolId);
       if (dnR.getXferAddr().equals(xferAddr)) {
         checked = true;
         assertEquals(dnR.getInfoPort(), conn.getURL().getPort());
       }
     }
-    assertTrue("The test never checked that location of " +
-               "the block and hftp desitnation are the same", checked);
+    assertTrue("The test never checked that location of "
+        + "the block and hftp desitnation are the same", checked);
   }
 
   /**
@@ -260,7 +254,7 @@ public class TestHftpFileSystem {
     os.writeBytes("0123456789");
     os.close();
 
-    // ByteRangeInputStream delays opens until reads.  Make sure it doesn't
+    // ByteRangeInputStream delays opens until reads. Make sure it doesn't
     // open a closed stream that has never been opened
     FSDataInputStream in = hftpFs.open(testFile);
     in.close();
@@ -298,16 +292,15 @@ public class TestHftpFileSystem {
     URI uri = URI.create("hftp://localhost");
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
+    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+        fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
 
     // HFTP uses http to get the token so canonical service name should
     // return the http port.
-    assertEquals(
-        "127.0.0.1:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:" + DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+        fs.getCanonicalServiceName());
   }
 
   @Test
@@ -324,10 +317,7 @@ public class TestHftpFileSystem {
 
     // HFTP uses http to get the token so canonical service name should
     // return the http port.
-    assertEquals(
-        "127.0.0.1:123",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:123", fs.getCanonicalServiceName());
   }
 
   @Test
@@ -336,13 +326,11 @@ public class TestHftpFileSystem {
     URI uri = URI.create("hftp://localhost:123");
     HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
 
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT, fs.getDefaultPort());
+    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTP_PORT_DEFAULT,
+        fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:123",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:123", fs.getCanonicalServiceName());
   }
 
   @Test
@@ -356,13 +344,20 @@ public class TestHftpFileSystem {
     assertEquals(123, fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:789",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:789", fs.getCanonicalServiceName());
   }
 
-  ///
+  @Test
+  public void testTimeout() throws IOException {
+    Configuration conf = new Configuration();
+    URI uri = URI.create("hftp://localhost");
+    HftpFileSystem fs = (HftpFileSystem) FileSystem.get(uri, conf);
+    URLConnection conn = fs.connectionFactory.openConnection(new URL("http://localhost"));
+    assertEquals(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, conn.getConnectTimeout());
+    assertEquals(URLConnectionFactory.DEFAULT_SOCKET_TIMEOUT, conn.getReadTimeout());
+  }
+
+  // /
 
   @Test
   public void testHsftpDefaultPorts() throws IOException {
@@ -370,13 +365,12 @@ public class TestHftpFileSystem {
     URI uri = URI.create("hsftp://localhost");
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
+    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
+        fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:"+DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:" + DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
+        fs.getCanonicalServiceName());
   }
 
   @Test
@@ -391,10 +385,7 @@ public class TestHftpFileSystem {
     assertEquals(456, fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:456",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:456", fs.getCanonicalServiceName());
   }
 
   @Test
@@ -403,13 +394,11 @@ public class TestHftpFileSystem {
     URI uri = URI.create("hsftp://localhost:123");
     HsftpFileSystem fs = (HsftpFileSystem) FileSystem.get(uri, conf);
 
-    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT, fs.getDefaultPort());
+    assertEquals(DFSConfigKeys.DFS_NAMENODE_HTTPS_PORT_DEFAULT,
+        fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:123",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:123", fs.getCanonicalServiceName());
   }
 
   @Test
@@ -424,9 +413,6 @@ public class TestHftpFileSystem {
     assertEquals(456, fs.getDefaultPort());
 
     assertEquals(uri, fs.getUri());
-    assertEquals(
-        "127.0.0.1:789",
-        fs.getCanonicalServiceName()
-    );
+    assertEquals("127.0.0.1:789", fs.getCanonicalServiceName());
   }
 }

+ 0 - 140
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestHftpURLTimeouts.java

@@ -1,140 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.hdfs.web;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-
-import java.io.IOException;
-import java.net.HttpURLConnection;
-import java.net.InetAddress;
-import java.net.ServerSocket;
-import java.net.SocketTimeoutException;
-import java.net.URI;
-import java.util.LinkedList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.hdfs.web.HftpFileSystem;
-import org.apache.hadoop.hdfs.web.HsftpFileSystem;
-import org.apache.hadoop.hdfs.web.URLConnectionFactory;
-import org.junit.Test;
-
-public class TestHftpURLTimeouts {
-
-  @Test
-  public void testHftpSocketTimeout() throws Exception {
-    Configuration conf = new Configuration();
-    ServerSocket socket = new ServerSocket(0,1);
-    URI uri = new URI("hftp", null,
-        InetAddress.getByName(null).getHostAddress(),
-        socket.getLocalPort(),
-        null, null, null);
-
-    HftpFileSystem fs = (HftpFileSystem)FileSystem.get(uri, conf);
-    fs.connectionFactory = new URLConnectionFactory(5);
-
-    boolean timedout = false;
-    try {
-      HttpURLConnection conn = fs.openConnection("/", "");
-      timedout = false;
-      try {
-        // this will consume the only slot in the backlog
-        conn.getInputStream();
-      } catch (SocketTimeoutException ste) {
-        timedout = true;
-        assertEquals("Read timed out", ste.getMessage());
-      } finally {
-        if (conn != null) conn.disconnect();
-      }
-      assertTrue("read timedout", timedout);
-      assertTrue("connect timedout", checkConnectTimeout(fs, false));
-    } finally {
-      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
-      fs.close();
-    }
-  }
-
-  @Test
-  public void testHsftpSocketTimeout() throws Exception {
-    Configuration conf = new Configuration();
-    ServerSocket socket = new ServerSocket(0,1);
-    URI uri = new URI("hsftp", null,
-        InetAddress.getByName(null).getHostAddress(),
-        socket.getLocalPort(),
-        null, null, null);
-    boolean timedout = false;
-
-    HsftpFileSystem fs = (HsftpFileSystem)FileSystem.get(uri, conf);
-    fs.connectionFactory = new URLConnectionFactory(5);
-
-    try {
-      HttpURLConnection conn = null;
-      timedout = false;
-      try {
-        // this will consume the only slot in the backlog
-        conn = fs.openConnection("/", "");
-      } catch (SocketTimeoutException ste) {
-        // SSL expects a negotiation, so it will timeout on read, unlike hftp
-        timedout = true;
-        assertEquals("Read timed out", ste.getMessage());
-      } finally {
-        if (conn != null) conn.disconnect();
-      }
-      assertTrue("ssl read connect timedout", timedout);
-      assertTrue("connect timedout", checkConnectTimeout(fs, true));
-    } finally {
-      fs.connectionFactory = URLConnectionFactory.DEFAULT_CONNECTION_FACTORY;
-      fs.close();
-    }
-  }
-
-  private boolean checkConnectTimeout(HftpFileSystem fs, boolean ignoreReadTimeout)
-      throws IOException {
-    boolean timedout = false;
-    List<HttpURLConnection> conns = new LinkedList<HttpURLConnection>();
-    try {
-      // with a listen backlog of 1, should only have to make one connection
-      // to trigger a connection timeout.  however... linux doesn't honor the
-      // socket's listen backlog so we have to try a bunch of times
-      for (int n=32; !timedout && n > 0; n--) {
-        try {
-          conns.add(fs.openConnection("/", ""));
-        } catch (SocketTimeoutException ste) {
-          String message = ste.getMessage();
-          assertNotNull(message);
-          // https will get a read timeout due to SSL negotiation, but
-          // a normal http will not, so need to ignore SSL read timeouts
-          // until a connect timeout occurs
-          if (!(ignoreReadTimeout && "Read timed out".equals(message))) {
-            timedout = true;
-            assertEquals("connect timed out", message);
-          }
-        }
-      }
-    } finally {
-      for (HttpURLConnection conn : conns) {
-        conn.disconnect();
-      }
-    }
-    return timedout;
-  }
-}