HADOOP-11614. Remove httpclient dependency from hadoop-openstack. Contributed by Akira Ajisaka, Brahma Reddy Battula, and Steve Loughran.
This commit is contained in:
parent
249cb210f5
commit
62579b69a0
|
@ -124,10 +124,8 @@
|
||||||
</dependency>
|
</dependency>
|
||||||
|
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>commons-httpclient</groupId>
|
<groupId>org.apache.httpcomponents</groupId>
|
||||||
<artifactId>commons-httpclient</artifactId>
|
<artifactId>httpcore</artifactId>
|
||||||
<version>3.1</version>
|
|
||||||
<scope>compile</scope>
|
|
||||||
</dependency>
|
</dependency>
|
||||||
<dependency>
|
<dependency>
|
||||||
<groupId>commons-logging</groupId>
|
<groupId>commons-logging</groupId>
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.exceptions;
|
package org.apache.hadoop.fs.swift.exceptions;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.HttpMethod;
|
import org.apache.http.HttpResponse;
|
||||||
|
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
|
||||||
|
@ -37,8 +37,8 @@ public class SwiftAuthenticationFailedException extends SwiftInvalidResponseExce
|
||||||
public SwiftAuthenticationFailedException(String message,
|
public SwiftAuthenticationFailedException(String message,
|
||||||
String operation,
|
String operation,
|
||||||
URI uri,
|
URI uri,
|
||||||
HttpMethod method) {
|
HttpResponse resp) {
|
||||||
super(message, operation, uri, method);
|
super(message, operation, uri, resp);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.exceptions;
|
package org.apache.hadoop.fs.swift.exceptions;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.HttpMethod;
|
import org.apache.http.HttpResponse;
|
||||||
|
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
|
||||||
|
@ -31,8 +31,8 @@ public class SwiftBadRequestException extends SwiftInvalidResponseException {
|
||||||
public SwiftBadRequestException(String message,
|
public SwiftBadRequestException(String message,
|
||||||
String operation,
|
String operation,
|
||||||
URI uri,
|
URI uri,
|
||||||
HttpMethod method) {
|
HttpResponse resp) {
|
||||||
super(message, operation, uri, method);
|
super(message, operation, uri, resp);
|
||||||
}
|
}
|
||||||
|
|
||||||
public SwiftBadRequestException(String message,
|
public SwiftBadRequestException(String message,
|
||||||
|
|
|
@ -18,7 +18,8 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.exceptions;
|
package org.apache.hadoop.fs.swift.exceptions;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.HttpMethod;
|
import org.apache.hadoop.fs.swift.util.HttpResponseUtils;
|
||||||
|
import org.apache.http.HttpResponse;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
@ -48,14 +49,14 @@ public class SwiftInvalidResponseException extends SwiftConnectionException {
|
||||||
public SwiftInvalidResponseException(String message,
|
public SwiftInvalidResponseException(String message,
|
||||||
String operation,
|
String operation,
|
||||||
URI uri,
|
URI uri,
|
||||||
HttpMethod method) {
|
HttpResponse resp) {
|
||||||
super(message);
|
super(message);
|
||||||
this.statusCode = method.getStatusCode();
|
this.statusCode = resp.getStatusLine().getStatusCode();
|
||||||
this.operation = operation;
|
this.operation = operation;
|
||||||
this.uri = uri;
|
this.uri = uri;
|
||||||
String bodyAsString;
|
String bodyAsString;
|
||||||
try {
|
try {
|
||||||
bodyAsString = method.getResponseBodyAsString();
|
bodyAsString = HttpResponseUtils.getResponseBodyAsString(resp);
|
||||||
if (bodyAsString == null) {
|
if (bodyAsString == null) {
|
||||||
bodyAsString = "";
|
bodyAsString = "";
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,7 +18,7 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.exceptions;
|
package org.apache.hadoop.fs.swift.exceptions;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.HttpMethod;
|
import org.apache.http.HttpResponse;
|
||||||
|
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
|
|
||||||
|
@ -31,7 +31,7 @@ public class SwiftThrottledRequestException extends
|
||||||
public SwiftThrottledRequestException(String message,
|
public SwiftThrottledRequestException(String message,
|
||||||
String operation,
|
String operation,
|
||||||
URI uri,
|
URI uri,
|
||||||
HttpMethod method) {
|
HttpResponse resp) {
|
||||||
super(message, operation, uri, method);
|
super(message, operation, uri, resp);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -18,24 +18,24 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.http;
|
package org.apache.hadoop.fs.swift.http;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.methods.EntityEnclosingMethod;
|
import org.apache.http.client.methods.HttpEntityEnclosingRequestBase;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation for SwiftRestClient to make copy requests.
|
* Implementation for SwiftRestClient to make copy requests.
|
||||||
* COPY is a method that came with WebDAV (RFC2518), and is not something that
|
* COPY is a method that came with WebDAV (RFC2518), and is not something that
|
||||||
* can be handled by all proxies en-route to a filesystem.
|
* can be handled by all proxies en-route to a filesystem.
|
||||||
*/
|
*/
|
||||||
class CopyMethod extends EntityEnclosingMethod {
|
class CopyRequest extends HttpEntityEnclosingRequestBase {
|
||||||
|
|
||||||
public CopyMethod(String uri) {
|
CopyRequest() {
|
||||||
super(uri);
|
super();
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* @return http method name
|
* @return http method name
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public String getName() {
|
public String getMethod() {
|
||||||
return "COPY";
|
return "COPY";
|
||||||
}
|
}
|
||||||
}
|
}
|
|
@ -18,11 +18,12 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.http;
|
package org.apache.hadoop.fs.swift.http;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.HttpMethod;
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException;
|
import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException;
|
||||||
import org.apache.hadoop.fs.swift.util.SwiftUtils;
|
import org.apache.hadoop.fs.swift.util.SwiftUtils;
|
||||||
|
import org.apache.http.HttpResponse;
|
||||||
|
import org.apache.http.client.methods.HttpRequestBase;
|
||||||
|
|
||||||
import java.io.ByteArrayInputStream;
|
import java.io.ByteArrayInputStream;
|
||||||
import java.io.EOFException;
|
import java.io.EOFException;
|
||||||
|
@ -46,7 +47,8 @@ public class HttpInputStreamWithRelease extends InputStream {
|
||||||
private static final Log LOG =
|
private static final Log LOG =
|
||||||
LogFactory.getLog(HttpInputStreamWithRelease.class);
|
LogFactory.getLog(HttpInputStreamWithRelease.class);
|
||||||
private final URI uri;
|
private final URI uri;
|
||||||
private HttpMethod method;
|
private HttpRequestBase req;
|
||||||
|
private HttpResponse resp;
|
||||||
//flag to say the stream is released -volatile so that read operations
|
//flag to say the stream is released -volatile so that read operations
|
||||||
//pick it up even while unsynchronized.
|
//pick it up even while unsynchronized.
|
||||||
private volatile boolean released;
|
private volatile boolean released;
|
||||||
|
@ -64,16 +66,17 @@ public class HttpInputStreamWithRelease extends InputStream {
|
||||||
*/
|
*/
|
||||||
private String reasonClosed = "unopened";
|
private String reasonClosed = "unopened";
|
||||||
|
|
||||||
public HttpInputStreamWithRelease(URI uri, HttpMethod method) throws
|
public HttpInputStreamWithRelease(URI uri, HttpRequestBase req,
|
||||||
IOException {
|
HttpResponse resp) throws IOException {
|
||||||
this.uri = uri;
|
this.uri = uri;
|
||||||
this.method = method;
|
this.req = req;
|
||||||
|
this.resp = resp;
|
||||||
constructionStack = LOG.isDebugEnabled() ? new Exception("stack") : null;
|
constructionStack = LOG.isDebugEnabled() ? new Exception("stack") : null;
|
||||||
if (method == null) {
|
if (req == null) {
|
||||||
throw new IllegalArgumentException("Null 'method' parameter ");
|
throw new IllegalArgumentException("Null 'request' parameter ");
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
inStream = method.getResponseBodyAsStream();
|
inStream = resp.getEntity().getContent();
|
||||||
} catch (IOException e) {
|
} catch (IOException e) {
|
||||||
inStream = new ByteArrayInputStream(new byte[]{});
|
inStream = new ByteArrayInputStream(new byte[]{});
|
||||||
throw releaseAndRethrow("getResponseBodyAsStream() in constructor -" + e, e);
|
throw releaseAndRethrow("getResponseBodyAsStream() in constructor -" + e, e);
|
||||||
|
@ -100,11 +103,11 @@ public class HttpInputStreamWithRelease extends InputStream {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug("Releasing connection to " + uri + ": " + reason, ex);
|
LOG.debug("Releasing connection to " + uri + ": " + reason, ex);
|
||||||
}
|
}
|
||||||
if (method != null) {
|
if (req != null) {
|
||||||
if (!dataConsumed) {
|
if (!dataConsumed) {
|
||||||
method.abort();
|
req.abort();
|
||||||
}
|
}
|
||||||
method.releaseConnection();
|
req.releaseConnection();
|
||||||
}
|
}
|
||||||
if (inStream != null) {
|
if (inStream != null) {
|
||||||
//this guard may seem un-needed, but a stack trace seen
|
//this guard may seem un-needed, but a stack trace seen
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -19,8 +19,9 @@ package org.apache.hadoop.fs.swift.snative;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.type.CollectionType;
|
import com.fasterxml.jackson.databind.type.CollectionType;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.Header;
|
import org.apache.http.Header;
|
||||||
import org.apache.commons.httpclient.HttpStatus;
|
import org.apache.http.HttpStatus;
|
||||||
|
import org.apache.http.message.BasicHeader;
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -166,9 +167,9 @@ public class SwiftNativeFileSystemStore {
|
||||||
}
|
}
|
||||||
|
|
||||||
swiftRestClient.upload(toObjectPath(path),
|
swiftRestClient.upload(toObjectPath(path),
|
||||||
new ByteArrayInputStream(new byte[0]),
|
new ByteArrayInputStream(new byte[0]),
|
||||||
0,
|
0,
|
||||||
new Header(SwiftProtocolConstants.X_OBJECT_MANIFEST, pathString));
|
new BasicHeader(SwiftProtocolConstants.X_OBJECT_MANIFEST, pathString));
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -0,0 +1,121 @@
|
||||||
|
/**
|
||||||
|
* 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.fs.swift.util;
|
||||||
|
|
||||||
|
import java.io.ByteArrayOutputStream;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStream;
|
||||||
|
|
||||||
|
import org.apache.http.Header;
|
||||||
|
import org.apache.http.HttpResponse;
|
||||||
|
import org.apache.http.util.EncodingUtils;
|
||||||
|
|
||||||
|
import static org.apache.hadoop.fs.swift.http.SwiftProtocolConstants.HEADER_CONTENT_LENGTH;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Utility class for parsing HttpResponse. This class is implemented like
|
||||||
|
* {@code org.apache.commons.httpclient.HttpMethodBase.java} in httpclient 3.x.
|
||||||
|
*/
|
||||||
|
public abstract class HttpResponseUtils {
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the response body of the HTTPResponse, if any, as an array of bytes.
|
||||||
|
* If response body is not available or cannot be read, returns <tt>null</tt>
|
||||||
|
*
|
||||||
|
* Note: This will cause the entire response body to be buffered in memory. A
|
||||||
|
* malicious server may easily exhaust all the VM memory. It is strongly
|
||||||
|
* recommended, to use getResponseAsStream if the content length of the
|
||||||
|
* response is unknown or reasonably large.
|
||||||
|
*
|
||||||
|
* @param resp HttpResponse
|
||||||
|
* @return The response body
|
||||||
|
* @throws IOException If an I/O (transport) problem occurs while obtaining
|
||||||
|
* the response body.
|
||||||
|
*/
|
||||||
|
public static byte[] getResponseBody(HttpResponse resp) throws IOException {
|
||||||
|
try(InputStream instream = resp.getEntity().getContent()) {
|
||||||
|
if (instream != null) {
|
||||||
|
long contentLength = resp.getEntity().getContentLength();
|
||||||
|
if (contentLength > Integer.MAX_VALUE) {
|
||||||
|
//guard integer cast from overflow
|
||||||
|
throw new IOException("Content too large to be buffered: "
|
||||||
|
+ contentLength +" bytes");
|
||||||
|
}
|
||||||
|
ByteArrayOutputStream outstream = new ByteArrayOutputStream(
|
||||||
|
contentLength > 0 ? (int) contentLength : 4*1024);
|
||||||
|
byte[] buffer = new byte[4096];
|
||||||
|
int len;
|
||||||
|
while ((len = instream.read(buffer)) > 0) {
|
||||||
|
outstream.write(buffer, 0, len);
|
||||||
|
}
|
||||||
|
outstream.close();
|
||||||
|
return outstream.toByteArray();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Returns the response body of the HTTPResponse, if any, as a {@link String}.
|
||||||
|
* If response body is not available or cannot be read, returns <tt>null</tt>
|
||||||
|
* The string conversion on the data is done using UTF-8.
|
||||||
|
*
|
||||||
|
* Note: This will cause the entire response body to be buffered in memory. A
|
||||||
|
* malicious server may easily exhaust all the VM memory. It is strongly
|
||||||
|
* recommended, to use getResponseAsStream if the content length of the
|
||||||
|
* response is unknown or reasonably large.
|
||||||
|
*
|
||||||
|
* @param resp HttpResponse
|
||||||
|
* @return The response body.
|
||||||
|
* @throws IOException If an I/O (transport) problem occurs while obtaining
|
||||||
|
* the response body.
|
||||||
|
*/
|
||||||
|
public static String getResponseBodyAsString(HttpResponse resp)
|
||||||
|
throws IOException {
|
||||||
|
byte[] rawdata = getResponseBody(resp);
|
||||||
|
if (rawdata != null) {
|
||||||
|
return EncodingUtils.getString(rawdata, "UTF-8");
|
||||||
|
} else {
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Return the length (in bytes) of the response body, as specified in a
|
||||||
|
* <tt>Content-Length</tt> header.
|
||||||
|
*
|
||||||
|
* <p>
|
||||||
|
* Return <tt>-1</tt> when the content-length is unknown.
|
||||||
|
* </p>
|
||||||
|
*
|
||||||
|
* @param resp HttpResponse
|
||||||
|
* @return content length, if <tt>Content-Length</tt> header is available.
|
||||||
|
* <tt>0</tt> indicates that the request has no body.
|
||||||
|
* If <tt>Content-Length</tt> header is not present, the method
|
||||||
|
* returns <tt>-1</tt>.
|
||||||
|
*/
|
||||||
|
public static long getContentLength(HttpResponse resp) {
|
||||||
|
Header header = resp.getFirstHeader(HEADER_CONTENT_LENGTH);
|
||||||
|
if (header == null) {
|
||||||
|
return -1;
|
||||||
|
} else {
|
||||||
|
return Long.parseLong(header.getValue());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -17,7 +17,6 @@
|
||||||
*/
|
*/
|
||||||
package org.apache.hadoop.fs.swift;
|
package org.apache.hadoop.fs.swift;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.Header;
|
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
|
@ -28,6 +27,7 @@ import org.apache.hadoop.fs.swift.snative.SwiftNativeFileSystem;
|
||||||
import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
|
import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
|
||||||
import org.apache.hadoop.fs.swift.util.SwiftUtils;
|
import org.apache.hadoop.fs.swift.util.SwiftUtils;
|
||||||
import org.apache.hadoop.io.IOUtils;
|
import org.apache.hadoop.io.IOUtils;
|
||||||
|
import org.apache.http.Header;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.internal.AssumptionViolatedException;
|
import org.junit.internal.AssumptionViolatedException;
|
||||||
|
|
||||||
|
|
|
@ -18,7 +18,6 @@
|
||||||
|
|
||||||
package org.apache.hadoop.fs.swift.http;
|
package org.apache.hadoop.fs.swift.http;
|
||||||
|
|
||||||
import org.apache.commons.httpclient.Header;
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
@ -28,6 +27,7 @@ import org.apache.hadoop.fs.swift.util.Duration;
|
||||||
import org.apache.hadoop.fs.swift.util.DurationStats;
|
import org.apache.hadoop.fs.swift.util.DurationStats;
|
||||||
import org.apache.hadoop.fs.swift.util.SwiftObjectPath;
|
import org.apache.hadoop.fs.swift.util.SwiftObjectPath;
|
||||||
import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
|
import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
|
||||||
|
import org.apache.http.Header;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Assume;
|
import org.junit.Assume;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
|
|
@ -37,6 +37,3 @@ log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
|
||||||
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
|
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n
|
||||||
#log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c %x - %m%n"
|
#log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c %x - %m%n"
|
||||||
#log4j.logger.org.apache.hadoop.fs.swift=DEBUG
|
#log4j.logger.org.apache.hadoop.fs.swift=DEBUG
|
||||||
|
|
||||||
#crank back on warnings about -1 content length GETs
|
|
||||||
log4j.logger.org.apache.commons.httpclient.HttpMethodBase=ERROR
|
|
||||||
|
|
Loading…
Reference in New Issue