Sfoglia il codice sorgente

HADOOP-11614. Remove httpclient dependency from hadoop-openstack. Contributed by Akira Ajisaka, Brahma Reddy Battula, and Steve Loughran.

Akira Ajisaka 8 anni fa
parent
commit
62579b69a0

+ 2 - 4
hadoop-tools/hadoop-openstack/pom.xml

@@ -124,10 +124,8 @@
     </dependency>
 
     <dependency>
-      <groupId>commons-httpclient</groupId>
-      <artifactId>commons-httpclient</artifactId>
-      <version>3.1</version>
-      <scope>compile</scope>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
     </dependency>
     <dependency>
       <groupId>commons-logging</groupId>

+ 3 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/exceptions/SwiftAuthenticationFailedException.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.swift.exceptions;
 
-import org.apache.commons.httpclient.HttpMethod;
+import org.apache.http.HttpResponse;
 
 import java.net.URI;
 
@@ -37,8 +37,8 @@ public class SwiftAuthenticationFailedException extends SwiftInvalidResponseExce
   public SwiftAuthenticationFailedException(String message,
                                             String operation,
                                             URI uri,
-                                            HttpMethod method) {
-    super(message, operation, uri, method);
+                                            HttpResponse resp) {
+    super(message, operation, uri, resp);
   }
 
   @Override

+ 3 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/exceptions/SwiftBadRequestException.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.swift.exceptions;
 
-import org.apache.commons.httpclient.HttpMethod;
+import org.apache.http.HttpResponse;
 
 import java.net.URI;
 
