001/*
002 *
003 * Licensed to the Apache Software Foundation (ASF) under one
004 * or more contributor license agreements.  See the NOTICE file
005 * distributed with this work for additional information
006 * regarding copyright ownership.  The ASF licenses this file
007 * to you under the Apache License, Version 2.0 (the
008 * "License"); you may not use this file except in compliance
009 * with the License.  You may obtain a copy of the License at
010 *
011 *     http://www.apache.org/licenses/LICENSE-2.0
012 *
013 * Unless required by applicable law or agreed to in writing, software
014 * distributed under the License is distributed on an "AS IS" BASIS,
015 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
016 * See the License for the specific language governing permissions and
017 * limitations under the License.
018 */
019
020package org.apache.hadoop.hbase.rest.client;
021
022import java.io.ByteArrayInputStream;
023import java.io.ByteArrayOutputStream;
024import java.io.File;
025import java.io.FileInputStream;
026import java.io.IOException;
027import java.io.InputStream;
028import java.net.URI;
029import java.net.URISyntaxException;
030import java.net.URL;
031import java.security.KeyManagementException;
032import java.security.KeyStore;
033import java.security.KeyStoreException;
034import java.security.NoSuchAlgorithmException;
035import java.security.cert.CertificateException;
036import java.util.Collections;
037import java.util.Map;
038import java.util.Optional;
039import java.util.concurrent.ConcurrentHashMap;
040import javax.net.ssl.SSLContext;
041import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
042import org.apache.hadoop.security.authentication.client.AuthenticationException;
043import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
044import org.apache.http.Header;
045import org.apache.http.HttpResponse;
046import org.apache.http.HttpStatus;
047import org.apache.http.client.HttpClient;
048import org.apache.http.client.config.RequestConfig;
049import org.apache.http.client.methods.HttpDelete;
050import org.apache.http.client.methods.HttpGet;
051import org.apache.http.client.methods.HttpHead;
052import org.apache.http.client.methods.HttpPost;
053import org.apache.http.client.methods.HttpPut;
054import org.apache.http.client.methods.HttpUriRequest;
055import org.apache.http.entity.InputStreamEntity;
056import org.apache.http.impl.client.HttpClientBuilder;
057import org.apache.http.impl.client.HttpClients;
058import org.apache.http.message.BasicHeader;
059import org.apache.http.ssl.SSLContexts;
060import org.apache.http.util.EntityUtils;
061import org.apache.yetus.audience.InterfaceAudience;
062import org.slf4j.Logger;
063import org.slf4j.LoggerFactory;
064
065/**
066 * A wrapper around HttpClient which provides some useful function and
067 * semantics for interacting with the REST gateway.
068 */
069@InterfaceAudience.Public
070public class Client {
071  public static final Header[] EMPTY_HEADER_ARRAY = new Header[0];
072
073  private static final Logger LOG = LoggerFactory.getLogger(Client.class);
074
075  private HttpClient httpClient;
076  private Cluster cluster;
077  private boolean sslEnabled;
078  private HttpResponse resp;
079  private HttpGet httpGet = null;
080
081  private Map<String, String> extraHeaders;
082
083  private static final String AUTH_COOKIE = "hadoop.auth";
084  private static final String AUTH_COOKIE_EQ = AUTH_COOKIE + "=";
085  private static final String COOKIE = "Cookie";
086
087  /**
088   * Default Constructor
089   */
090  public Client() {
091    this(null);
092  }
093
094  private void initialize(Cluster cluster, boolean sslEnabled, Optional<KeyStore> trustStore) {
095    this.cluster = cluster;
096    this.sslEnabled = sslEnabled;
097    extraHeaders = new ConcurrentHashMap<>();
098    String clspath = System.getProperty("java.class.path");
099    LOG.debug("classpath " + clspath);
100    HttpClientBuilder httpClientBuilder = HttpClients.custom();
101
102    RequestConfig requestConfig = RequestConfig.custom().
103      setConnectTimeout(2000).build();
104    httpClientBuilder.setDefaultRequestConfig(requestConfig);
105
106    // Since HBASE-25267 we don't use the deprecated DefaultHttpClient anymore.
107    // The new http client would decompress the gzip content automatically.
108    // In order to keep the original behaviour of this public class, we disable
109    // automatic content compression.
110    httpClientBuilder.disableContentCompression();
111
112    if(sslEnabled && trustStore.isPresent()) {
113      try {
114        SSLContext sslcontext =
115          SSLContexts.custom().loadTrustMaterial(trustStore.get(), null).build();
116        httpClientBuilder.setSSLContext(sslcontext);
117      } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) {
118        throw new ClientTrustStoreInitializationException("Error while processing truststore", e);
119      }
120    }
121
122    this.httpClient = httpClientBuilder.build();
123  }
124
125  /**
126   * Constructor
127   * @param cluster the cluster definition
128   */
129  public Client(Cluster cluster) {
130    this(cluster, false);
131  }
132
133  /**
134   * Constructor
135   * @param cluster the cluster definition
136   * @param sslEnabled enable SSL or not
137   */
138  public Client(Cluster cluster, boolean sslEnabled) {
139    initialize(cluster, sslEnabled, Optional.empty());
140  }
141
142  /**
143   * Constructor, allowing to define custom trust store (only for SSL connections)
144   *
145   * @param cluster the cluster definition
146   * @param trustStorePath custom trust store to use for SSL connections
147   * @param trustStorePassword password to use for custom trust store
148   * @param trustStoreType type of custom trust store
149   *
150   * @throws ClientTrustStoreInitializationException if the trust store file can not be loaded
151   */
152  public Client(Cluster cluster, String trustStorePath,
153    Optional<String> trustStorePassword, Optional<String> trustStoreType) {
154
155    char[] password = trustStorePassword.map(String::toCharArray).orElse(null);
156    String type = trustStoreType.orElse(KeyStore.getDefaultType());
157
158    KeyStore trustStore;
159    try(FileInputStream inputStream = new FileInputStream(new File(trustStorePath))) {
160      trustStore = KeyStore.getInstance(type);
161      trustStore.load(inputStream, password);
162    } catch (KeyStoreException e) {
163      throw new ClientTrustStoreInitializationException(
164        "Invalid trust store type: " + type, e);
165    } catch (CertificateException | NoSuchAlgorithmException | IOException e) {
166      throw new ClientTrustStoreInitializationException(
167        "Trust store load error: " + trustStorePath, e);
168    }
169
170    initialize(cluster, true, Optional.of(trustStore));
171  }
172
173  /**
174   * Shut down the client. Close any open persistent connections.
175   */
176  public void shutdown() {
177  }
178
179  /**
180   * @return the wrapped HttpClient
181   */
182  public HttpClient getHttpClient() {
183    return httpClient;
184  }
185
186  /**
187   * Add extra headers.  These extra headers will be applied to all http
188   * methods before they are removed. If any header is not used any more,
189   * client needs to remove it explicitly.
190   */
191  public void addExtraHeader(final String name, final String value) {
192    extraHeaders.put(name, value);
193  }
194
195  /**
196   * Get an extra header value.
197   */
198  public String getExtraHeader(final String name) {
199    return extraHeaders.get(name);
200  }
201
202  /**
203   * Get all extra headers (read-only).
204   */
205  public Map<String, String> getExtraHeaders() {
206    return Collections.unmodifiableMap(extraHeaders);
207  }
208
209  /**
210   * Remove an extra header.
211   */
212  public void removeExtraHeader(final String name) {
213    extraHeaders.remove(name);
214  }
215
216  /**
217   * Execute a transaction method given only the path. Will select at random
218   * one of the members of the supplied cluster definition and iterate through
219   * the list until a transaction can be successfully completed. The
220   * definition of success here is a complete HTTP transaction, irrespective
221   * of result code.
222   * @param cluster the cluster definition
223   * @param method the transaction method
224   * @param headers HTTP header values to send
225   * @param path the properly urlencoded path
226   * @return the HTTP response code
227   * @throws IOException
228   */
229  public HttpResponse executePathOnly(Cluster cluster, HttpUriRequest method,
230      Header[] headers, String path) throws IOException {
231    IOException lastException;
232    if (cluster.nodes.size() < 1) {
233      throw new IOException("Cluster is empty");
234    }
235    int start = (int)Math.round((cluster.nodes.size() - 1) * Math.random());
236    int i = start;
237    do {
238      cluster.lastHost = cluster.nodes.get(i);
239      try {
240        StringBuilder sb = new StringBuilder();
241        if (sslEnabled) {
242          sb.append("https://");
243        } else {
244          sb.append("http://");
245        }
246        sb.append(cluster.lastHost);
247        sb.append(path);
248        URI uri = new URI(sb.toString());
249        if (method instanceof HttpPut) {
250          HttpPut put = new HttpPut(uri);
251          put.setEntity(((HttpPut) method).getEntity());
252          put.setHeaders(method.getAllHeaders());
253          method = put;
254        } else if (method instanceof HttpGet) {
255          method = new HttpGet(uri);
256        } else if (method instanceof HttpHead) {
257          method = new HttpHead(uri);
258        } else if (method instanceof HttpDelete) {
259          method = new HttpDelete(uri);
260        } else if (method instanceof HttpPost) {
261          HttpPost post = new HttpPost(uri);
262          post.setEntity(((HttpPost) method).getEntity());
263          post.setHeaders(method.getAllHeaders());
264          method = post;
265        }
266        return executeURI(method, headers, uri.toString());
267      } catch (IOException e) {
268        lastException = e;
269      } catch (URISyntaxException use) {
270        lastException = new IOException(use);
271      }
272    } while (++i != start && i < cluster.nodes.size());
273    throw lastException;
274  }
275
276  /**
277   * Execute a transaction method given a complete URI.
278   * @param method the transaction method
279   * @param headers HTTP header values to send
280   * @param uri a properly urlencoded URI
281   * @return the HTTP response code
282   * @throws IOException
283   */
284  public HttpResponse executeURI(HttpUriRequest method, Header[] headers, String uri)
285      throws IOException {
286    // method.setURI(new URI(uri, true));
287    for (Map.Entry<String, String> e: extraHeaders.entrySet()) {
288      method.addHeader(e.getKey(), e.getValue());
289    }
290    if (headers != null) {
291      for (Header header: headers) {
292        method.addHeader(header);
293      }
294    }
295    long startTime = System.currentTimeMillis();
296    if (resp != null) EntityUtils.consumeQuietly(resp.getEntity());
297    resp = httpClient.execute(method);
298    if (resp.getStatusLine().getStatusCode() == HttpStatus.SC_UNAUTHORIZED) {
299      // Authentication error
300      LOG.debug("Performing negotiation with the server.");
301      negotiate(method, uri);
302      resp = httpClient.execute(method);
303    }
304
305    long endTime = System.currentTimeMillis();
306    if (LOG.isTraceEnabled()) {
307      LOG.trace(method.getMethod() + " " + uri + " " + resp.getStatusLine().getStatusCode() + " " +
308          resp.getStatusLine().getReasonPhrase() + " in " + (endTime - startTime) + " ms");
309    }
310    return resp;
311  }
312
313  /**
314   * Execute a transaction method. Will call either <tt>executePathOnly</tt>
315   * or <tt>executeURI</tt> depending on whether a path only is supplied in
316   * 'path', or if a complete URI is passed instead, respectively.
317   * @param cluster the cluster definition
318   * @param method the HTTP method
319   * @param headers HTTP header values to send
320   * @param path the properly urlencoded path or URI
321   * @return the HTTP response code
322   * @throws IOException
323   */
324  public HttpResponse execute(Cluster cluster, HttpUriRequest method, Header[] headers,
325      String path) throws IOException {
326    if (path.startsWith("/")) {
327      return executePathOnly(cluster, method, headers, path);
328    }
329    return executeURI(method, headers, path);
330  }
331
332  /**
333   * Initiate client side Kerberos negotiation with the server.
334   * @param method method to inject the authentication token into.
335   * @param uri the String to parse as a URL.
336   * @throws IOException if unknown protocol is found.
337   */
338  private void negotiate(HttpUriRequest method, String uri) throws IOException {
339    try {
340      AuthenticatedURL.Token token = new AuthenticatedURL.Token();
341      KerberosAuthenticator authenticator = new KerberosAuthenticator();
342      authenticator.authenticate(new URL(uri), token);
343      // Inject the obtained negotiated token in the method cookie
344      injectToken(method, token);
345    } catch (AuthenticationException e) {
346      LOG.error("Failed to negotiate with the server.", e);
347      throw new IOException(e);
348    }
349  }
350
351  /**
352   * Helper method that injects an authentication token to send with the method.
353   * @param method method to inject the authentication token into.
354   * @param token authentication token to inject.
355   */
356  private void injectToken(HttpUriRequest method, AuthenticatedURL.Token token) {
357    String t = token.toString();
358    if (t != null) {
359      if (!t.startsWith("\"")) {
360        t = "\"" + t + "\"";
361      }
362      method.addHeader(COOKIE, AUTH_COOKIE_EQ + t);
363    }
364  }
365
366  /**
367   * @return the cluster definition
368   */
369  public Cluster getCluster() {
370    return cluster;
371  }
372
373  /**
374   * @param cluster the cluster definition
375   */
376  public void setCluster(Cluster cluster) {
377    this.cluster = cluster;
378  }
379
380  /**
381   * Send a HEAD request
382   * @param path the path or URI
383   * @return a Response object with response detail
384   * @throws IOException
385   */
386  public Response head(String path) throws IOException {
387    return head(cluster, path, null);
388  }
389
390  /**
391   * Send a HEAD request
392   * @param cluster the cluster definition
393   * @param path the path or URI
394   * @param headers the HTTP headers to include in the request
395   * @return a Response object with response detail
396   * @throws IOException
397   */
398  public Response head(Cluster cluster, String path, Header[] headers)
399      throws IOException {
400    HttpHead method = new HttpHead(path);
401    try {
402      HttpResponse resp = execute(cluster, method, null, path);
403      return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), null);
404    } finally {
405      method.releaseConnection();
406    }
407  }
408
409  /**
410   * Send a GET request
411   * @param path the path or URI
412   * @return a Response object with response detail
413   * @throws IOException
414   */
415  public Response get(String path) throws IOException {
416    return get(cluster, path);
417  }
418
419  /**
420   * Send a GET request
421   * @param cluster the cluster definition
422   * @param path the path or URI
423   * @return a Response object with response detail
424   * @throws IOException
425   */
426  public Response get(Cluster cluster, String path) throws IOException {
427    return get(cluster, path, EMPTY_HEADER_ARRAY);
428  }
429
430  /**
431   * Send a GET request
432   * @param path the path or URI
433   * @param accept Accept header value
434   * @return a Response object with response detail
435   * @throws IOException
436   */
437  public Response get(String path, String accept) throws IOException {
438    return get(cluster, path, accept);
439  }
440
441  /**
442   * Send a GET request
443   * @param cluster the cluster definition
444   * @param path the path or URI
445   * @param accept Accept header value
446   * @return a Response object with response detail
447   * @throws IOException
448   */
449  public Response get(Cluster cluster, String path, String accept)
450      throws IOException {
451    Header[] headers = new Header[1];
452    headers[0] = new BasicHeader("Accept", accept);
453    return get(cluster, path, headers);
454  }
455
456  /**
457   * Send a GET request
458   * @param path the path or URI
459   * @param headers the HTTP headers to include in the request,
460   * <tt>Accept</tt> must be supplied
461   * @return a Response object with response detail
462   * @throws IOException
463   */
464  public Response get(String path, Header[] headers) throws IOException {
465    return get(cluster, path, headers);
466  }
467
468  /**
469   * Returns the response body of the HTTPResponse, if any, as an array of bytes.
470   * If response body is not available or cannot be read, returns <tt>null</tt>
471   *
472   * Note: This will cause the entire response body to be buffered in memory. A
473   * malicious server may easily exhaust all the VM memory. It is strongly
474   * recommended, to use getResponseAsStream if the content length of the response
475   * is unknown or reasonably large.
476   *
477   * @param resp HttpResponse
478   * @return The response body, null if body is empty
479   * @throws IOException If an I/O (transport) problem occurs while obtaining the
480   * response body.
481   */
482  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value =
483      "NP_LOAD_OF_KNOWN_NULL_VALUE", justification = "null is possible return value")
484  public static byte[] getResponseBody(HttpResponse resp) throws IOException {
485    if (resp.getEntity() == null) return null;
486    try (InputStream instream = resp.getEntity().getContent()) {
487      if (instream != null) {
488        long contentLength = resp.getEntity().getContentLength();
489        if (contentLength > Integer.MAX_VALUE) {
490          //guard integer cast from overflow
491          throw new IOException("Content too large to be buffered: " + contentLength +" bytes");
492        }
493        ByteArrayOutputStream outstream = new ByteArrayOutputStream(
494            contentLength > 0 ? (int) contentLength : 4*1024);
495        byte[] buffer = new byte[4096];
496        int len;
497        while ((len = instream.read(buffer)) > 0) {
498          outstream.write(buffer, 0, len);
499        }
500        outstream.close();
501        return outstream.toByteArray();
502      }
503      return null;
504    }
505  }
506
507  /**
508   * Send a GET request
509   * @param c the cluster definition
510   * @param path the path or URI
511   * @param headers the HTTP headers to include in the request
512   * @return a Response object with response detail
513   * @throws IOException
514   */
515  public Response get(Cluster c, String path, Header[] headers)
516      throws IOException {
517    if (httpGet != null) {
518      httpGet.releaseConnection();
519    }
520    httpGet = new HttpGet(path);
521    HttpResponse resp = execute(c, httpGet, headers, path);
522    return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(),
523        resp, resp.getEntity() == null ? null : resp.getEntity().getContent());
524  }
525
526  /**
527   * Send a PUT request
528   * @param path the path or URI
529   * @param contentType the content MIME type
530   * @param content the content bytes
531   * @return a Response object with response detail
532   * @throws IOException
533   */
534  public Response put(String path, String contentType, byte[] content)
535      throws IOException {
536    return put(cluster, path, contentType, content);
537  }
538
539  /**
540   * Send a PUT request
541   * @param path the path or URI
542   * @param contentType the content MIME type
543   * @param content the content bytes
544   * @param extraHdr extra Header to send
545   * @return a Response object with response detail
546   * @throws IOException
547   */
548  public Response put(String path, String contentType, byte[] content, Header extraHdr)
549      throws IOException {
550    return put(cluster, path, contentType, content, extraHdr);
551  }
552
553  /**
554   * Send a PUT request
555   * @param cluster the cluster definition
556   * @param path the path or URI
557   * @param contentType the content MIME type
558   * @param content the content bytes
559   * @return a Response object with response detail
560   * @throws IOException for error
561   */
562  public Response put(Cluster cluster, String path, String contentType,
563      byte[] content) throws IOException {
564    Header[] headers = new Header[1];
565    headers[0] = new BasicHeader("Content-Type", contentType);
566    return put(cluster, path, headers, content);
567  }
568
569  /**
570   * Send a PUT request
571   * @param cluster the cluster definition
572   * @param path the path or URI
573   * @param contentType the content MIME type
574   * @param content the content bytes
575   * @param extraHdr additional Header to send
576   * @return a Response object with response detail
577   * @throws IOException for error
578   */
579  public Response put(Cluster cluster, String path, String contentType,
580      byte[] content, Header extraHdr) throws IOException {
581    int cnt = extraHdr == null ? 1 : 2;
582    Header[] headers = new Header[cnt];
583    headers[0] = new BasicHeader("Content-Type", contentType);
584    if (extraHdr != null) {
585      headers[1] = extraHdr;
586    }
587    return put(cluster, path, headers, content);
588  }
589
590  /**
591   * Send a PUT request
592   * @param path the path or URI
593   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be
594   * supplied
595   * @param content the content bytes
596   * @return a Response object with response detail
597   * @throws IOException
598   */
599  public Response put(String path, Header[] headers, byte[] content)
600      throws IOException {
601    return put(cluster, path, headers, content);
602  }
603
604  /**
605   * Send a PUT request
606   * @param cluster the cluster definition
607   * @param path the path or URI
608   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be
609   * supplied
610   * @param content the content bytes
611   * @return a Response object with response detail
612   * @throws IOException
613   */
614  public Response put(Cluster cluster, String path, Header[] headers,
615      byte[] content) throws IOException {
616    HttpPut method = new HttpPut(path);
617    try {
618      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
619      HttpResponse resp = execute(cluster, method, headers, path);
620      headers = resp.getAllHeaders();
621      content = getResponseBody(resp);
622      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
623    } finally {
624      method.releaseConnection();
625    }
626  }
627
628  /**
629   * Send a POST request
630   * @param path the path or URI
631   * @param contentType the content MIME type
632   * @param content the content bytes
633   * @return a Response object with response detail
634   * @throws IOException
635   */
636  public Response post(String path, String contentType, byte[] content)
637      throws IOException {
638    return post(cluster, path, contentType, content);
639  }
640
641  /**
642   * Send a POST request
643   * @param path the path or URI
644   * @param contentType the content MIME type
645   * @param content the content bytes
646   * @param extraHdr additional Header to send
647   * @return a Response object with response detail
648   * @throws IOException
649   */
650  public Response post(String path, String contentType, byte[] content, Header extraHdr)
651      throws IOException {
652    return post(cluster, path, contentType, content, extraHdr);
653  }
654
655  /**
656   * Send a POST request
657   * @param cluster the cluster definition
658   * @param path the path or URI
659   * @param contentType the content MIME type
660   * @param content the content bytes
661   * @return a Response object with response detail
662   * @throws IOException for error
663   */
664  public Response post(Cluster cluster, String path, String contentType,
665      byte[] content) throws IOException {
666    Header[] headers = new Header[1];
667    headers[0] = new BasicHeader("Content-Type", contentType);
668    return post(cluster, path, headers, content);
669  }
670
671  /**
672   * Send a POST request
673   * @param cluster the cluster definition
674   * @param path the path or URI
675   * @param contentType the content MIME type
676   * @param content the content bytes
677   * @param extraHdr additional Header to send
678   * @return a Response object with response detail
679   * @throws IOException for error
680   */
681  public Response post(Cluster cluster, String path, String contentType,
682      byte[] content, Header extraHdr) throws IOException {
683    int cnt = extraHdr == null ? 1 : 2;
684    Header[] headers = new Header[cnt];
685    headers[0] = new BasicHeader("Content-Type", contentType);
686    if (extraHdr != null) {
687      headers[1] = extraHdr;
688    }
689    return post(cluster, path, headers, content);
690  }
691
692  /**
693   * Send a POST request
694   * @param path the path or URI
695   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be
696   * supplied
697   * @param content the content bytes
698   * @return a Response object with response detail
699   * @throws IOException
700   */
701  public Response post(String path, Header[] headers, byte[] content)
702      throws IOException {
703    return post(cluster, path, headers, content);
704  }
705
706  /**
707   * Send a POST request
708   * @param cluster the cluster definition
709   * @param path the path or URI
710   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be
711   * supplied
712   * @param content the content bytes
713   * @return a Response object with response detail
714   * @throws IOException
715   */
716  public Response post(Cluster cluster, String path, Header[] headers,
717      byte[] content) throws IOException {
718    HttpPost method = new HttpPost(path);
719    try {
720      method.setEntity(new InputStreamEntity(new ByteArrayInputStream(content), content.length));
721      HttpResponse resp = execute(cluster, method, headers, path);
722      headers = resp.getAllHeaders();
723      content = getResponseBody(resp);
724      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
725    } finally {
726      method.releaseConnection();
727    }
728  }
729
730  /**
731   * Send a DELETE request
732   * @param path the path or URI
733   * @return a Response object with response detail
734   * @throws IOException
735   */
736  public Response delete(String path) throws IOException {
737    return delete(cluster, path);
738  }
739
740  /**
741   * Send a DELETE request
742   * @param path the path or URI
743   * @param extraHdr additional Header to send
744   * @return a Response object with response detail
745   * @throws IOException
746   */
747  public Response delete(String path, Header extraHdr) throws IOException {
748    return delete(cluster, path, extraHdr);
749  }
750
751  /**
752   * Send a DELETE request
753   * @param cluster the cluster definition
754   * @param path the path or URI
755   * @return a Response object with response detail
756   * @throws IOException for error
757   */
758  public Response delete(Cluster cluster, String path) throws IOException {
759    HttpDelete method = new HttpDelete(path);
760    try {
761      HttpResponse resp = execute(cluster, method, null, path);
762      Header[] headers = resp.getAllHeaders();
763      byte[] content = getResponseBody(resp);
764      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
765    } finally {
766      method.releaseConnection();
767    }
768  }
769
770  /**
771   * Send a DELETE request
772   * @param cluster the cluster definition
773   * @param path the path or URI
774   * @return a Response object with response detail
775   * @throws IOException for error
776   */
777  public Response delete(Cluster cluster, String path, Header extraHdr) throws IOException {
778    HttpDelete method = new HttpDelete(path);
779    try {
780      Header[] headers = { extraHdr };
781      HttpResponse resp = execute(cluster, method, headers, path);
782      headers = resp.getAllHeaders();
783      byte[] content = getResponseBody(resp);
784      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
785    } finally {
786      method.releaseConnection();
787    }
788  }
789
790
791  public static class ClientTrustStoreInitializationException extends RuntimeException {
792
793    public ClientTrustStoreInitializationException(String message, Throwable cause) {
794      super(message, cause);
795    }
796  }
797}