diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index 81709c7201c..a3e8af1b425 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -554,6 +554,8 @@ Release 2.0.1-alpha - UNRELEASED HDFS-3469. start-dfs.sh will start zkfc, but stop-dfs.sh will not stop zkfc similarly. (Vinay via umamahesh) + + HDFS-1490. TransferFSImage should timeout (Dmytro Molkov and Vinay via todd) BREAKDOWN OF HDFS-3042 SUBTASKS diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java index 19dccf9703a..f03de4b89a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java @@ -329,6 +329,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.image.transfer.bandwidthPerSec"; public static final long DFS_IMAGE_TRANSFER_RATE_DEFAULT = 0; //no throttling + // Image transfer timeout + public static final String DFS_IMAGE_TRANSFER_TIMEOUT_KEY = "dfs.image.transfer.timeout"; + public static final int DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT = 60 * 1000; + //Keys with no defaults public static final String DFS_DATANODE_PLUGINS_KEY = "dfs.datanode.plugins"; public static final String DFS_DATANODE_FSDATASET_FACTORY_KEY = "dfs.datanode.fsdataset.factory"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java index 665d3dbe0f6..3c05bce2870 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java @@ -32,9 +32,12 @@ import javax.servlet.http.HttpServletResponse; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.http.HttpConfig; import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.util.Time; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.common.StorageErrorReporter; import org.apache.hadoop.hdfs.server.common.Storage; @@ -43,6 +46,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.util.DataTransferThrottler; import org.apache.hadoop.io.MD5Hash; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; @@ -54,6 +58,8 @@ public class TransferFsImage { public final static String CONTENT_LENGTH = "Content-Length"; public final static String MD5_HEADER = "X-MD5-Digest"; + @VisibleForTesting + static int timeout = 0; private static final Log LOG = LogFactory.getLog(TransferFsImage.class); @@ -223,6 +229,18 @@ public class TransferFsImage { HttpURLConnection connection = (HttpURLConnection) SecurityUtil.openSecureHttpConnection(url); + if (timeout <= 0) { + // Set the ping interval as timeout + Configuration conf = new HdfsConfiguration(); + timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY, + DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT); + } + + if (timeout > 0) { + connection.setConnectTimeout(timeout); + connection.setReadTimeout(timeout); + } + if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) { throw new HttpGetFailedException( "Image transfer servlet at " + url + diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java index ae3f1a9459e..cba634fee92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java @@ -17,17 +17,27 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; import java.io.File; import java.io.IOException; +import java.net.SocketTimeoutException; +import java.net.URL; import java.util.Collections; import java.util.List; +import javax.servlet.ServletException; +import javax.servlet.http.HttpServlet; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; + import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.http.HttpServer; +import org.apache.hadoop.http.HttpServerFunctionalTest; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.util.StringUtils; import org.junit.Test; @@ -100,4 +110,45 @@ public class TestTransferFsImage { cluster.shutdown(); } } + + /** + * Test to verify the read timeout + */ + @Test(timeout = 5000) + public void testImageTransferTimeout() throws Exception { + HttpServer testServer = HttpServerFunctionalTest.createServer("hdfs"); + try { + testServer.addServlet("GetImage", "/getimage", TestGetImageServlet.class); + testServer.start(); + URL serverURL = HttpServerFunctionalTest.getServerURL(testServer); + TransferFsImage.timeout = 2000; + try { + TransferFsImage.getFileClient(serverURL.getAuthority(), "txid=1", null, + null, false); + fail("TransferImage Should fail with timeout"); + } catch (SocketTimeoutException e) { + assertEquals("Read should timeout", "Read timed out", e.getMessage()); + } + } finally { + if (testServer != null) { + testServer.stop(); + } + } + } + + public static class TestGetImageServlet extends HttpServlet { + private static final long serialVersionUID = 1L; + + @Override + protected void doGet(HttpServletRequest req, HttpServletResponse resp) + throws ServletException, IOException { + synchronized (this) { + try { + wait(5000); + } catch (InterruptedException e) { + // Ignore + } + } + } + } }