@@ -31,8 +31,8 @@ public class SwiftBadRequestException extends SwiftInvalidResponseException {
   public SwiftBadRequestException(String message,
                                   String operation,
                                   URI uri,
-                                  HttpMethod method) {
-    super(message, operation, uri, method);
+                                  HttpResponse resp) {
+    super(message, operation, uri, resp);
   }
 
   public SwiftBadRequestException(String message,

+ 5 - 4
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/exceptions/SwiftInvalidResponseException.java

@@ -18,7 +18,8 @@
 
 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.net.URI;
@@ -48,14 +49,14 @@ public class SwiftInvalidResponseException extends SwiftConnectionException {
   public SwiftInvalidResponseException(String message,
                                        String operation,
                                        URI uri,
-                                       HttpMethod method) {
+                                       HttpResponse resp) {
     super(message);
-    this.statusCode = method.getStatusCode();
+    this.statusCode = resp.getStatusLine().getStatusCode();
     this.operation = operation;
     this.uri = uri;
     String bodyAsString;
     try {
-      bodyAsString = method.getResponseBodyAsString();
+      bodyAsString = HttpResponseUtils.getResponseBodyAsString(resp);
       if (bodyAsString == null) {
         bodyAsString = "";
       }

+ 3 - 3
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/exceptions/SwiftThrottledRequestException.java

@@ -18,7 +18,7 @@
 
 package org.apache.hadoop.fs.swift.exceptions;
 
-import org.apache.commons.httpclient.HttpMethod;
+import org.apache.http.HttpResponse;
 
 import java.net.URI;
 
@@ -31,7 +31,7 @@ public class SwiftThrottledRequestException extends
   public SwiftThrottledRequestException(String message,
                                         String operation,
                                         URI uri,
-                                        HttpMethod method) {
-    super(message, operation, uri, method);
+                                        HttpResponse resp) {
+    super(message, operation, uri, resp);
   }
 }

+ 6 - 6
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/CopyMethod.java → hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/CopyRequest.java

@@ -18,24 +18,24 @@
 
 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.
  * 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.
  */
-class CopyMethod extends EntityEnclosingMethod {
+class CopyRequest extends HttpEntityEnclosingRequestBase {
 
-  public CopyMethod(String uri) {
-    super(uri);
+  CopyRequest() {
+    super();
   }
 
   /**
    * @return http method name
    */
   @Override
-  public String getName() {
+  public String getMethod() {
     return "COPY";
   }
-}
+}

+ 14 - 11
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/HttpInputStreamWithRelease.java

@@ -18,11 +18,12 @@
 
 package org.apache.hadoop.fs.swift.http;
 
-import org.apache.commons.httpclient.HttpMethod;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.swift.exceptions.SwiftConnectionClosedException;
 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.EOFException;
@@ -46,7 +47,8 @@ public class HttpInputStreamWithRelease extends InputStream {
   private static final Log LOG =
     LogFactory.getLog(HttpInputStreamWithRelease.class);
   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
   //pick it up even while unsynchronized.
   private volatile boolean released;
@@ -64,16 +66,17 @@ public class HttpInputStreamWithRelease extends InputStream {
    */
   private String reasonClosed = "unopened";
 
-  public HttpInputStreamWithRelease(URI uri, HttpMethod method) throws
-                                                                IOException {
+  public HttpInputStreamWithRelease(URI uri, HttpRequestBase req,
+      HttpResponse resp) throws IOException {
     this.uri = uri;
-    this.method = method;
+    this.req = req;
+    this.resp = resp;
     constructionStack = LOG.isDebugEnabled() ? new Exception("stack") : null;
-    if (method == null) {
-      throw new IllegalArgumentException("Null 'method' parameter ");
+    if (req == null) {
+      throw new IllegalArgumentException("Null 'request' parameter ");
     }
     try {
-      inStream = method.getResponseBodyAsStream();
+      inStream = resp.getEntity().getContent();
     } catch (IOException e) {
       inStream = new ByteArrayInputStream(new byte[]{});
       throw releaseAndRethrow("getResponseBodyAsStream() in constructor -" + e, e);
@@ -100,11 +103,11 @@ public class HttpInputStreamWithRelease extends InputStream {
         if (LOG.isDebugEnabled()) {
           LOG.debug("Releasing connection to " + uri + ":  " + reason, ex);
         }
-        if (method != null) {
+        if (req != null) {
           if (!dataConsumed) {
-            method.abort();
+            req.abort();
           }
-          method.releaseConnection();
+          req.releaseConnection();
         }
         if (inStream != null) {
           //this guard may seem un-needed, but a stack trace seen

File diff suppressed because it is too large
+ 296 - 250
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/http/SwiftRestClient.java


+ 6 - 5
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/snative/SwiftNativeFileSystemStore.java

@@ -19,8 +19,9 @@ package org.apache.hadoop.fs.swift.snative;
 
 import com.fasterxml.jackson.databind.type.CollectionType;
 
-import org.apache.commons.httpclient.Header;
-import org.apache.commons.httpclient.HttpStatus;
+import org.apache.http.Header;
+import org.apache.http.HttpStatus;
+import org.apache.http.message.BasicHeader;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -166,9 +167,9 @@ public class SwiftNativeFileSystemStore {
     }
 
     swiftRestClient.upload(toObjectPath(path),
-            new ByteArrayInputStream(new byte[0]),
-            0,
-            new Header(SwiftProtocolConstants.X_OBJECT_MANIFEST, pathString));
+        new ByteArrayInputStream(new byte[0]),
+        0,
+        new BasicHeader(SwiftProtocolConstants.X_OBJECT_MANIFEST, pathString));
   }
 
   /**

+ 121 - 0
hadoop-tools/hadoop-openstack/src/main/java/org/apache/hadoop/fs/swift/util/HttpResponseUtils.java

@@ -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());
+    }
+  }
+}

+ 1 - 1
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/TestSwiftFileSystemPartitionedUploads.java

@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.fs.swift;
 
-import org.apache.commons.httpclient.Header;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 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.SwiftUtils;
 import org.apache.hadoop.io.IOUtils;
+import org.apache.http.Header;
 import org.junit.Test;
 import org.junit.internal.AssumptionViolatedException;
 

+ 1 - 1
hadoop-tools/hadoop-openstack/src/test/java/org/apache/hadoop/fs/swift/http/TestSwiftRestClient.java

@@ -18,7 +18,6 @@
 
 package org.apache.hadoop.fs.swift.http;
 
-import org.apache.commons.httpclient.Header;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 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.SwiftObjectPath;
 import org.apache.hadoop.fs.swift.util.SwiftTestUtils;
+import org.apache.http.Header;
 import org.junit.Assert;
 import org.junit.Assume;
 import org.junit.Before;

+ 0 - 3
hadoop-tools/hadoop-openstack/src/test/resources/log4j.properties

@@ -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 %x - %m%n"
 #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

Some files were not shown because too many files changed in this diff