001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.rest.client;
019
020import java.io.BufferedInputStream;
021import java.io.File;
022import java.io.FileOutputStream;
023import java.io.IOException;
024import java.io.InputStream;
025import java.net.URI;
026import java.net.URISyntaxException;
027import java.net.URL;
028import java.nio.file.Files;
029import java.nio.file.Path;
030import java.security.GeneralSecurityException;
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 java.util.concurrent.ThreadLocalRandom;
041import javax.net.ssl.SSLContext;
042import org.apache.hadoop.conf.Configuration;
043import org.apache.hadoop.hbase.HBaseConfiguration;
044import org.apache.hadoop.hbase.rest.Constants;
045import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
046import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
047import org.apache.hadoop.security.authentication.client.AuthenticationException;
048import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
049import org.apache.hadoop.security.ssl.SSLFactory;
050import org.apache.hadoop.security.ssl.SSLFactory.Mode;
051import org.apache.http.Header;
052import org.apache.http.HttpHeaders;
053import org.apache.http.HttpResponse;
054import org.apache.http.HttpStatus;
055import org.apache.http.auth.AuthScope;
056import org.apache.http.auth.UsernamePasswordCredentials;
057import org.apache.http.client.HttpClient;
058import org.apache.http.client.config.RequestConfig;
059import org.apache.http.client.methods.HttpDelete;
060import org.apache.http.client.methods.HttpGet;
061import org.apache.http.client.methods.HttpHead;
062import org.apache.http.client.methods.HttpPost;
063import org.apache.http.client.methods.HttpPut;
064import org.apache.http.client.methods.HttpUriRequest;
065import org.apache.http.client.protocol.HttpClientContext;
066import org.apache.http.entity.ByteArrayEntity;
067import org.apache.http.impl.client.BasicCredentialsProvider;
068import org.apache.http.impl.client.HttpClientBuilder;
069import org.apache.http.impl.client.HttpClients;
070import org.apache.http.impl.cookie.BasicClientCookie;
071import org.apache.http.message.BasicHeader;
072import org.apache.http.ssl.SSLContexts;
073import org.apache.http.util.EntityUtils;
074import org.apache.yetus.audience.InterfaceAudience;
075import org.slf4j.Logger;
076import org.slf4j.LoggerFactory;
077
078import org.apache.hbase.thirdparty.com.google.common.io.ByteStreams;
079import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
080
081/**
082 * A wrapper around HttpClient which provides some useful function and semantics for interacting
083 * with the REST gateway.
084 */
085@InterfaceAudience.Public
086public class Client {
087  public static final Header[] EMPTY_HEADER_ARRAY = new Header[0];
088
089  private static final Logger LOG = LoggerFactory.getLogger(Client.class);
090
091  private HttpClient httpClient;
092  private Cluster cluster;
093  private Integer lastNodeId;
094  private boolean sticky = false;
095  private Configuration conf;
096  private boolean sslEnabled;
097  private HttpResponse resp;
098  private HttpGet httpGet = null;
099  private HttpClientContext stickyContext = null;
100  private BasicCredentialsProvider provider;
101  private Optional<KeyStore> trustStore;
102  private Map<String, String> extraHeaders;
103  private KerberosAuthenticator authenticator;
104
105  private static final String AUTH_COOKIE = "hadoop.auth";
106  private static final String AUTH_COOKIE_EQ = AUTH_COOKIE + "=";
107  private static final String COOKIE = "Cookie";
108
109  /**
110   * Default Constructor
111   */
112  public Client() {
113    this(null);
114  }
115
116  private void initialize(Cluster cluster, Configuration conf, boolean sslEnabled, boolean sticky,
117    Optional<KeyStore> trustStore, Optional<String> userName, Optional<String> password,
118    Optional<String> bearerToken) {
119    this.cluster = cluster;
120    this.conf = conf;
121    this.sslEnabled = sslEnabled;
122    this.trustStore = trustStore;
123    extraHeaders = new ConcurrentHashMap<>();
124    String clspath = System.getProperty("java.class.path");
125    LOG.debug("classpath " + clspath);
126    HttpClientBuilder httpClientBuilder = HttpClients.custom();
127
128    int connTimeout = this.conf.getInt(Constants.REST_CLIENT_CONN_TIMEOUT,
129      Constants.DEFAULT_REST_CLIENT_CONN_TIMEOUT);
130    int socketTimeout = this.conf.getInt(Constants.REST_CLIENT_SOCKET_TIMEOUT,
131      Constants.DEFAULT_REST_CLIENT_SOCKET_TIMEOUT);
132    RequestConfig requestConfig = RequestConfig.custom().setConnectTimeout(connTimeout)
133      .setSocketTimeout(socketTimeout).setNormalizeUri(false) // URIs should not be normalized, see
134                                                              // HBASE-26903
135      .build();
136    httpClientBuilder.setDefaultRequestConfig(requestConfig);
137
138    // Since HBASE-25267 we don't use the deprecated DefaultHttpClient anymore.
139    // The new http client would decompress the gzip content automatically.
140    // In order to keep the original behaviour of this public class, we disable
141    // automatic content compression.
142    httpClientBuilder.disableContentCompression();
143
144    if (sslEnabled && trustStore.isPresent()) {
145      try {
146        SSLContext sslcontext =
147          SSLContexts.custom().loadTrustMaterial(trustStore.get(), null).build();
148        httpClientBuilder.setSSLContext(sslcontext);
149      } catch (NoSuchAlgorithmException | KeyStoreException | KeyManagementException e) {
150        throw new ClientTrustStoreInitializationException("Error while processing truststore", e);
151      }
152    }
153
154    if (userName.isPresent() && password.isPresent()) {
155      // We want to stick to the old very limited authentication and session handling when sticky is
156      // not set
157      // to preserve backwards compatibility
158      if (!sticky) {
159        throw new IllegalArgumentException("BASIC auth is only implemented when sticky is set");
160      }
161      provider = new BasicCredentialsProvider();
162      // AuthScope.ANY is required for pre-emptive auth. We only ever use a single auth method
163      // anyway.
164      AuthScope anyAuthScope = AuthScope.ANY;
165      this.provider.setCredentials(anyAuthScope,
166        new UsernamePasswordCredentials(userName.get(), password.get()));
167    }
168
169    if (bearerToken.isPresent()) {
170      // We want to stick to the old very limited authentication and session handling when sticky is
171      // not set
172      // to preserve backwards compatibility
173      if (!sticky) {
174        throw new IllegalArgumentException("BEARER auth is only implemented when sticky is set");
175      }
176      // We could also put the header into the context or connection, but that would have the same
177      // effect.
178      extraHeaders.put(HttpHeaders.AUTHORIZATION, "Bearer " + bearerToken.get());
179    }
180
181    this.httpClient = httpClientBuilder.build();
182    setSticky(sticky);
183  }
184
185  /**
186   * Constructor This constructor will create an object using the old faulty load balancing logic.
187   * When specifying multiple servers in the cluster object, it is highly recommended to call
188   * setSticky() on the created client, or use one of the preferred constructors instead.
189   * @param cluster the cluster definition
190   */
191  public Client(Cluster cluster) {
192    this(cluster, false);
193  }
194
195  /**
196   * Constructor This constructor will create an object using the old faulty load balancing logic.
197   * When specifying multiple servers in the cluster object, it is highly recommended to call
198   * setSticky() on the created client, or use one of the preferred constructors instead.
199   * @param cluster    the cluster definition
200   * @param sslEnabled enable SSL or not
201   */
202  public Client(Cluster cluster, boolean sslEnabled) {
203    initialize(cluster, HBaseConfiguration.create(), sslEnabled, false, Optional.empty(),
204      Optional.empty(), Optional.empty(), Optional.empty());
205  }
206
207  /**
208   * Constructor This constructor will create an object using the old faulty load balancing logic.
209   * When specifying multiple servers in the cluster object, it is highly recommended to call
210   * setSticky() on the created client, or use one of the preferred constructors instead.
211   * @param cluster    the cluster definition
212   * @param conf       Configuration
213   * @param sslEnabled enable SSL or not
214   */
215  public Client(Cluster cluster, Configuration conf, boolean sslEnabled) {
216    initialize(cluster, conf, sslEnabled, false, Optional.empty(), Optional.empty(),
217      Optional.empty(), Optional.empty());
218  }
219
220  /**
221   * Constructor, allowing to define custom trust store (only for SSL connections) This constructor
222   * will create an object using the old faulty load balancing logic. When specifying multiple
223   * servers in the cluster object, it is highly recommended to call setSticky() on the created
224   * client, or use one of the preferred constructors instead.
225   * @param cluster            the cluster definition
226   * @param trustStorePath     custom trust store to use for SSL connections
227   * @param trustStorePassword password to use for custom trust store
228   * @param trustStoreType     type of custom trust store
229   * @throws ClientTrustStoreInitializationException if the trust store file can not be loaded
230   */
231  public Client(Cluster cluster, String trustStorePath, Optional<String> trustStorePassword,
232    Optional<String> trustStoreType) {
233    this(cluster, HBaseConfiguration.create(), true, trustStorePath, trustStorePassword,
234      trustStoreType);
235  }
236
237  /**
238   * Constructor that accepts an optional trustStore and authentication information for either BASIC
239   * or BEARER authentication in sticky mode, which does not use the old faulty load balancing
240   * logic, and enables correct session handling. If neither userName/password, nor the bearer token
241   * is specified, the client falls back to SPNEGO auth. The loadTrustsore static method can be used
242   * to load a local trustStore file. This is the preferred constructor to use.
243   * @param cluster     the cluster definition
244   * @param conf        HBase/Hadoop configuration
245   * @param sslEnabled  use HTTPS
246   * @param trustStore  the optional trustStore object
247   * @param userName    for BASIC auth
248   * @param password    for BASIC auth
249   * @param bearerToken for BEAERER auth
250   */
251  public Client(Cluster cluster, Configuration conf, boolean sslEnabled,
252    Optional<KeyStore> trustStore, Optional<String> userName, Optional<String> password,
253    Optional<String> bearerToken) {
254    initialize(cluster, conf, sslEnabled, true, trustStore, userName, password, bearerToken);
255  }
256
257  /**
258   * Constructor, allowing to define custom trust store (only for SSL connections) This constructor
259   * also enables sticky mode. This is a preferred constructor when not using BASIC or JWT
260   * authentication. Clients created by this will use the old faulty load balancing logic.
261   * @param cluster            the cluster definition
262   * @param conf               HBase/Hadoop Configuration
263   * @param trustStorePath     custom trust store to use for SSL connections
264   * @param trustStorePassword password to use for custom trust store
265   * @param trustStoreType     type of custom trust store
266   * @throws ClientTrustStoreInitializationException if the trust store file can not be loaded
267   */
268  public Client(Cluster cluster, Configuration conf, boolean sslEnabled, String trustStorePath,
269    Optional<String> trustStorePassword, Optional<String> trustStoreType) {
270    KeyStore trustStore = loadTruststore(trustStorePath, trustStorePassword, trustStoreType);
271    initialize(cluster, conf, sslEnabled, false, Optional.of(trustStore), Optional.empty(),
272      Optional.empty(), Optional.empty());
273  }
274
275  /**
276   * Loads a trustStore from the local fileSystem. Can be used to load the trustStore for the
277   * preferred constructor.
278   */
279  public static KeyStore loadTruststore(String trustStorePath, Optional<String> trustStorePassword,
280    Optional<String> trustStoreType) {
281
282    char[] truststorePassword = trustStorePassword.map(String::toCharArray).orElse(null);
283    String type = trustStoreType.orElse(KeyStore.getDefaultType());
284
285    KeyStore trustStore;
286    try {
287      trustStore = KeyStore.getInstance(type);
288    } catch (KeyStoreException e) {
289      throw new ClientTrustStoreInitializationException("Invalid trust store type: " + type, e);
290    }
291    try (InputStream inputStream =
292      new BufferedInputStream(Files.newInputStream(new File(trustStorePath).toPath()))) {
293      trustStore.load(inputStream, truststorePassword);
294    } catch (CertificateException | NoSuchAlgorithmException | IOException e) {
295      throw new ClientTrustStoreInitializationException("Trust store load error: " + trustStorePath,
296        e);
297    }
298    return trustStore;
299  }
300
301  /**
302   * Shut down the client. Close any open persistent connections.
303   */
304  public void shutdown() {
305  }
306
307  /** Returns the wrapped HttpClient */
308  public HttpClient getHttpClient() {
309    return httpClient;
310  }
311
312  /**
313   * Add extra headers. These extra headers will be applied to all http methods before they are
314   * removed. If any header is not used any more, client needs to remove it explicitly.
315   */
316  public void addExtraHeader(final String name, final String value) {
317    extraHeaders.put(name, value);
318  }
319
320  /**
321   * Get an extra header value.
322   */
323  public String getExtraHeader(final String name) {
324    return extraHeaders.get(name);
325  }
326
327  /**
328   * Get all extra headers (read-only).
329   */
330  public Map<String, String> getExtraHeaders() {
331    return Collections.unmodifiableMap(extraHeaders);
332  }
333
334  /**
335   * Remove an extra header.
336   */
337  public void removeExtraHeader(final String name) {
338    extraHeaders.remove(name);
339  }
340
341  /**
342   * Execute a transaction method given only the path. If sticky is false: Will select at random one
343   * of the members of the supplied cluster definition and iterate through the list until a
344   * transaction can be successfully completed. The definition of success here is a complete HTTP
345   * transaction, irrespective of result code. If sticky is true: For the first request it will
346   * select a random one of the members of the supplied cluster definition. For subsequent requests
347   * it will use the same member, and it will not automatically re-try if the call fails.
348   * @param cluster the cluster definition
349   * @param method  the transaction method
350   * @param headers HTTP header values to send
351   * @param path    the properly urlencoded path
352   * @return the HTTP response code
353   */
354  public HttpResponse executePathOnly(Cluster cluster, HttpUriRequest method, Header[] headers,
355    String path) throws IOException {
356    IOException lastException;
357    if (cluster.nodes.size() < 1) {
358      throw new IOException("Cluster is empty");
359    }
360    if (lastNodeId == null || !sticky) {
361      lastNodeId = ThreadLocalRandom.current().nextInt(cluster.nodes.size());
362    }
363    int start = lastNodeId;
364    do {
365      cluster.lastHost = cluster.nodes.get(lastNodeId);
366      try {
367        StringBuilder sb = new StringBuilder();
368        if (sslEnabled) {
369          sb.append("https://");
370        } else {
371          sb.append("http://");
372        }
373        sb.append(cluster.lastHost);
374        sb.append(path);
375        URI uri = new URI(sb.toString());
376        if (method instanceof HttpPut) {
377          HttpPut put = new HttpPut(uri);
378          put.setEntity(((HttpPut) method).getEntity());
379          put.setHeaders(method.getAllHeaders());
380          method = put;
381        } else if (method instanceof HttpGet) {
382          method = new HttpGet(uri);
383        } else if (method instanceof HttpHead) {
384          method = new HttpHead(uri);
385        } else if (method instanceof HttpDelete) {
386          method = new HttpDelete(uri);
387        } else if (method instanceof HttpPost) {
388          HttpPost post = new HttpPost(uri);
389          post.setEntity(((HttpPost) method).getEntity());
390          post.setHeaders(method.getAllHeaders());
391          method = post;
392        }
393        return executeURI(method, headers, uri.toString());
394      } catch (IOException e) {
395        lastException = e;
396      } catch (URISyntaxException use) {
397        lastException = new IOException(use);
398      }
399      if (!sticky) {
400        lastNodeId = (++lastNodeId) % cluster.nodes.size();
401      }
402      // Do not retry if sticky. Let the caller handle the error.
403    } while (!sticky && lastNodeId != start);
404    throw lastException;
405  }
406
407  /**
408   * Execute a transaction method given a complete URI.
409   * @param method  the transaction method
410   * @param headers HTTP header values to send
411   * @param uri     a properly urlencoded URI
412   * @return the HTTP response code
413   */
414  public HttpResponse executeURI(HttpUriRequest method, Header[] headers, String uri)
415    throws IOException {
416    // method.setURI(new URI(uri, true));
417    for (Map.Entry<String, String> e : extraHeaders.entrySet()) {
418      method.addHeader(e.getKey(), e.getValue());
419    }
420    if (headers != null) {
421      for (Header header : headers) {
422        method.addHeader(header);
423      }
424    }
425    long startTime = EnvironmentEdgeManager.currentTime();
426    if (resp != null) EntityUtils.consumeQuietly(resp.getEntity());
427    if (stickyContext != null) {
428      resp = httpClient.execute(method, stickyContext);
429    } else {
430      resp = httpClient.execute(method);
431    }
432    if (resp.getStatusLine().getStatusCode() == HttpStatus.SC_UNAUTHORIZED) {
433      // Authentication error
434      LOG.debug("Performing negotiation with the server.");
435      try {
436        negotiate(method, uri);
437      } catch (GeneralSecurityException e) {
438        throw new IOException(e);
439      }
440      if (stickyContext != null) {
441        resp = httpClient.execute(method, stickyContext);
442      } else {
443        resp = httpClient.execute(method);
444      }
445    }
446
447    long endTime = EnvironmentEdgeManager.currentTime();
448    if (LOG.isTraceEnabled()) {
449      LOG.trace(method.getMethod() + " " + uri + " " + resp.getStatusLine().getStatusCode() + " "
450        + resp.getStatusLine().getReasonPhrase() + " in " + (endTime - startTime) + " ms");
451    }
452    return resp;
453  }
454
455  /**
456   * Execute a transaction method. Will call either <tt>executePathOnly</tt> or <tt>executeURI</tt>
457   * depending on whether a path only is supplied in 'path', or if a complete URI is passed instead,
458   * respectively.
459   * @param cluster the cluster definition
460   * @param method  the HTTP method
461   * @param headers HTTP header values to send
462   * @param path    the properly urlencoded path or URI
463   * @return the HTTP response code
464   */
465  public HttpResponse execute(Cluster cluster, HttpUriRequest method, Header[] headers, String path)
466    throws IOException {
467    if (path.startsWith("/")) {
468      return executePathOnly(cluster, method, headers, path);
469    }
470    return executeURI(method, headers, path);
471  }
472
473  /**
474   * Initiate client side Kerberos negotiation with the server.
475   * @param method method to inject the authentication token into.
476   * @param uri    the String to parse as a URL.
477   * @throws IOException if unknown protocol is found.
478   */
479  private void negotiate(HttpUriRequest method, String uri)
480    throws IOException, GeneralSecurityException {
481    try {
482      AuthenticatedURL.Token token = new AuthenticatedURL.Token();
483      if (authenticator == null) {
484        authenticator = new KerberosAuthenticator();
485        if (trustStore.isPresent()) {
486          // The authenticator does not use Apache HttpClient, so we need to
487          // configure it separately to use the specified trustStore
488          Configuration sslConf = setupTrustStoreForHadoop(trustStore.get());
489          SSLFactory sslFactory = new SSLFactory(Mode.CLIENT, sslConf);
490          sslFactory.init();
491          authenticator.setConnectionConfigurator(sslFactory);
492        }
493      }
494      URL url = new URL(uri);
495      authenticator.authenticate(url, token);
496      if (sticky) {
497        BasicClientCookie authCookie = new BasicClientCookie("hadoop.auth", token.toString());
498        // Hadoop eats the domain even if set by server
499        authCookie.setDomain(url.getHost());
500        stickyContext.getCookieStore().addCookie(authCookie);
501      } else {
502        // session cookie is NOT set for backwards compatibility for non-sticky mode
503        // Inject the obtained negotiated token in the method cookie
504        // This is only done for this single request, the next one will trigger a new SPENGO
505        // handshake
506        injectToken(method, token);
507      }
508    } catch (AuthenticationException e) {
509      LOG.error("Failed to negotiate with the server.", e);
510      throw new IOException(e);
511    }
512  }
513
514  private Configuration setupTrustStoreForHadoop(KeyStore trustStore)
515    throws IOException, KeyStoreException, NoSuchAlgorithmException, CertificateException {
516    Path tmpDirPath = Files.createTempDirectory("hbase_rest_client_truststore");
517    File trustStoreFile = tmpDirPath.resolve("truststore.jks").toFile();
518    // Shouldn't be needed with the secure temp dir, but let's generate a password anyway
519    String password = Double.toString(Math.random());
520    try (FileOutputStream fos = new FileOutputStream(trustStoreFile)) {
521      trustStore.store(fos, password.toCharArray());
522    }
523
524    Configuration sslConf = new Configuration();
525    // Type is the Java default, we use the same JVM to read this back
526    sslConf.set("ssl.client.keystore.location", trustStoreFile.getAbsolutePath());
527    sslConf.set("ssl.client.keystore.password", password);
528    return sslConf;
529  }
530
531  /**
532   * Helper method that injects an authentication token to send with the method.
533   * @param method method to inject the authentication token into.
534   * @param token  authentication token to inject.
535   */
536  private void injectToken(HttpUriRequest method, AuthenticatedURL.Token token) {
537    String t = token.toString();
538    if (t != null) {
539      if (!t.startsWith("\"")) {
540        t = "\"" + t + "\"";
541      }
542      method.addHeader(COOKIE, AUTH_COOKIE_EQ + t);
543    }
544  }
545
546  /** Returns the cluster definition */
547  public Cluster getCluster() {
548    return cluster;
549  }
550
551  /**
552   * @param cluster the cluster definition
553   */
554  public void setCluster(Cluster cluster) {
555    this.cluster = cluster;
556  }
557
558  /**
559   * The default behaviour is load balancing by sending each request to a random host. This DOES NOT
560   * work with scans, which have state on the REST servers. Make sure sticky is set to true before
561   * attempting Scan related operations if more than one host is defined in the cluster.
562   * @return whether subsequent requests will use the same host
563   */
564  public boolean isSticky() {
565    return sticky;
566  }
567
568  /**
569   * The default behaviour is load balancing by sending each request to a random host. This DOES NOT
570   * work with scans, which have state on the REST servers. Set sticky to true before attempting
571   * Scan related operations if more than one host is defined in the cluster. Nodes must not be
572   * added or removed from the Cluster object while sticky is true. Setting the sticky flag also
573   * enables session handling, which eliminates the need to re-authenticate each request, and lets
574   * the client handle any other cookies (like the sticky cookie set by load balancers) correctly.
575   * @param sticky whether subsequent requests will use the same host
576   */
577  public void setSticky(boolean sticky) {
578    lastNodeId = null;
579    if (sticky) {
580      stickyContext = new HttpClientContext();
581      if (provider != null) {
582        stickyContext.setCredentialsProvider(provider);
583      }
584    } else {
585      stickyContext = null;
586    }
587    this.sticky = sticky;
588  }
589
590  /**
591   * Send a HEAD request
592   * @param path the path or URI
593   * @return a Response object with response detail
594   */
595  public Response head(String path) throws IOException {
596    return head(cluster, path, null);
597  }
598
599  /**
600   * Send a HEAD request
601   * @param cluster the cluster definition
602   * @param path    the path or URI
603   * @param headers the HTTP headers to include in the request
604   * @return a Response object with response detail
605   */
606  public Response head(Cluster cluster, String path, Header[] headers) throws IOException {
607    HttpHead method = new HttpHead(path);
608    try {
609      HttpResponse resp = execute(cluster, method, null, path);
610      return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), null);
611    } finally {
612      method.releaseConnection();
613    }
614  }
615
616  /**
617   * Send a GET request
618   * @param path the path or URI
619   * @return a Response object with response detail
620   */
621  public Response get(String path) throws IOException {
622    return get(cluster, path);
623  }
624
625  /**
626   * Send a GET request
627   * @param cluster the cluster definition
628   * @param path    the path or URI
629   * @return a Response object with response detail
630   */
631  public Response get(Cluster cluster, String path) throws IOException {
632    return get(cluster, path, EMPTY_HEADER_ARRAY);
633  }
634
635  /**
636   * Send a GET request
637   * @param path   the path or URI
638   * @param accept Accept header value
639   * @return a Response object with response detail
640   */
641  public Response get(String path, String accept) throws IOException {
642    return get(cluster, path, accept);
643  }
644
645  /**
646   * Send a GET request
647   * @param cluster the cluster definition
648   * @param path    the path or URI
649   * @param accept  Accept header value
650   * @return a Response object with response detail
651   */
652  public Response get(Cluster cluster, String path, String accept) throws IOException {
653    Header[] headers = new Header[1];
654    headers[0] = new BasicHeader("Accept", accept);
655    return get(cluster, path, headers);
656  }
657
658  /**
659   * Send a GET request
660   * @param path    the path or URI
661   * @param headers the HTTP headers to include in the request, <tt>Accept</tt> must be supplied
662   * @return a Response object with response detail
663   */
664  public Response get(String path, Header[] headers) throws IOException {
665    return get(cluster, path, headers);
666  }
667
668  /**
669   * Returns the response body of the HTTPResponse, if any, as an array of bytes. If response body
670   * is not available or cannot be read, returns <tt>null</tt> Note: This will cause the entire
671   * response body to be buffered in memory. A malicious server may easily exhaust all the VM
672   * memory. It is strongly recommended, to use getResponseAsStream if the content length of the
673   * response is unknown or reasonably large.
674   * @param resp HttpResponse
675   * @return The response body, null if body is empty
676   * @throws IOException If an I/O (transport) problem occurs while obtaining the response body.
677   */
678  public static byte[] getResponseBody(HttpResponse resp) throws IOException {
679    if (resp.getEntity() == null) {
680      return null;
681    }
682    InputStream instream = resp.getEntity().getContent();
683    if (instream == null) {
684      return null;
685    }
686    try {
687      long contentLength = resp.getEntity().getContentLength();
688      if (contentLength > Integer.MAX_VALUE) {
689        // guard integer cast from overflow
690        throw new IOException("Content too large to be buffered: " + contentLength + " bytes");
691      }
692      if (contentLength > 0) {
693        byte[] content = new byte[(int) contentLength];
694        ByteStreams.readFully(instream, content);
695        return content;
696      } else {
697        return ByteStreams.toByteArray(instream);
698      }
699    } finally {
700      Closeables.closeQuietly(instream);
701    }
702  }
703
704  /**
705   * Send a GET request
706   * @param c       the cluster definition
707   * @param path    the path or URI
708   * @param headers the HTTP headers to include in the request
709   * @return a Response object with response detail
710   */
711  public Response get(Cluster c, String path, Header[] headers) throws IOException {
712    if (httpGet != null) {
713      httpGet.releaseConnection();
714    }
715    httpGet = new HttpGet(path);
716    HttpResponse resp = execute(c, httpGet, headers, path);
717    return new Response(resp.getStatusLine().getStatusCode(), resp.getAllHeaders(), resp,
718      resp.getEntity() == null ? null : resp.getEntity().getContent());
719  }
720
721  /**
722   * Send a PUT request
723   * @param path        the path or URI
724   * @param contentType the content MIME type
725   * @param content     the content bytes
726   * @return a Response object with response detail
727   */
728  public Response put(String path, String contentType, byte[] content) throws IOException {
729    return put(cluster, path, contentType, content);
730  }
731
732  /**
733   * Send a PUT request
734   * @param path        the path or URI
735   * @param contentType the content MIME type
736   * @param content     the content bytes
737   * @param extraHdr    extra Header to send
738   * @return a Response object with response detail
739   */
740  public Response put(String path, String contentType, byte[] content, Header extraHdr)
741    throws IOException {
742    return put(cluster, path, contentType, content, extraHdr);
743  }
744
745  /**
746   * Send a PUT request
747   * @param cluster     the cluster definition
748   * @param path        the path or URI
749   * @param contentType the content MIME type
750   * @param content     the content bytes
751   * @return a Response object with response detail
752   * @throws IOException for error
753   */
754  public Response put(Cluster cluster, String path, String contentType, byte[] content)
755    throws IOException {
756    Header[] headers = new Header[1];
757    headers[0] = new BasicHeader("Content-Type", contentType);
758    return put(cluster, path, headers, content);
759  }
760
761  /**
762   * Send a PUT request
763   * @param cluster     the cluster definition
764   * @param path        the path or URI
765   * @param contentType the content MIME type
766   * @param content     the content bytes
767   * @param extraHdr    additional Header to send
768   * @return a Response object with response detail
769   * @throws IOException for error
770   */
771  public Response put(Cluster cluster, String path, String contentType, byte[] content,
772    Header extraHdr) throws IOException {
773    int cnt = extraHdr == null ? 1 : 2;
774    Header[] headers = new Header[cnt];
775    headers[0] = new BasicHeader("Content-Type", contentType);
776    if (extraHdr != null) {
777      headers[1] = extraHdr;
778    }
779    return put(cluster, path, headers, content);
780  }
781
782  /**
783   * Send a PUT request
784   * @param path    the path or URI
785   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be supplied
786   * @param content the content bytes
787   * @return a Response object with response detail
788   */
789  public Response put(String path, Header[] headers, byte[] content) throws IOException {
790    return put(cluster, path, headers, content);
791  }
792
793  /**
794   * Send a PUT request
795   * @param cluster the cluster definition
796   * @param path    the path or URI
797   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be supplied
798   * @param content the content bytes
799   * @return a Response object with response detail
800   */
801  public Response put(Cluster cluster, String path, Header[] headers, byte[] content)
802    throws IOException {
803    HttpPut method = new HttpPut(path);
804    try {
805      method.setEntity(new ByteArrayEntity(content));
806      HttpResponse resp = execute(cluster, method, headers, path);
807      headers = resp.getAllHeaders();
808      content = getResponseBody(resp);
809      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
810    } finally {
811      method.releaseConnection();
812    }
813  }
814
815  /**
816   * Send a POST request
817   * @param path        the path or URI
818   * @param contentType the content MIME type
819   * @param content     the content bytes
820   * @return a Response object with response detail
821   */
822  public Response post(String path, String contentType, byte[] content) throws IOException {
823    return post(cluster, path, contentType, content);
824  }
825
826  /**
827   * Send a POST request
828   * @param path        the path or URI
829   * @param contentType the content MIME type
830   * @param content     the content bytes
831   * @param extraHdr    additional Header to send
832   * @return a Response object with response detail
833   */
834  public Response post(String path, String contentType, byte[] content, Header extraHdr)
835    throws IOException {
836    return post(cluster, path, contentType, content, extraHdr);
837  }
838
839  /**
840   * Send a POST request
841   * @param cluster     the cluster definition
842   * @param path        the path or URI
843   * @param contentType the content MIME type
844   * @param content     the content bytes
845   * @return a Response object with response detail
846   * @throws IOException for error
847   */
848  public Response post(Cluster cluster, String path, String contentType, byte[] content)
849    throws IOException {
850    Header[] headers = new Header[1];
851    headers[0] = new BasicHeader("Content-Type", contentType);
852    return post(cluster, path, headers, content);
853  }
854
855  /**
856   * Send a POST request
857   * @param cluster     the cluster definition
858   * @param path        the path or URI
859   * @param contentType the content MIME type
860   * @param content     the content bytes
861   * @param extraHdr    additional Header to send
862   * @return a Response object with response detail
863   * @throws IOException for error
864   */
865  public Response post(Cluster cluster, String path, String contentType, byte[] content,
866    Header extraHdr) throws IOException {
867    int cnt = extraHdr == null ? 1 : 2;
868    Header[] headers = new Header[cnt];
869    headers[0] = new BasicHeader("Content-Type", contentType);
870    if (extraHdr != null) {
871      headers[1] = extraHdr;
872    }
873    return post(cluster, path, headers, content);
874  }
875
876  /**
877   * Send a POST request
878   * @param path    the path or URI
879   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be supplied
880   * @param content the content bytes
881   * @return a Response object with response detail
882   */
883  public Response post(String path, Header[] headers, byte[] content) throws IOException {
884    return post(cluster, path, headers, content);
885  }
886
887  /**
888   * Send a POST request
889   * @param cluster the cluster definition
890   * @param path    the path or URI
891   * @param headers the HTTP headers to include, <tt>Content-Type</tt> must be supplied
892   * @param content the content bytes
893   * @return a Response object with response detail
894   */
895  public Response post(Cluster cluster, String path, Header[] headers, byte[] content)
896    throws IOException {
897    HttpPost method = new HttpPost(path);
898    try {
899      method.setEntity(new ByteArrayEntity(content));
900      HttpResponse resp = execute(cluster, method, headers, path);
901      headers = resp.getAllHeaders();
902      content = getResponseBody(resp);
903      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
904    } finally {
905      method.releaseConnection();
906    }
907  }
908
909  /**
910   * Send a DELETE request
911   * @param path the path or URI
912   * @return a Response object with response detail
913   */
914  public Response delete(String path) throws IOException {
915    return delete(cluster, path);
916  }
917
918  /**
919   * Send a DELETE request
920   * @param path     the path or URI
921   * @param extraHdr additional Header to send
922   * @return a Response object with response detail
923   */
924  public Response delete(String path, Header extraHdr) throws IOException {
925    return delete(cluster, path, extraHdr);
926  }
927
928  /**
929   * Send a DELETE request
930   * @param cluster the cluster definition
931   * @param path    the path or URI
932   * @return a Response object with response detail
933   * @throws IOException for error
934   */
935  public Response delete(Cluster cluster, String path) throws IOException {
936    HttpDelete method = new HttpDelete(path);
937    try {
938      HttpResponse resp = execute(cluster, method, null, path);
939      Header[] headers = resp.getAllHeaders();
940      byte[] content = getResponseBody(resp);
941      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
942    } finally {
943      method.releaseConnection();
944    }
945  }
946
947  /**
948   * Send a DELETE request
949   * @param cluster the cluster definition
950   * @param path    the path or URI
951   * @return a Response object with response detail
952   * @throws IOException for error
953   */
954  public Response delete(Cluster cluster, String path, Header extraHdr) throws IOException {
955    HttpDelete method = new HttpDelete(path);
956    try {
957      Header[] headers = { extraHdr };
958      HttpResponse resp = execute(cluster, method, headers, path);
959      headers = resp.getAllHeaders();
960      byte[] content = getResponseBody(resp);
961      return new Response(resp.getStatusLine().getStatusCode(), headers, content);
962    } finally {
963      method.releaseConnection();
964    }
965  }
966
967  public static class ClientTrustStoreInitializationException extends RuntimeException {
968
969    public ClientTrustStoreInitializationException(String message, Throwable cause) {
970      super(message, cause);
971    }
972  }
973}