Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/TransferFsImage.java
 Tue Mar 11 00:40:46 2014
@@ -19,22 +19,18 @@ package org.apache.hadoop.hdfs.server.na
 
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
-import java.io.OutputStream;
 import java.net.HttpURLConnection;
-import java.net.URISyntaxException;
 import java.net.URL;
 import java.security.DigestInputStream;
 import java.security.MessageDigest;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
 
-import javax.servlet.http.HttpServletRequest;
+import javax.servlet.ServletOutputStream;
+import javax.servlet.ServletResponse;
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.commons.logging.Log;
@@ -53,12 +49,10 @@ import org.apache.hadoop.hdfs.server.nam
 import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
-import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.security.UserGroupInformation;
 import 
org.apache.hadoop.security.authentication.client.AuthenticationException;
 import org.apache.hadoop.util.Time;
-import org.apache.http.client.utils.URIBuilder;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Lists;
@@ -71,12 +65,7 @@ import com.google.common.collect.Lists;
 public class TransferFsImage {
   
   public final static String CONTENT_LENGTH = "Content-Length";
-  public final static String FILE_LENGTH = "File-Length";
   public final static String MD5_HEADER = "X-MD5-Digest";
-
-  private final static String CONTENT_TYPE = "Content-Type";
-  private final static String CONTENT_TRANSFER_ENCODING = 
"Content-Transfer-Encoding";
-
   @VisibleForTesting
   static int timeout = 0;
   private static URLConnectionFactory connectionFactory;
@@ -93,14 +82,14 @@ public class TransferFsImage {
   
   public static void downloadMostRecentImageToDirectory(URL infoServer,
       File dir) throws IOException {
-    String fileId = ImageServlet.getParamStringForMostRecentImage();
+    String fileId = GetImageServlet.getParamStringForMostRecentImage();
     getFileClient(infoServer, fileId, Lists.newArrayList(dir),
         null, false);
   }
 
   public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
       Storage dstStorage, boolean needDigest) throws IOException {
-    String fileid = ImageServlet.getParamStringForImage(null,
+    String fileid = GetImageServlet.getParamStringForImage(null,
         imageTxId, dstStorage);
     String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
     
@@ -115,31 +104,12 @@ public class TransferFsImage {
         dstFiles.get(0).length() + " bytes.");
     return hash;
   }
-
-  static MD5Hash handleUploadImageRequest(HttpServletRequest request,
-      long imageTxId, Storage dstStorage, InputStream stream,
-      long advertisedSize, DataTransferThrottler throttler) throws IOException 
{
-
-    String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
-
-    List<File> dstFiles = dstStorage.getFiles(NameNodeDirType.IMAGE, fileName);
-    if (dstFiles.isEmpty()) {
-      throw new IOException("No targets in destination storage!");
-    }
-
-    MD5Hash advertisedDigest = parseMD5Header(request);
-    MD5Hash hash = receiveFile(fileName, dstFiles, dstStorage, true,
-        advertisedSize, advertisedDigest, fileName, stream, throttler);
-    LOG.info("Downloaded file " + dstFiles.get(0).getName() + " size "
-        + dstFiles.get(0).length() + " bytes.");
-    return hash;
-  }
-
+  
   static void downloadEditsToStorage(URL fsName, RemoteEditLog log,
       NNStorage dstStorage) throws IOException {
     assert log.getStartTxId() > 0 && log.getEndTxId() > 0 :
       "bad log: " + log;
-    String fileid = ImageServlet.getParamStringForLog(
+    String fileid = GetImageServlet.getParamStringForLog(
         log, dstStorage);
     String finalFileName = NNStorage.getFinalizedEditsFileName(
         log.getStartTxId(), log.getEndTxId());
@@ -189,19 +159,22 @@ public class TransferFsImage {
    * Requests that the NameNode download an image from this node.
    *
    * @param fsName the http address for the remote NN
-   * @param conf Configuration
+   * @param myNNAddress the host/port where the local node is running an
+   *                           HTTPServer hosting GetImageServlet
    * @param storage the storage directory to transfer the image from
    * @param nnf the NameNodeFile type of the image
    * @param txid the transaction ID of the image to be uploaded
    */
-  public static void uploadImageFromStorage(URL fsName, Configuration conf,
-      NNStorage storage, NameNodeFile nnf, long txid) throws IOException {
+  public static void uploadImageFromStorage(URL fsName, URL myNNAddress,
+      Storage storage, NameNodeFile nnf, long txid) throws IOException {
     
-    URL url = new URL(fsName, ImageServlet.PATH_SPEC);
-    long startTime = Time.monotonicNow();
+    String fileid = GetImageServlet.getParamStringToPutImage(nnf, txid,
+        myNNAddress, storage);
+    // this doesn't directly upload an image, but rather asks the NN
+    // to connect back to the 2NN to download the specified image.
     try {
-      uploadImage(url, conf, storage, nnf, txid);
-    } catch (HttpPutFailedException e) {
+      TransferFsImage.getFileClient(fsName, fileid, null, null, false);
+    } catch (HttpGetFailedException e) {
       if (e.getResponseCode() == HttpServletResponse.SC_CONFLICT) {
         // this is OK - this means that a previous attempt to upload
         // this checkpoint succeeded even though we thought it failed.
@@ -213,105 +186,25 @@ public class TransferFsImage {
         throw e;
       }
     }
-    double xferSec = Math.max(
-        ((float) (Time.monotonicNow() - startTime)) / 1000.0, 0.001);
-    LOG.info("Uploaded image with txid " + txid + " to namenode at " + fsName
-        + " in " + xferSec + " seconds");
-  }
-
-  /*
-   * Uploads the imagefile using HTTP PUT method
-   */
-  private static void uploadImage(URL url, Configuration conf,
-      NNStorage storage, NameNodeFile nnf, long txId) throws IOException {
-
-    File imageFile = storage.findImageFile(nnf, txId);
-    if (imageFile == null) {
-      throw new IOException("Could not find image with txid " + txId);
-    }
-
-    HttpURLConnection connection = null;
-    try {
-      URIBuilder uriBuilder = new URIBuilder(url.toURI());
-
-      // write all params for image upload request as query itself.
-      // Request body contains the image to be uploaded.
-      Map<String, String> params = ImageServlet.getParamsForPutImage(storage,
-          txId, imageFile.length(), nnf);
-      for (Entry<String, String> entry : params.entrySet()) {
-        uriBuilder.addParameter(entry.getKey(), entry.getValue());
-      }
-
-      URL urlWithParams = uriBuilder.build().toURL();
-      connection = (HttpURLConnection) connectionFactory.openConnection(
-          urlWithParams, UserGroupInformation.isSecurityEnabled());
-      // Set the request to PUT
-      connection.setRequestMethod("PUT");
-      connection.setDoOutput(true);
-
-      
-      int chunkSize = conf.getInt(
-          DFSConfigKeys.DFS_IMAGE_TRANSFER_CHUNKSIZE_KEY,
-          DFSConfigKeys.DFS_IMAGE_TRANSFER_CHUNKSIZE_DEFAULT);
-      if (imageFile.length() > chunkSize) {
-        // using chunked streaming mode to support upload of 2GB+ files and to
-        // avoid internal buffering.
-        // this mode should be used only if more than chunkSize data is present
-        // to upload. otherwise upload may not happen sometimes.
-        connection.setChunkedStreamingMode(chunkSize);
-      }
-
-      setTimeout(connection);
-
-      // set headers for verification
-      ImageServlet.setVerificationHeadersForPut(connection, imageFile);
-
-      // Write the file to output stream.
-      writeFileToPutRequest(conf, connection, imageFile);
-
-      int responseCode = connection.getResponseCode();
-      if (responseCode != HttpURLConnection.HTTP_OK) {
-        throw new HttpPutFailedException(connection.getResponseMessage(),
-            responseCode);
-      }
-    } catch (AuthenticationException e) {
-      throw new IOException(e);
-    } catch (URISyntaxException e) {
-      throw new IOException(e);
-    } finally {
-      if (connection != null) {
-        connection.disconnect();
-      }
-    }
-  }
-
-  private static void writeFileToPutRequest(Configuration conf,
-      HttpURLConnection connection, File imageFile)
-      throws FileNotFoundException, IOException {
-    connection.setRequestProperty(CONTENT_TYPE, "application/octet-stream");
-    connection.setRequestProperty(CONTENT_TRANSFER_ENCODING, "binary");
-    OutputStream output = connection.getOutputStream();
-    FileInputStream input = new FileInputStream(imageFile);
-    try {
-      copyFileToStream(output, imageFile, input,
-          ImageServlet.getThrottler(conf));
-    } finally {
-      IOUtils.closeStream(input);
-      IOUtils.closeStream(output);
-    }
+    LOG.info("Uploaded image with txid " + txid + " to namenode at " +
+               fsName);
   }
 
+  
   /**
    * A server-side method to respond to a getfile http request
    * Copies the contents of the local file into the output stream.
    */
-  public static void copyFileToStream(OutputStream out, File localfile,
-      FileInputStream infile, DataTransferThrottler throttler)
+  public static void getFileServer(ServletResponse response, File localfile,
+      FileInputStream infile,
+      DataTransferThrottler throttler) 
     throws IOException {
     byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
+    ServletOutputStream out = null;
     try {
       CheckpointFaultInjector.getInstance()
           .aboutToSendFile(localfile);
+      out = response.getOutputStream();
 
       if (CheckpointFaultInjector.getInstance().
             shouldSendShortFile(localfile)) {
@@ -357,13 +250,14 @@ public class TransferFsImage {
   static MD5Hash getFileClient(URL infoServer,
       String queryString, List<File> localPaths,
       Storage dstStorage, boolean getChecksum) throws IOException {
-    URL url = new URL(infoServer, ImageServlet.PATH_SPEC + "?" + queryString);
+    URL url = new URL(infoServer, "/getimage?" + queryString);
     LOG.info("Opening connection to " + url);
     return doGetUrl(url, localPaths, dstStorage, getChecksum);
   }
   
   public static MD5Hash doGetUrl(URL url, List<File> localPaths,
       Storage dstStorage, boolean getChecksum) throws IOException {
+    long startTime = Time.monotonicNow();
     HttpURLConnection connection;
     try {
       connection = (HttpURLConnection)
@@ -372,7 +266,16 @@ public class TransferFsImage {
       throw new IOException(e);
     }
 
-    setTimeout(connection);
+    if (timeout <= 0) {
+      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(
@@ -390,37 +293,10 @@ public class TransferFsImage {
       throw new IOException(CONTENT_LENGTH + " header is not provided " +
                             "by the namenode when trying to fetch " + url);
     }
-    MD5Hash advertisedDigest = parseMD5Header(connection);
-    String fsImageName = connection
-        .getHeaderField(ImageServlet.HADOOP_IMAGE_EDITS_HEADER);
-    InputStream stream = connection.getInputStream();
-
-    return receiveFile(url.toExternalForm(), localPaths, dstStorage,
-        getChecksum, advertisedSize, advertisedDigest, fsImageName, stream,
-        null);
-  }
-
-  private static void setTimeout(HttpURLConnection connection) {
-    if (timeout <= 0) {
-      Configuration conf = new HdfsConfiguration();
-      timeout = conf.getInt(DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_KEY,
-          DFSConfigKeys.DFS_IMAGE_TRANSFER_TIMEOUT_DEFAULT);
-      LOG.info("Image Transfer timeout configured to " + timeout
-          + " milliseconds");
-    }
-
-    if (timeout > 0) {
-      connection.setConnectTimeout(timeout);
-      connection.setReadTimeout(timeout);
-    }
-  }
-
-  private static MD5Hash receiveFile(String url, List<File> localPaths,
-      Storage dstStorage, boolean getChecksum, long advertisedSize,
-      MD5Hash advertisedDigest, String fsImageName, InputStream stream,
-      DataTransferThrottler throttler) throws IOException {
-    long startTime = Time.monotonicNow();
+    
     if (localPaths != null) {
+      String fsImageName = connection.getHeaderField(
+          GetImageServlet.HADOOP_IMAGE_EDITS_HEADER);
       // If the local paths refer to directories, use the server-provided 
header
       // as the filename within that directory
       List<File> newLocalPaths = new ArrayList<File>();
@@ -437,8 +313,10 @@ public class TransferFsImage {
       localPaths = newLocalPaths;
     }
     
+    MD5Hash advertisedDigest = parseMD5Header(connection);
 
     long received = 0;
+    InputStream stream = connection.getInputStream();
     MessageDigest digester = null;
     if (getChecksum) {
       digester = MD5Hash.getDigester();
@@ -483,9 +361,6 @@ public class TransferFsImage {
           for (FileOutputStream fos : outputStreams) {
             fos.write(buf, 0, num);
           }
-          if (throttler != null) {
-            throttler.throttle(num);
-          }
         }
       }
       finishedReceiving = true;
@@ -529,12 +404,7 @@ public class TransferFsImage {
     String header = connection.getHeaderField(MD5_HEADER);
     return (header != null) ? new MD5Hash(header) : null;
   }
-
-  private static MD5Hash parseMD5Header(HttpServletRequest request) {
-    String header = request.getHeader(MD5_HEADER);
-    return (header != null) ? new MD5Hash(header) : null;
-  }
-
+  
   public static class HttpGetFailedException extends IOException {
     private static final long serialVersionUID = 1L;
     private final int responseCode;
@@ -549,18 +419,4 @@ public class TransferFsImage {
     }
   }
 
-  public static class HttpPutFailedException extends IOException {
-    private static final long serialVersionUID = 1L;
-    private final int responseCode;
-
-    HttpPutFailedException(String msg, int responseCode) throws IOException {
-      super(msg);
-      this.responseCode = responseCode;
-    }
-
-    public int getResponseCode() {
-      return responseCode;
-    }
-  }
-
 }

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ha/StandbyCheckpointer.java
 Tue Mar 11 00:40:46 2014
@@ -63,7 +63,6 @@ public class StandbyCheckpointer {
   private static final Log LOG = LogFactory.getLog(StandbyCheckpointer.class);
   private static final long PREVENT_AFTER_CANCEL_MS = 2*60*1000L;
   private final CheckpointConf checkpointConf;
-  private final Configuration conf;
   private final FSNamesystem namesystem;
   private long lastCheckpointTime;
   private final CheckpointerThread thread;
@@ -81,7 +80,6 @@ public class StandbyCheckpointer {
   public StandbyCheckpointer(Configuration conf, FSNamesystem ns)
       throws IOException {
     this.namesystem = ns;
-    this.conf = conf;
     this.checkpointConf = new CheckpointConf(conf); 
     this.thread = new CheckpointerThread();
     this.uploadThreadFactory = new ThreadFactoryBuilder().setDaemon(true)
@@ -195,7 +193,7 @@ public class StandbyCheckpointer {
     Future<Void> upload = executor.submit(new Callable<Void>() {
       @Override
       public Void call() throws IOException {
-        TransferFsImage.uploadImageFromStorage(activeNNAddress, conf,
+        TransferFsImage.uploadImageFromStorage(activeNNAddress, myNNAddress,
             namesystem.getFSImage().getStorage(), imageType, txid);
         return null;
       }

Propchange: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/native/
------------------------------------------------------------------------------
  Reverse-merged 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/native:r1575611

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
 Tue Mar 11 00:40:46 2014
@@ -858,13 +858,15 @@
 
 <property>
   <name>dfs.image.transfer.timeout</name>
-  <value>60000</value>
+  <value>600000</value>
   <description>
-        Socket timeout for image transfer in milliseconds. This timeout and 
the related
+        Timeout for image transfer in milliseconds. This timeout and the 
related
         dfs.image.transfer.bandwidthPerSec parameter should be configured such
-        that normal image transfer can complete successfully.
+        that normal image transfer can complete within the timeout.
         This timeout prevents client hangs when the sender fails during
-        image transfer. This is socket timeout during image tranfer.
+        image transfer, which is particularly important during checkpointing.
+        Note that this timeout applies to the entirety of image transfer, and
+        is not a socket timeout.
   </description>
 </property>
 
@@ -882,16 +884,6 @@
 </property>
 
 <property>
-  <name>dfs.image.transfer.chunksize</name>
-  <value>65536</value>
-  <description>
-        Chunksize in bytes to upload the checkpoint.
-        Chunked streaming is used to avoid internal buffering of contents
-        of image file of huge size.
-  </description>
-</property>
-
-<property>
   <name>dfs.namenode.support.allow.format</name>
   <value>true</value>
   <description>Does HDFS namenode allow itself to be formatted?

Propchange: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode/
------------------------------------------------------------------------------
  Reverse-merged 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/datanode:r1575611

Propchange: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs/
------------------------------------------------------------------------------
  Reverse-merged 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/hdfs:r1575611

Propchange: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary/
------------------------------------------------------------------------------
  Reverse-merged 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/main/webapps/secondary:r1575611

Propchange: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs/
------------------------------------------------------------------------------
  Reverse-merged 
/hadoop/common/trunk/hadoop-hdfs-project/hadoop-hdfs/src/test/hdfs:r1575611

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java
 Tue Mar 11 00:40:46 2014
@@ -31,7 +31,6 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.FilenameFilter;
 import java.io.IOException;
 import java.io.RandomAccessFile;
@@ -565,9 +564,23 @@ public class TestCheckpoint {
   }
 
   /**
-   * Simulate a secondary node failure to transfer image. Uses an unchecked
-   * error and fail transfer before even setting the length header. This used 
to
-   * cause image truncation. Regression test for HDFS-3330.
+   * Simulate a secondary node failure to transfer image
+   * back to the name-node.
+   * Used to truncate primary fsimage file.
+   */
+  @Test
+  public void testSecondaryFailsToReturnImage() throws IOException {
+    Mockito.doThrow(new IOException("If this exception is not caught by the " +
+        "name-node, fs image will be truncated."))
+        .when(faultInjector).aboutToSendFile(filePathContaining("secondary"));
+
+    doSecondaryFailsToReturnImage();
+  }
+  
+  /**
+   * Similar to above test, but uses an unchecked Error, and causes it
+   * before even setting the length header. This used to cause image
+   * truncation. Regression test for HDFS-3330.
    */
   @Test
   public void testSecondaryFailsWithErrorBeforeSettingHeaders()
@@ -1965,14 +1978,7 @@ public class TestCheckpoint {
       Mockito.doReturn(Lists.newArrayList(new File("/wont-be-written")))
         .when(dstImage).getFiles(
             Mockito.<NameNodeDirType>anyObject(), Mockito.anyString());
-
-      File mockImageFile = File.createTempFile("image", "");
-      FileOutputStream imageFile = new FileOutputStream(mockImageFile);
-      imageFile.write("data".getBytes());
-      imageFile.close();
-      Mockito.doReturn(mockImageFile).when(dstImage)
-          .findImageFile(Mockito.any(NameNodeFile.class), Mockito.anyLong());
-
+      
       Mockito.doReturn(new StorageInfo(1, 1, "X", 1, 
NodeType.NAME_NODE).toColonSeparatedString())
         .when(dstImage).toColonSeparatedString();
 
@@ -1993,8 +1999,8 @@ public class TestCheckpoint {
       }
 
       try {
-        TransferFsImage.uploadImageFromStorage(fsName, conf, dstImage,
-            NameNodeFile.IMAGE, 0);
+        TransferFsImage.uploadImageFromStorage(fsName, new URL(
+            "http://localhost:1234";), dstImage, NameNodeFile.IMAGE, 0);
         fail("Storage info was not verified");
       } catch (IOException ioe) {
         String msg = StringUtils.stringifyException(ioe);

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestGetImageServlet.java
 Tue Mar 11 00:40:46 2014
@@ -69,7 +69,7 @@ public class TestGetImageServlet {
     
Mockito.when(context.getAttribute(HttpServer2.ADMINS_ACL)).thenReturn(acls);
     
     // Make sure that NN2 is considered a valid fsimage/edits requestor.
-    assertTrue(ImageServlet.isValidRequestor(context,
+    assertTrue(GetImageServlet.isValidRequestor(context,
         "hdfs/ho...@test-realm.com", conf));
     
     // Mark atm as an admin.
@@ -81,15 +81,15 @@ public class TestGetImageServlet {
     }))).thenReturn(true);
     
     // Make sure that NN2 is still considered a valid requestor.
-    assertTrue(ImageServlet.isValidRequestor(context,
+    assertTrue(GetImageServlet.isValidRequestor(context,
         "hdfs/ho...@test-realm.com", conf));
     
     // Make sure an admin is considered a valid requestor.
-    assertTrue(ImageServlet.isValidRequestor(context,
+    assertTrue(GetImageServlet.isValidRequestor(context,
         "a...@test-realm.com", conf));
     
     // Make sure other users are *not* considered valid requestors.
-    assertFalse(ImageServlet.isValidRequestor(context,
+    assertFalse(GetImageServlet.isValidRequestor(context,
         "t...@test-realm.com", conf));
   }
 }

Modified: 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
URL: 
http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java?rev=1576155&r1=1576154&r2=1576155&view=diff
==============================================================================
--- 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
 (original)
+++ 
hadoop/common/branches/branch-2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestTransferFsImage.java
 Tue Mar 11 00:40:46 2014
@@ -22,7 +22,6 @@ import static org.junit.Assert.assertTru
 import static org.junit.Assert.fail;
 
 import java.io.File;
-import java.io.FileOutputStream;
 import java.io.IOException;
 import java.net.SocketTimeoutException;
 import java.net.URL;
@@ -35,11 +34,9 @@ import javax.servlet.http.HttpServletReq
 import javax.servlet.http.HttpServletResponse;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystemTestHelper;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
 import org.apache.hadoop.http.HttpServer2;
 import org.apache.hadoop.http.HttpServerFunctionalTest;
 import org.apache.hadoop.test.PathUtils;
@@ -121,11 +118,10 @@ public class TestTransferFsImage {
    * Test to verify the read timeout
    */
   @Test(timeout = 5000)
-  public void testGetImageTimeout() throws Exception {
+  public void testImageTransferTimeout() throws Exception {
     HttpServer2 testServer = HttpServerFunctionalTest.createServer("hdfs");
     try {
-      testServer.addServlet("ImageTransfer", ImageServlet.PATH_SPEC,
-          TestImageTransferServlet.class);
+      testServer.addServlet("GetImage", "/getimage", 
TestGetImageServlet.class);
       testServer.start();
       URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
       TransferFsImage.timeout = 2000;
@@ -143,48 +139,7 @@ public class TestTransferFsImage {
     }
   }
 
-  /**
-   * Test to verify the timeout of Image upload
-   */
-  @Test(timeout = 10000)
-  public void testImageUploadTimeout() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    NNStorage mockStorage = Mockito.mock(NNStorage.class);
-    HttpServer2 testServer = HttpServerFunctionalTest.createServer("hdfs");
-    try {
-      testServer.addServlet("ImageTransfer", ImageServlet.PATH_SPEC,
-          TestImageTransferServlet.class);
-      testServer.start();
-      URL serverURL = HttpServerFunctionalTest.getServerURL(testServer);
-      // set the timeout here, otherwise it will take default.
-      TransferFsImage.timeout = 2000;
-
-      File tmpDir = new File(new FileSystemTestHelper().getTestRootDir());
-      tmpDir.mkdirs();
-
-      File mockImageFile = File.createTempFile("image", "", tmpDir);
-      FileOutputStream imageFile = new FileOutputStream(mockImageFile);
-      imageFile.write("data".getBytes());
-      imageFile.close();
-      Mockito.when(
-          mockStorage.findImageFile(Mockito.any(NameNodeFile.class),
-              Mockito.anyLong())).thenReturn(mockImageFile);
-      Mockito.when(mockStorage.toColonSeparatedString()).thenReturn(
-          "storage:info:string");
-      
-      try {
-        TransferFsImage.uploadImageFromStorage(serverURL, conf, mockStorage,
-            NameNodeFile.IMAGE, 1L);
-        fail("TransferImage Should fail with timeout");
-      } catch (SocketTimeoutException e) {
-        assertEquals("Upload should timeout", "Read timed out", 
e.getMessage());
-      }
-    } finally {
-      testServer.stop();
-    }
-  }
-
-  public static class TestImageTransferServlet extends HttpServlet {
+  public static class TestGetImageServlet extends HttpServlet {
     private static final long serialVersionUID = 1L;
 
     @Override
@@ -198,17 +153,5 @@ public class TestTransferFsImage {
         }
       }
     }
-
-    @Override
-    protected void doPut(HttpServletRequest req, HttpServletResponse resp)
-        throws ServletException, IOException {
-      synchronized (this) {
-        try {
-          wait(5000);
-        } catch (InterruptedException e) {
-          // Ignore
-        }
-      }
-    }
   }
 }


Reply via email to