|
@@ -19,18 +19,22 @@ package org.apache.hadoop.hdfs.server.namenode;
|
|
|
|
|
|
import java.io.File;
|
|
import java.io.File;
|
|
import java.io.FileInputStream;
|
|
import java.io.FileInputStream;
|
|
|
|
+import java.io.FileNotFoundException;
|
|
import java.io.FileOutputStream;
|
|
import java.io.FileOutputStream;
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.io.InputStream;
|
|
import java.io.InputStream;
|
|
|
|
+import java.io.OutputStream;
|
|
import java.net.HttpURLConnection;
|
|
import java.net.HttpURLConnection;
|
|
|
|
+import java.net.URISyntaxException;
|
|
import java.net.URL;
|
|
import java.net.URL;
|
|
import java.security.DigestInputStream;
|
|
import java.security.DigestInputStream;
|
|
import java.security.MessageDigest;
|
|
import java.security.MessageDigest;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
|
|
+import java.util.Map;
|
|
|
|
+import java.util.Map.Entry;
|
|
|
|
|
|
-import javax.servlet.ServletOutputStream;
|
|
|
|
-import javax.servlet.ServletResponse;
|
|
|
|
|
|
+import javax.servlet.http.HttpServletRequest;
|
|
import javax.servlet.http.HttpServletResponse;
|
|
import javax.servlet.http.HttpServletResponse;
|
|
|
|
|
|
import org.apache.commons.logging.Log;
|
|
import org.apache.commons.logging.Log;
|
|
@@ -49,10 +53,12 @@ import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeFile;
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
|
import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog;
|
|
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
|
|
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
|
import org.apache.hadoop.hdfs.web.URLConnectionFactory;
|
|
|
|
+import org.apache.hadoop.io.IOUtils;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.io.MD5Hash;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
|
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
|
import org.apache.hadoop.util.Time;
|
|
import org.apache.hadoop.util.Time;
|
|
|
|
+import org.apache.http.client.utils.URIBuilder;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
import com.google.common.collect.Lists;
|
|
import com.google.common.collect.Lists;
|
|
@@ -65,7 +71,12 @@ import com.google.common.collect.Lists;
|
|
public class TransferFsImage {
|
|
public class TransferFsImage {
|
|
|
|
|
|
public final static String CONTENT_LENGTH = "Content-Length";
|
|
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";
|
|
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
|
|
@VisibleForTesting
|
|
static int timeout = 0;
|
|
static int timeout = 0;
|
|
private static URLConnectionFactory connectionFactory;
|
|
private static URLConnectionFactory connectionFactory;
|
|
@@ -82,14 +93,14 @@ public class TransferFsImage {
|
|
|
|
|
|
public static void downloadMostRecentImageToDirectory(URL infoServer,
|
|
public static void downloadMostRecentImageToDirectory(URL infoServer,
|
|
File dir) throws IOException {
|
|
File dir) throws IOException {
|
|
- String fileId = GetImageServlet.getParamStringForMostRecentImage();
|
|
|
|
|
|
+ String fileId = ImageServlet.getParamStringForMostRecentImage();
|
|
getFileClient(infoServer, fileId, Lists.newArrayList(dir),
|
|
getFileClient(infoServer, fileId, Lists.newArrayList(dir),
|
|
null, false);
|
|
null, false);
|
|
}
|
|
}
|
|
|
|
|
|
public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
|
|
public static MD5Hash downloadImageToStorage(URL fsName, long imageTxId,
|
|
Storage dstStorage, boolean needDigest) throws IOException {
|
|
Storage dstStorage, boolean needDigest) throws IOException {
|
|
- String fileid = GetImageServlet.getParamStringForImage(null,
|
|
|
|
|
|
+ String fileid = ImageServlet.getParamStringForImage(null,
|
|
imageTxId, dstStorage);
|
|
imageTxId, dstStorage);
|
|
String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
|
|
String fileName = NNStorage.getCheckpointImageFileName(imageTxId);
|
|
|
|
|
|
@@ -104,12 +115,31 @@ public class TransferFsImage {
|
|
dstFiles.get(0).length() + " bytes.");
|
|
dstFiles.get(0).length() + " bytes.");
|
|
return hash;
|
|
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,
|
|
static void downloadEditsToStorage(URL fsName, RemoteEditLog log,
|
|
NNStorage dstStorage) throws IOException {
|
|
NNStorage dstStorage) throws IOException {
|
|
assert log.getStartTxId() > 0 && log.getEndTxId() > 0 :
|
|
assert log.getStartTxId() > 0 && log.getEndTxId() > 0 :
|
|
"bad log: " + log;
|
|
"bad log: " + log;
|
|
- String fileid = GetImageServlet.getParamStringForLog(
|
|
|
|
|
|
+ String fileid = ImageServlet.getParamStringForLog(
|
|
log, dstStorage);
|
|
log, dstStorage);
|
|
String finalFileName = NNStorage.getFinalizedEditsFileName(
|
|
String finalFileName = NNStorage.getFinalizedEditsFileName(
|
|
log.getStartTxId(), log.getEndTxId());
|
|
log.getStartTxId(), log.getEndTxId());
|
|
@@ -159,22 +189,19 @@ public class TransferFsImage {
|
|
* Requests that the NameNode download an image from this node.
|
|
* Requests that the NameNode download an image from this node.
|
|
*
|
|
*
|
|
* @param fsName the http address for the remote NN
|
|
* @param fsName the http address for the remote NN
|
|
- * @param myNNAddress the host/port where the local node is running an
|
|
|
|
- * HTTPServer hosting GetImageServlet
|
|
|
|
|
|
+ * @param conf Configuration
|
|
* @param storage the storage directory to transfer the image from
|
|
* @param storage the storage directory to transfer the image from
|
|
* @param nnf the NameNodeFile type of the image
|
|
* @param nnf the NameNodeFile type of the image
|
|
* @param txid the transaction ID of the image to be uploaded
|
|
* @param txid the transaction ID of the image to be uploaded
|
|
*/
|
|
*/
|
|
- public static void uploadImageFromStorage(URL fsName, URL myNNAddress,
|
|
|
|
- Storage storage, NameNodeFile nnf, long txid) throws IOException {
|
|
|
|
|
|
+ public static void uploadImageFromStorage(URL fsName, Configuration conf,
|
|
|
|
+ NNStorage storage, NameNodeFile nnf, long txid) throws IOException {
|
|
|
|
|
|
- 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.
|
|
|
|
|
|
+ URL url = new URL(fsName, ImageServlet.PATH_SPEC);
|
|
|
|
+ long startTime = Time.monotonicNow();
|
|
try {
|
|
try {
|
|
- TransferFsImage.getFileClient(fsName, fileid, null, null, false);
|
|
|
|
- } catch (HttpGetFailedException e) {
|
|
|
|
|
|
+ uploadImage(url, conf, storage, nnf, txid);
|
|
|
|
+ } catch (HttpPutFailedException e) {
|
|
if (e.getResponseCode() == HttpServletResponse.SC_CONFLICT) {
|
|
if (e.getResponseCode() == HttpServletResponse.SC_CONFLICT) {
|
|
// this is OK - this means that a previous attempt to upload
|
|
// this is OK - this means that a previous attempt to upload
|
|
// this checkpoint succeeded even though we thought it failed.
|
|
// this checkpoint succeeded even though we thought it failed.
|
|
@@ -186,25 +213,105 @@ public class TransferFsImage {
|
|
throw e;
|
|
throw e;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- LOG.info("Uploaded image with txid " + txid + " to namenode at " +
|
|
|
|
- fsName);
|
|
|
|
|
|
+ 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);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
|
|
|
|
-
|
|
|
|
/**
|
|
/**
|
|
* A server-side method to respond to a getfile http request
|
|
* A server-side method to respond to a getfile http request
|
|
* Copies the contents of the local file into the output stream.
|
|
* Copies the contents of the local file into the output stream.
|
|
*/
|
|
*/
|
|
- public static void getFileServer(ServletResponse response, File localfile,
|
|
|
|
- FileInputStream infile,
|
|
|
|
- DataTransferThrottler throttler)
|
|
|
|
|
|
+ public static void copyFileToStream(OutputStream out, File localfile,
|
|
|
|
+ FileInputStream infile, DataTransferThrottler throttler)
|
|
throws IOException {
|
|
throws IOException {
|
|
byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
|
|
byte buf[] = new byte[HdfsConstants.IO_FILE_BUFFER_SIZE];
|
|
- ServletOutputStream out = null;
|
|
|
|
try {
|
|
try {
|
|
CheckpointFaultInjector.getInstance()
|
|
CheckpointFaultInjector.getInstance()
|
|
.aboutToSendFile(localfile);
|
|
.aboutToSendFile(localfile);
|
|
- out = response.getOutputStream();
|
|
|
|
|
|
|
|
if (CheckpointFaultInjector.getInstance().
|
|
if (CheckpointFaultInjector.getInstance().
|
|
shouldSendShortFile(localfile)) {
|
|
shouldSendShortFile(localfile)) {
|
|
@@ -250,14 +357,13 @@ public class TransferFsImage {
|
|
static MD5Hash getFileClient(URL infoServer,
|
|
static MD5Hash getFileClient(URL infoServer,
|
|
String queryString, List<File> localPaths,
|
|
String queryString, List<File> localPaths,
|
|
Storage dstStorage, boolean getChecksum) throws IOException {
|
|
Storage dstStorage, boolean getChecksum) throws IOException {
|
|
- URL url = new URL(infoServer, "/getimage?" + queryString);
|
|
|
|
|
|
+ URL url = new URL(infoServer, ImageServlet.PATH_SPEC + "?" + queryString);
|
|
LOG.info("Opening connection to " + url);
|
|
LOG.info("Opening connection to " + url);
|
|
return doGetUrl(url, localPaths, dstStorage, getChecksum);
|
|
return doGetUrl(url, localPaths, dstStorage, getChecksum);
|
|
}
|
|
}
|
|
|
|
|
|
public static MD5Hash doGetUrl(URL url, List<File> localPaths,
|
|
public static MD5Hash doGetUrl(URL url, List<File> localPaths,
|
|
Storage dstStorage, boolean getChecksum) throws IOException {
|
|
Storage dstStorage, boolean getChecksum) throws IOException {
|
|
- long startTime = Time.monotonicNow();
|
|
|
|
HttpURLConnection connection;
|
|
HttpURLConnection connection;
|
|
try {
|
|
try {
|
|
connection = (HttpURLConnection)
|
|
connection = (HttpURLConnection)
|
|
@@ -266,16 +372,7 @@ public class TransferFsImage {
|
|
throw new IOException(e);
|
|
throw new IOException(e);
|
|
}
|
|
}
|
|
|
|
|
|
- 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);
|
|
|
|
- }
|
|
|
|
|
|
+ setTimeout(connection);
|
|
|
|
|
|
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
|
|
if (connection.getResponseCode() != HttpURLConnection.HTTP_OK) {
|
|
throw new HttpGetFailedException(
|
|
throw new HttpGetFailedException(
|
|
@@ -293,10 +390,37 @@ public class TransferFsImage {
|
|
throw new IOException(CONTENT_LENGTH + " header is not provided " +
|
|
throw new IOException(CONTENT_LENGTH + " header is not provided " +
|
|
"by the namenode when trying to fetch " + url);
|
|
"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) {
|
|
if (localPaths != null) {
|
|
- String fsImageName = connection.getHeaderField(
|
|
|
|
- GetImageServlet.HADOOP_IMAGE_EDITS_HEADER);
|
|
|
|
// If the local paths refer to directories, use the server-provided header
|
|
// If the local paths refer to directories, use the server-provided header
|
|
// as the filename within that directory
|
|
// as the filename within that directory
|
|
List<File> newLocalPaths = new ArrayList<File>();
|
|
List<File> newLocalPaths = new ArrayList<File>();
|
|
@@ -313,10 +437,8 @@ public class TransferFsImage {
|
|
localPaths = newLocalPaths;
|
|
localPaths = newLocalPaths;
|
|
}
|
|
}
|
|
|
|
|
|
- MD5Hash advertisedDigest = parseMD5Header(connection);
|
|
|
|
|
|
|
|
long received = 0;
|
|
long received = 0;
|
|
- InputStream stream = connection.getInputStream();
|
|
|
|
MessageDigest digester = null;
|
|
MessageDigest digester = null;
|
|
if (getChecksum) {
|
|
if (getChecksum) {
|
|
digester = MD5Hash.getDigester();
|
|
digester = MD5Hash.getDigester();
|
|
@@ -361,6 +483,9 @@ public class TransferFsImage {
|
|
for (FileOutputStream fos : outputStreams) {
|
|
for (FileOutputStream fos : outputStreams) {
|
|
fos.write(buf, 0, num);
|
|
fos.write(buf, 0, num);
|
|
}
|
|
}
|
|
|
|
+ if (throttler != null) {
|
|
|
|
+ throttler.throttle(num);
|
|
|
|
+ }
|
|
}
|
|
}
|
|
}
|
|
}
|
|
finishedReceiving = true;
|
|
finishedReceiving = true;
|
|
@@ -404,7 +529,12 @@ public class TransferFsImage {
|
|
String header = connection.getHeaderField(MD5_HEADER);
|
|
String header = connection.getHeaderField(MD5_HEADER);
|
|
return (header != null) ? new MD5Hash(header) : null;
|
|
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 {
|
|
public static class HttpGetFailedException extends IOException {
|
|
private static final long serialVersionUID = 1L;
|
|
private static final long serialVersionUID = 1L;
|
|
private final int responseCode;
|
|
private final int responseCode;
|
|
@@ -419,4 +549,18 @@ 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;
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
}
|
|
}
|