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