You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2016/08/02 17:54:28 UTC

hbase git commit: HBASE-16267 Remove commons-httpclient dependency from hbase-rest module

Repository: hbase
Updated Branches:
  refs/heads/master b35cf8f93 -> 379b86c5d


HBASE-16267 Remove commons-httpclient dependency from hbase-rest module


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/379b86c5
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/379b86c5
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/379b86c5

Branch: refs/heads/master
Commit: 379b86c5df783eaeb53e6913afdde92da5351ec2
Parents: b35cf8f
Author: tedyu <yu...@gmail.com>
Authored: Tue Aug 2 10:53:57 2016 -0700
Committer: tedyu <yu...@gmail.com>
Committed: Tue Aug 2 10:53:57 2016 -0700

----------------------------------------------------------------------
 hbase-rest/pom.xml                              |   4 +
 .../hadoop/hbase/rest/MultiRowResource.java     |   2 +-
 .../apache/hadoop/hbase/rest/client/Client.java | 210 ++++++++++++-------
 .../hadoop/hbase/rest/client/Response.java      |  21 +-
 .../hbase/rest/TestGetAndPutResource.java       |   4 +-
 .../hadoop/hbase/rest/TestGzipFilter.java       |  21 +-
 .../hadoop/hbase/rest/TestMultiRowResource.java |   5 +-
 .../hadoop/hbase/rest/TestScannerResource.java  |   2 +-
 .../hadoop/hbase/rest/TestSchemaResource.java   |   5 +-
 .../hbase/rest/client/TestRemoteTable.java      |  11 +-
 pom.xml                                         |  18 +-
 11 files changed, 196 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 2e1b245..4d42a7a 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -279,6 +279,10 @@
       <artifactId>httpclient</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.httpcomponents</groupId>
+      <artifactId>httpcore</artifactId>
+    </dependency>
+    <dependency>
       <groupId>commons-lang</groupId>
       <artifactId>commons-lang</artifactId>
     </dependency>

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
index 9cafe27..b952c00 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/MultiRowResource.java
@@ -115,7 +115,7 @@ public class MultiRowResource extends ResourceBase implements Constants {
         servlet.getMetrics().incrementSucessfulGetRequests(1);
         return Response.ok(model).build();
       }
