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.client;
019
020import static org.apache.hadoop.hbase.util.FutureUtils.addListener;
021
022import java.io.IOException;
023import java.lang.reflect.Constructor;
024import java.security.PrivilegedExceptionAction;
025import java.util.concurrent.CompletableFuture;
026import java.util.concurrent.ExecutorService;
027import org.apache.hadoop.conf.Configuration;
028import org.apache.hadoop.hbase.AuthUtil;
029import org.apache.hadoop.hbase.HBaseConfiguration;
030import org.apache.hadoop.hbase.security.User;
031import org.apache.hadoop.hbase.security.UserProvider;
032import org.apache.hadoop.hbase.util.ReflectionUtils;
033import org.apache.yetus.audience.InterfaceAudience;
034
035/**
036 * A non-instantiable class that manages creation of {@link Connection}s. Managing the lifecycle of
037 * the {@link Connection}s to the cluster is the responsibility of the caller. From a
038 * {@link Connection}, {@link Table} implementations are retrieved with
039 * {@link Connection#getTable(org.apache.hadoop.hbase.TableName)}. Example:
040 *
041 * <pre>
042 * Connection connection = ConnectionFactory.createConnection(config);
043 * Table table = connection.getTable(TableName.valueOf("table1"));
044 * try {
045 *   // Use the table as needed, for a single operation and a single thread
046 * } finally {
047 *   table.close();
048 *   connection.close();
049 * }
050 * </pre>
051 *
052 * Since 2.2.0, Connection created by ConnectionFactory can contain user-specified kerberos
053 * credentials if caller has following two configurations set:
054 * <ul>
055 * <li>hbase.client.keytab.file, points to a valid keytab on the local filesystem
056 * <li>hbase.client.kerberos.principal, gives the Kerberos principal to use
057 * </ul>
058 * By this way, caller can directly connect to kerberized cluster without caring login and
059 * credentials renewal logic in application.
060 *
061 * <pre>
062 * </pre>
063 *
064 * Similarly, {@link Connection} also returns {@link Admin} and {@link RegionLocator}
065 * implementations.
066 * @see Connection
067 * @since 0.99.0
068 */
069@InterfaceAudience.Public
070public class ConnectionFactory {
071
072  public static final String HBASE_CLIENT_ASYNC_CONNECTION_IMPL =
073    "hbase.client.async.connection.impl";
074
075  /** No public c.tors */
076  protected ConnectionFactory() {
077  }
078
079  /**
080   * Create a new Connection instance using default HBaseConfiguration. Connection encapsulates all
081   * housekeeping for a connection to the cluster. All tables and interfaces created from returned
082   * connection share zookeeper connection, meta cache, and connections to region servers and
083   * masters. <br>
084   * The caller is responsible for calling {@link Connection#close()} on the returned connection
085   * instance. Typical usage:
086   *
087   * <pre>
088   * Connection connection = ConnectionFactory.createConnection();
089   * Table table = connection.getTable(TableName.valueOf("mytable"));
090   * try {
091   *   table.get(...);
092   *   ...
093   * } finally {
094   *   table.close();
095   *   connection.close();
096   * }
097   * </pre>
098   *
099   * @return Connection object for <code>conf</code>
100   */
101  public static Connection createConnection() throws IOException {
102    Configuration conf = HBaseConfiguration.create();
103    return createConnection(conf, null, AuthUtil.loginClient(conf));
104  }
105
106  /**
107   * Create a new Connection instance using the passed <code>conf</code> instance. Connection
108   * encapsulates all housekeeping for a connection to the cluster. All tables and interfaces
109   * created from returned connection share zookeeper connection, meta cache, and connections to
110   * region servers and masters. <br>
111   * The caller is responsible for calling {@link Connection#close()} on the returned connection
112   * instance. Typical usage:
113   *
114   * <pre>
115   * Connection connection = ConnectionFactory.createConnection(conf);
116   * Table table = connection.getTable(TableName.valueOf("mytable"));
117   * try {
118   *   table.get(...);
119   *   ...
120   * } finally {
121   *   table.close();
122   *   connection.close();
123   * }
124   * </pre>
125   *
126   * @param conf configuration
127   * @return Connection object for <code>conf</code>
128   */
129  public static Connection createConnection(Configuration conf) throws IOException {
130    return createConnection(conf, null, AuthUtil.loginClient(conf));
131  }
132
133  /**
134   * Create a new Connection instance using the passed <code>conf</code> instance. Connection
135   * encapsulates all housekeeping for a connection to the cluster. All tables and interfaces
136   * created from returned connection share zookeeper connection, meta cache, and connections to
137   * region servers and masters. <br>
138   * The caller is responsible for calling {@link Connection#close()} on the returned connection
139   * instance. Typical usage:
140   *
141   * <pre>
142   * Connection connection = ConnectionFactory.createConnection(conf);
143   * Table table = connection.getTable(TableName.valueOf("mytable"));
144   * try {
145   *   table.get(...);
146   *   ...
147   * } finally {
148   *   table.close();
149   *   connection.close();
150   * }
151   * </pre>
152   *
153   * @param conf configuration
154   * @param pool the thread pool to use for batch operations
155   * @return Connection object for <code>conf</code>
156   */
157  public static Connection createConnection(Configuration conf, ExecutorService pool)
158    throws IOException {
159    return createConnection(conf, pool, AuthUtil.loginClient(conf));
160  }
161
162  /**
163   * Create a new Connection instance using the passed <code>conf</code> instance. Connection
164   * encapsulates all housekeeping for a connection to the cluster. All tables and interfaces
165   * created from returned connection share zookeeper connection, meta cache, and connections to
166   * region servers and masters. <br>
167   * The caller is responsible for calling {@link Connection#close()} on the returned connection
168   * instance. Typical usage:
169   *
170   * <pre>
171   * Connection connection = ConnectionFactory.createConnection(conf);
172   * Table table = connection.getTable(TableName.valueOf("table1"));
173   * try {
174   *   table.get(...);
175   *   ...
176   * } finally {
177   *   table.close();
178   *   connection.close();
179   * }
180   * </pre>
181   *
182   * @param conf configuration
183   * @param user the user the connection is for
184   * @return Connection object for <code>conf</code>
185   */
186  public static Connection createConnection(Configuration conf, User user) throws IOException {
187    return createConnection(conf, null, user);
188  }
189
190  /**
191   * Create a new Connection instance using the passed <code>conf</code> instance. Connection
192   * encapsulates all housekeeping for a connection to the cluster. All tables and interfaces
193   * created from returned connection share zookeeper connection, meta cache, and connections to
194   * region servers and masters. <br>
195   * The caller is responsible for calling {@link Connection#close()} on the returned connection
196   * instance. Typical usage:
197   *
198   * <pre>
199   * Connection connection = ConnectionFactory.createConnection(conf);
200   * Table table = connection.getTable(TableName.valueOf("table1"));
201   * try {
202   *   table.get(...);
203   *   ...
204   * } finally {
205   *   table.close();
206   *   connection.close();
207   * }
208   * </pre>
209   *
210   * @param conf configuration
211   * @param user the user the connection is for
212   * @param pool the thread pool to use for batch operations
213   * @return Connection object for <code>conf</code>
214   */
215  public static Connection createConnection(Configuration conf, ExecutorService pool,
216    final User user) throws IOException {
217    String className = conf.get(ClusterConnection.HBASE_CLIENT_CONNECTION_IMPL,
218      ConnectionImplementation.class.getName());
219    Class<?> clazz;
220    try {
221      clazz = Class.forName(className);
222    } catch (ClassNotFoundException e) {
223      throw new IOException(e);
224    }
225    try {
226      // Default HCM#HCI is not accessible; make it so before invoking.
227      Constructor<?> constructor =
228        clazz.getDeclaredConstructor(Configuration.class, ExecutorService.class, User.class);
229      constructor.setAccessible(true);
230      return user.runAs((PrivilegedExceptionAction<
231        Connection>) () -> (Connection) constructor.newInstance(conf, pool, user));
232    } catch (Exception e) {
233      throw new IOException(e);
234    }
235  }
236
237  /**
238   * Call {@link #createAsyncConnection(Configuration)} using default HBaseConfiguration.
239   * @see #createAsyncConnection(Configuration)
240   * @return AsyncConnection object wrapped by CompletableFuture
241   */
242  public static CompletableFuture<AsyncConnection> createAsyncConnection() {
243    return createAsyncConnection(HBaseConfiguration.create());
244  }
245
246  /**
247   * Call {@link #createAsyncConnection(Configuration, User)} using the given {@code conf} and a
248   * User object created by {@link UserProvider}. The given {@code conf} will also be used to
249   * initialize the {@link UserProvider}.
250   * @param conf configuration
251   * @return AsyncConnection object wrapped by CompletableFuture
252   * @see #createAsyncConnection(Configuration, User)
253   * @see UserProvider
254   */
255  public static CompletableFuture<AsyncConnection> createAsyncConnection(Configuration conf) {
256    User user;
257    try {
258      user = AuthUtil.loginClient(conf);
259    } catch (IOException e) {
260      CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
261      future.completeExceptionally(e);
262      return future;
263    }
264    return createAsyncConnection(conf, user);
265  }
266
267  /**
268   * Create a new AsyncConnection instance using the passed {@code conf} and {@code user}.
269   * AsyncConnection encapsulates all housekeeping for a connection to the cluster. All tables and
270   * interfaces created from returned connection share zookeeper connection, meta cache, and
271   * connections to region servers and masters.
272   * <p>
273   * The caller is responsible for calling {@link AsyncConnection#close()} on the returned
274   * connection instance.
275   * <p>
276   * Usually you should only create one AsyncConnection instance in your code and use it everywhere
277   * as it is thread safe.
278   * @param conf configuration
279   * @param user the user the asynchronous connection is for
280   * @return AsyncConnection object wrapped by CompletableFuture
281   */
282  public static CompletableFuture<AsyncConnection> createAsyncConnection(Configuration conf,
283    final User user) {
284    CompletableFuture<AsyncConnection> future = new CompletableFuture<>();
285    ConnectionRegistry registry = ConnectionRegistryFactory.getRegistry(conf);
286    addListener(registry.getClusterId(), (clusterId, error) -> {
287      if (error != null) {
288        registry.close();
289        future.completeExceptionally(error);
290        return;
291      }
292      if (clusterId == null) {
293        registry.close();
294        future.completeExceptionally(new IOException("clusterid came back null"));
295        return;
296      }
297      Class<? extends AsyncConnection> clazz = conf.getClass(HBASE_CLIENT_ASYNC_CONNECTION_IMPL,
298        AsyncConnectionImpl.class, AsyncConnection.class);
299      try {
300        future.complete(
301          user.runAs((PrivilegedExceptionAction<? extends AsyncConnection>) () -> ReflectionUtils
302            .newInstance(clazz, conf, registry, clusterId, user)));
303      } catch (Exception e) {
304        registry.close();
305        future.completeExceptionally(e);
306      }
307    });
308    return future;
309  }
310}