-    } catch (Exception e) {
+    } catch (IOException e) {
       servlet.getMetrics().incrementFailedGetRequests(1);
       return processException(e);
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
index e26de63..204f688 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Client.java
@@ -19,30 +19,34 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.net.URI;
+import java.net.URISyntaxException;
 import java.util.Collections;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 
-import org.apache.commons.httpclient.Header;
-import org.apache.commons.httpclient.HttpClient;
-import org.apache.commons.httpclient.HttpMethod;
-import org.apache.commons.httpclient.HttpVersion;
-import org.apache.commons.httpclient.MultiThreadedHttpConnectionManager;
-import org.apache.commons.httpclient.URI;
-import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
-import org.apache.commons.httpclient.methods.DeleteMethod;
-import org.apache.commons.httpclient.methods.GetMethod;
-import org.apache.commons.httpclient.methods.HeadMethod;
-import org.apache.commons.httpclient.methods.PostMethod;
-import org.apache.commons.httpclient.methods.PutMethod;
-import org.apache.commons.httpclient.params.HttpClientParams;
-import org.apache.commons.httpclient.params.HttpConnectionManagerParams;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
+import org.apache.http.client.HttpClient;
+import org.apache.http.client.methods.HttpDelete;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpHead;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpPut;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.InputStreamEntity;
+import org.apache.http.impl.client.DefaultHttpClient;
+import org.apache.http.message.BasicHeader;
+import org.apache.http.params.CoreConnectionPNames;
+import org.apache.http.util.EntityUtils;
 
 /**
  * A wrapper around HttpClient which provides some useful function and
@@ -58,6 +62,8 @@ public class Client {
   private HttpClient httpClient;
   private Cluster cluster;
   private boolean sslEnabled;
+  private HttpResponse resp;
+  private HttpGet httpGet = null;
 
   private Map<String, String> extraHeaders;
 
@@ -71,18 +77,13 @@ public class Client {
   private void initialize(Cluster cluster, boolean sslEnabled) {
     this.cluster = cluster;
     this.sslEnabled = sslEnabled;
-    MultiThreadedHttpConnectionManager manager =
-      new MultiThreadedHttpConnectionManager();
-    HttpConnectionManagerParams managerParams = manager.getParams();
-    managerParams.setConnectionTimeout(2000); // 2 s
-    managerParams.setDefaultMaxConnectionsPerHost(10);
-    managerParams.setMaxTotalConnections(100);
     extraHeaders = new ConcurrentHashMap<String, String>();
-    this.httpClient = new HttpClient(manager);
-    HttpClientParams clientParams = httpClient.getParams();
-    clientParams.setVersion(HttpVersion.HTTP_1_1);
-
+    String clspath = System.getProperty("java.class.path");
+    LOG.debug("classpath " + clspath);
+    this.httpClient = new DefaultHttpClient();
+    this.httpClient.getParams().setIntParameter(CoreConnectionPNames.CONNECTION_TIMEOUT, 2000);
   }
+
   /**
    * Constructor
    * @param cluster the cluster definition
@@ -104,9 +105,6 @@ public class Client {
    * Shut down the client. Close any open persistent connections.
    */
   public void shutdown() {
-    MultiThreadedHttpConnectionManager manager =
-      (MultiThreadedHttpConnectionManager) httpClient.getHttpConnectionManager();
-    manager.shutdown();
   }
 
   /**
@@ -159,7 +157,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int executePathOnly(Cluster cluster, HttpMethod method,
+  public HttpResponse executePathOnly(Cluster cluster, HttpUriRequest method,
       Header[] headers, String path) throws IOException {
     IOException lastException;
     if (cluster.nodes.size() < 1) {
@@ -178,10 +176,29 @@ public class Client {
         }
         sb.append(cluster.lastHost);
         sb.append(path);
-        URI uri = new URI(sb.toString(), true);
+        URI uri = new URI(sb.toString());
+        if (method instanceof HttpPut) {
+          HttpPut put = new HttpPut(uri);
+          put.setEntity(((HttpPut) method).getEntity());
+          put.setHeaders(method.getAllHeaders());
+          method = put;
+        } else if (method instanceof HttpGet) {
+          method = new HttpGet(uri);
+        } else if (method instanceof HttpHead) {
+          method = new HttpHead(uri);
+        } else if (method instanceof HttpDelete) {
+          method = new HttpDelete(uri);
+        } else if (method instanceof HttpPost) {
+          HttpPost post = new HttpPost(uri);
+          post.setEntity(((HttpPost) method).getEntity());
+          post.setHeaders(method.getAllHeaders());
+          method = post;
+        }
         return executeURI(method, headers, uri.toString());
       } catch (IOException e) {
         lastException = e;
+      } catch (URISyntaxException use) {
+        lastException = new IOException(use);
       }
     } while (++i != start && i < cluster.nodes.size());
     throw lastException;
@@ -195,25 +212,27 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int executeURI(HttpMethod method, Header[] headers, String uri)
+  public HttpResponse executeURI(HttpUriRequest method, Header[] headers, String uri)
       throws IOException {
-    method.setURI(new URI(uri, true));
+    // method.setURI(new URI(uri, true));
     for (Map.Entry<String, String> e: extraHeaders.entrySet()) {
-      method.addRequestHeader(e.getKey(), e.getValue());
+      method.addHeader(e.getKey(), e.getValue());
     }
     if (headers != null) {
       for (Header header: headers) {
-        method.addRequestHeader(header);
+        method.addHeader(header);
       }
     }
     long startTime = System.currentTimeMillis();
-    int code = httpClient.executeMethod(method);
+    if (resp != null) EntityUtils.consumeQuietly(resp.getEntity());
+    resp = httpClient.execute(method);
+
     long endTime = System.currentTimeMillis();
     if (LOG.isTraceEnabled()) {
-      LOG.trace(method.getName() + " " + uri + " " + code + " " +
-        method.getStatusText() + " in " + (endTime - startTime) + " ms");
+      LOG.trace(method.getMethod() + " " + uri + " " + resp.getStatusLine().getStatusCode() + " " +
+          resp.getStatusLine().getReasonPhrase() + " in " + (endTime - startTime) + " ms");
     }
-    return code;
+    return resp;
   }
 
   /**
@@ -227,7 +246,7 @@ public class Client {
    * @return the HTTP response code
    * @throws IOException
    */
-  public int execute(Cluster cluster, HttpMethod method, Header[] headers,
+  public HttpResponse execute(Cluster cluster, HttpUriRequest method, Header[] headers,
       String path) throws IOException {
     if (path.startsWith("/")) {
       return executePathOnly(cluster, method, headers, path);
@@ -269,11 +288,10 @@ public class Client {
    */
   public Response head(Cluster cluster, String path, Header[] headers)
       throws IOException {
-    HeadMethod method = new HeadMethod();
+    HttpHead method = new HttpHead(path);
     try {
-      int code = execute(cluster, method, null, path);
-      headers = method.getResponseHeaders();
-      return new Response(code, headers, null);
+      HttpResponse resp = execute(cluster, method, null, path);
+      return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), null);
     } finally {
       method.releaseConnection();
     }
@@ -322,7 +340,7 @@ public class Client {
   public Response get(Cluster cluster, String path, String accept)
       throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Accept", accept);
+    headers[0] = new BasicHeader("Accept", accept);
     return get(cluster, path, headers);
   }
 
@@ -339,6 +357,45 @@ public class Client {
   }
 
   /**
+   * 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, null if body is empty
+   * @throws IOException If an I/O (transport) problem occurs while obtaining the
+   * response body.
+   */
+  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
+      "NP_LOAD_OF_KNOWN_NULL_VALUE", justification = "null is possible return value")
+  public static byte[] getResponseBody(HttpResponse resp) throws IOException {
+    if (resp.getEntity() == null) return null;
+    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;
+    }
+  }
+
+  /**
    * Send a GET request
    * @param c the cluster definition
    * @param path the path or URI
@@ -348,16 +405,13 @@ public class Client {
    */
   public Response get(Cluster c, String path, Header[] headers)
       throws IOException {
-    GetMethod method = new GetMethod();
-    try {
-      int code = execute(c, method, headers, path);
-      headers = method.getResponseHeaders();
-      byte[] body = method.getResponseBody();
-      InputStream in = method.getResponseBodyAsStream();
-      return new Response(code, headers, body, in);
-    } finally {
-      method.releaseConnection();
+    if (httpGet != null) {
+      httpGet.releaseConnection();
     }
+    httpGet = new HttpGet(path);
+    HttpResponse resp = execute(c, httpGet, headers, path);
+    return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(),
+        resp, resp.getEntity() == null ? null : resp.getEntity().getContent());
   }
 
   /**
@@ -399,7 +453,7 @@ public class Client {
   public Response put(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     return put(cluster, path, headers, content);
   }
 
@@ -417,7 +471,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -450,13 +504,13 @@ public class Client {
    */
   public Response put(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    PutMethod method = new PutMethod();
+    HttpPut method = new HttpPut(path);
     try {
-      method.setRequestEntity(new ByteArrayRequestEntity(content));
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      content = method.getResponseBody();
-      return new Response(code, headers, content);
+      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -501,7 +555,7 @@ public class Client {
   public Response post(Cluster cluster, String path, String contentType,
       byte[] content) throws IOException {
     Header[] headers = new Header[1];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     return post(cluster, path, headers, content);
   }
 
@@ -519,7 +573,7 @@ public class Client {
       byte[] content, Header extraHdr) throws IOException {
     int cnt = extraHdr == null ? 1 : 2;
     Header[] headers = new Header[cnt];
-    headers[0] = new Header("Content-Type", contentType);
+    headers[0] = new BasicHeader("Content-Type", contentType);
     if (extraHdr != null) {
       headers[1] = extraHdr;
     }
@@ -552,13 +606,13 @@ public class Client {
    */
   public Response post(Cluster cluster, String path, Header[] headers,
       byte[] content) throws IOException {
-    PostMethod method = new PostMethod();
+    HttpPost method = new HttpPost(path);
     try {
-      method.setRequestEntity(new ByteArrayRequestEntity(content));
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      content = method.getResponseBody();
-      return new Response(code, headers, content);
+      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -593,12 +647,12 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path) throws IOException {
-    DeleteMethod method = new DeleteMethod();
+    HttpDelete method = new HttpDelete(path);
     try {
-      int code = execute(cluster, method, null, path);
-      Header[] headers = method.getResponseHeaders();
-      byte[] content = method.getResponseBody();
-      return new Response(code, headers, content);
+      HttpResponse resp = execute(cluster, method, null, path);
+      Header[] headers = resp.getAllHeaders();
+      byte[] content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }
@@ -612,13 +666,13 @@ public class Client {
    * @throws IOException for error
    */
   public Response delete(Cluster cluster, String path, Header extraHdr) throws IOException {
-    DeleteMethod method = new DeleteMethod();
+    HttpDelete method = new HttpDelete(path);
     try {
       Header[] headers = { extraHdr };
-      int code = execute(cluster, method, headers, path);
-      headers = method.getResponseHeaders();
-      byte[] content = method.getResponseBody();
-      return new Response(code, headers, content);
+      HttpResponse resp = execute(cluster, method, headers, path);
+      headers = resp.getAllHeaders();
+      byte[] content = getResponseBody(resp);
+      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
     } finally {
       method.releaseConnection();
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
index 871b646..27db365 100644
--- a/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
+++ b/hbase-rest/src/main/java/org/apache/hadoop/hbase/rest/client/Response.java
@@ -19,11 +19,14 @@
 
 package org.apache.hadoop.hbase.rest.client;
 
+import java.io.IOException;
 import java.io.InputStream;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.HttpResponse;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
 import org.apache.hadoop.hbase.classification.InterfaceStability;
+import org.mortbay.log.Log;
 
 /**
  * The HTTP result code, response headers, and body of a HTTP response.
@@ -34,6 +37,7 @@ public class Response {
   private int code;
   private Header[] headers;
   private byte[] body;
+  private HttpResponse resp;
   private InputStream stream;
 
   /**
@@ -69,13 +73,15 @@ public class Response {
    * Constructor
    * @param code the HTTP response code
    * @param headers headers the HTTP response headers
-   * @param body the response body, can be null
+   * @param resp the response
    * @param in Inputstream if the response had one.
+   * Note: this is not thread-safe
    */
-  public Response(int code, Header[] headers, byte[] body, InputStream in) {
+  public Response(int code, Header[] headers, HttpResponse resp, InputStream in) {
     this.code = code;
     this.headers = headers;
-    this.body = body;
+    this.body = null;
+    this.resp = resp;
     this.stream = in;
   }
 
@@ -129,6 +135,13 @@ public class Response {
    * @return the HTTP response body
    */
   public byte[] getBody() {
+    if (body == null) {
+      try {
+        body = Client.getResponseBody(resp);
+      } catch (IOException ioe) {
+        Log.debug("encountered ioe when obtaining body", ioe);
+      }
+    }
     return body;
   }
 

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
index c6fb2ff..d6eb1b3 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGetAndPutResource.java
@@ -24,14 +24,12 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.io.StringWriter;
 import java.net.URLEncoder;
-import java.util.Dictionary;
 import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
 import javax.xml.bind.JAXBException;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
 import org.apache.hadoop.hbase.CompatibilityFactory;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.rest.client.Response;

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
index 42d355d..5097454 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestGzipFilter.java
@@ -27,7 +27,8 @@ import java.io.ByteArrayOutputStream;
 import java.util.zip.GZIPInputStream;
 import java.util.zip.GZIPOutputStream;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -95,8 +96,8 @@ public class TestGzipFilter {
     // input side filter
 
     Header[] headers = new Header[2];
-    headers[0] = new Header("Content-Type", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Content-Encoding", "gzip");
+    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Content-Encoding", "gzip");
     Response response = client.put(path, headers, value_1_gzip);
     assertEquals(response.getCode(), 200);
 
@@ -110,8 +111,8 @@ public class TestGzipFilter {
 
     // output side filter
 
-    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
     response = client.get(path, headers);
     assertEquals(response.getCode(), 200);
     ByteArrayInputStream bis = new ByteArrayInputStream(response.getBody());
@@ -128,8 +129,8 @@ public class TestGzipFilter {
   @Test
   public void testErrorNotGzipped() throws Exception {
     Header[] headers = new Header[2];
-    headers[0] = new Header("Accept", Constants.MIMETYPE_BINARY);
-    headers[1] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Accept", Constants.MIMETYPE_BINARY);
+    headers[1] = new BasicHeader("Accept-Encoding", "gzip");
     Response response = client.get("/" + TABLE + "/" + ROW_1 + "/" + COLUMN_2, headers);
     assertEquals(response.getCode(), 404);
     String contentEncoding = response.getHeader("Content-Encoding");
@@ -142,9 +143,9 @@ public class TestGzipFilter {
 
   void testScannerResultCodes() throws Exception {
     Header[] headers = new Header[3];
-    headers[0] = new Header("Content-Type", Constants.MIMETYPE_XML);
-    headers[1] = new Header("Accept", Constants.MIMETYPE_JSON);
-    headers[2] = new Header("Accept-Encoding", "gzip");
+    headers[0] = new BasicHeader("Content-Type", Constants.MIMETYPE_XML);
+    headers[1] = new BasicHeader("Accept", Constants.MIMETYPE_JSON);
+    headers[2] = new BasicHeader("Accept-Encoding", "gzip");
     Response response = client.post("/" + TABLE + "/scanner", headers,
         "<Scanner/>".getBytes());
     assertEquals(response.getCode(), 201);

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
index 958cb15..84d1855 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestMultiRowResource.java
@@ -18,7 +18,8 @@
  */
 package org.apache.hadoop.hbase.rest;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.client.Admin;
@@ -96,7 +97,7 @@ public class TestMultiRowResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     context = JAXBContext.newInstance(

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
index 5114b11..be7ee9a 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestScannerResource.java
@@ -37,7 +37,7 @@ import javax.xml.bind.JAXBException;
 import javax.xml.bind.Marshaller;
 import javax.xml.bind.Unmarshaller;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.HColumnDescriptor;

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
index d005445..19fdaf0 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/TestSchemaResource.java
@@ -28,7 +28,8 @@ import java.util.List;
 import javax.xml.bind.JAXBContext;
 import javax.xml.bind.JAXBException;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -86,7 +87,7 @@ public class TestSchemaResource {
   public static void setUpBeforeClass() throws Exception {
     conf = TEST_UTIL.getConfiguration();
     conf.setBoolean(RESTServer.REST_CSRF_ENABLED_KEY, csrfEnabled);
-    extraHdr = new Header(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
+    extraHdr = new BasicHeader(RESTServer.REST_CSRF_CUSTOM_HEADER_DEFAULT, "");
     TEST_UTIL.startMiniCluster();
     REST_TEST_UTIL.startServletContainer(conf);
     client = new Client(new Cluster().add("localhost",

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
----------------------------------------------------------------------
diff --git a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
index 19d0587..1ac37fa 100644
--- a/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
+++ b/hbase-rest/src/test/java/org/apache/hadoop/hbase/rest/client/TestRemoteTable.java
@@ -30,7 +30,8 @@ import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
 
-import org.apache.commons.httpclient.Header;
+import org.apache.http.Header;
+import org.apache.http.message.BasicHeader;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
@@ -513,16 +514,16 @@ public class TestRemoteTable {
     Response response = new Response(200);
     assertEquals(200, response.getCode());
     Header[] headers = new Header[2];
-    headers[0] = new Header("header1", "value1");
-    headers[1] = new Header("header2", "value2");
+    headers[0] = new BasicHeader("header1", "value1");
+    headers[1] = new BasicHeader("header2", "value2");
     response = new Response(200, headers);
     assertEquals("value1", response.getHeader("header1"));
     assertFalse(response.hasBody());
     response.setCode(404);
     assertEquals(404, response.getCode());
     headers = new Header[2];
-    headers[0] = new Header("header1", "value1.1");
-    headers[1] = new Header("header2", "value2");
+    headers[0] = new BasicHeader("header1", "value1.1");
+    headers[1] = new BasicHeader("header2", "value2");
     response.setHeaders(headers);
     assertEquals("value1.1", response.getHeader("header1"));
     response.setBody(Bytes.toBytes("body"));

http://git-wip-us.apache.org/repos/asf/hbase/blob/379b86c5/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index ea7743f..c73f798 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1254,7 +1254,7 @@
     <disruptor.version>3.3.0</disruptor.version>
     <!-- Do not use versions earlier than 3.2.2 due to a security vulnerability -->
     <collections.version>3.2.2</collections.version>
-    <httpclient.version>4.3.6</httpclient.version>
+    <httpclient.version>4.5.2</httpclient.version>
     <httpcore.version>4.4.4</httpcore.version>
     <metrics-core.version>3.1.2</metrics-core.version>
     <guava.version>12.0.1</guava.version>
@@ -2194,6 +2194,10 @@
             <artifactId>hadoop-common</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
               <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
@@ -2224,6 +2228,10 @@
             <artifactId>hadoop-minicluster</artifactId>
             <version>${hadoop-two.version}</version>
             <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
               <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
@@ -2360,6 +2368,10 @@
            <artifactId>hadoop-common</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
              <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>
@@ -2395,6 +2407,10 @@
            <artifactId>hadoop-minicluster</artifactId>
            <version>${hadoop-three.version}</version>
            <exclusions>
+            <exclusion>
+              <groupId>commons-httpclient</groupId>
+              <artifactId>commons-httpclient</artifactId>
+            </exclusion>
              <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>