2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org
.apache
.hadoop
.hbase
.client
;
20 import java
.io
.Closeable
;
21 import java
.io
.IOException
;
22 import java
.util
.concurrent
.CompletableFuture
;
23 import java
.util
.concurrent
.ExecutorService
;
24 import org
.apache
.hadoop
.conf
.Configuration
;
25 import org
.apache
.hadoop
.hbase
.HBaseInterfaceAudience
;
26 import org
.apache
.hadoop
.hbase
.ServerName
;
27 import org
.apache
.hadoop
.hbase
.TableName
;
28 import org
.apache
.yetus
.audience
.InterfaceAudience
;
31 * The asynchronous version of Connection.
34 @InterfaceAudience.Public
35 public interface AsyncConnection
extends Closeable
{
38 * Returns the {@link org.apache.hadoop.conf.Configuration} object used by this instance.
40 * The reference returned is not a copy, so any change made to it will affect this instance.
42 Configuration
getConfiguration();
45 * Retrieve a AsyncRegionLocator implementation to inspect region information on a table. The
46 * returned AsyncRegionLocator is not thread-safe, so a new instance should be created for each
47 * using thread. This is a lightweight operation. Pooling or caching of the returned
48 * AsyncRegionLocator is neither required nor desired.
49 * @param tableName Name of the table who's region is to be examined
50 * @return An AsyncRegionLocator instance
52 AsyncTableRegionLocator
getRegionLocator(TableName tableName
);
55 * Clear all the entries in the region location cache, for all the tables.
57 * If you only want to clear the cache for a specific table, use
58 * {@link AsyncTableRegionLocator#clearRegionLocationCache()}.
60 * This may cause performance issue so use it with caution.
62 void clearRegionLocationCache();
65 * Retrieve an {@link AsyncTable} implementation for accessing a table.
67 * The returned instance will use default configs. Use {@link #getTableBuilder(TableName)} if you
68 * want to customize some configs.
70 * This method no longer checks table existence. An exception will be thrown if the table does not
71 * exist only when the first operation is attempted.
73 * The returned {@code CompletableFuture} will be finished directly in the rpc framework's
74 * callback thread, so typically you should not do any time consuming work inside these methods.
75 * And also the observer style scan API will use {@link AdvancedScanResultConsumer} which is
76 * designed for experts only. Only use it when you know what you are doing.
77 * @param tableName the name of the table
78 * @return an AsyncTable to use for interactions with this table
79 * @see #getTableBuilder(TableName)
81 default AsyncTable
<AdvancedScanResultConsumer
> getTable(TableName tableName
) {
82 return getTableBuilder(tableName
).build();
86 * Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}.
88 * This method no longer checks table existence. An exception will be thrown if the table does not
89 * exist only when the first operation is attempted.
90 * @param tableName the name of the table
92 AsyncTableBuilder
<AdvancedScanResultConsumer
> getTableBuilder(TableName tableName
);
95 * Retrieve an {@link AsyncTable} implementation for accessing a table.
97 * This method no longer checks table existence. An exception will be thrown if the table does not
98 * exist only when the first operation is attempted.
99 * @param tableName the name of the table
100 * @param pool the thread pool to use for executing callback
101 * @return an AsyncTable to use for interactions with this table
103 default AsyncTable
<ScanResultConsumer
> getTable(TableName tableName
, ExecutorService pool
) {
104 return getTableBuilder(tableName
, pool
).build();
108 * Returns an {@link AsyncTableBuilder} for creating {@link AsyncTable}.
110 * This method no longer checks table existence. An exception will be thrown if the table does not
111 * exist only when the first operation is attempted.
112 * @param tableName the name of the table
113 * @param pool the thread pool to use for executing callback
115 AsyncTableBuilder
<ScanResultConsumer
> getTableBuilder(TableName tableName
, ExecutorService pool
);
118 * Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.
120 * The returned instance will use default configs. Use {@link #getAdminBuilder()} if you want to
121 * customize some configs.
123 * The admin operation's returned {@code CompletableFuture} will be finished directly in the rpc
124 * framework's callback thread, so typically you should not do any time consuming work inside
126 * @return an {@link AsyncAdmin} instance for cluster administration
128 default AsyncAdmin
getAdmin() {
129 return getAdminBuilder().build();
133 * Returns an {@link AsyncAdminBuilder} for creating {@link AsyncAdmin}.
135 * The admin operation's returned {@code CompletableFuture} will be finished directly in the rpc
136 * framework's callback thread, so typically you should not do any time consuming work inside
139 AsyncAdminBuilder
getAdminBuilder();
142 * Retrieve an {@link AsyncAdmin} implementation to administer an HBase cluster.
144 * The returned instance will use default configs. Use {@link #getAdminBuilder(ExecutorService)}
145 * if you want to customize some configs.
146 * @param pool the thread pool to use for executing callback
147 * @return an {@link AsyncAdmin} instance for cluster administration
149 default AsyncAdmin
getAdmin(ExecutorService pool
) {
150 return getAdminBuilder(pool
).build();
154 * Returns an {@link AsyncAdminBuilder} for creating {@link AsyncAdmin}.
155 * @param pool the thread pool to use for executing callback
157 AsyncAdminBuilder
getAdminBuilder(ExecutorService pool
);
160 * Retrieve an {@link AsyncBufferedMutator} for performing client-side buffering of writes.
162 * The returned instance will use default configs. Use
163 * {@link #getBufferedMutatorBuilder(TableName)} if you want to customize some configs.
164 * @param tableName the name of the table
165 * @return an {@link AsyncBufferedMutator} for the supplied tableName.
167 default AsyncBufferedMutator
getBufferedMutator(TableName tableName
) {
168 return getBufferedMutatorBuilder(tableName
).build();
172 * Returns an {@link AsyncBufferedMutatorBuilder} for creating {@link AsyncBufferedMutator}.
173 * @param tableName the name of the table
175 AsyncBufferedMutatorBuilder
getBufferedMutatorBuilder(TableName tableName
);
178 * Retrieve an {@link AsyncBufferedMutator} for performing client-side buffering of writes.
180 * The returned instance will use default configs. Use
181 * {@link #getBufferedMutatorBuilder(TableName, ExecutorService)} if you want to customize some
183 * @param tableName the name of the table
184 * @param pool the thread pool to use for executing callback
185 * @return an {@link AsyncBufferedMutator} for the supplied tableName.
187 default AsyncBufferedMutator
getBufferedMutator(TableName tableName
, ExecutorService pool
) {
188 return getBufferedMutatorBuilder(tableName
, pool
).build();
192 * Returns an {@link AsyncBufferedMutatorBuilder} for creating {@link AsyncBufferedMutator}.
193 * @param tableName the name of the table
194 * @param pool the thread pool to use for executing callback
196 AsyncBufferedMutatorBuilder
getBufferedMutatorBuilder(TableName tableName
, ExecutorService pool
);
199 * Returns whether the connection is closed or not.
200 * @return true if this connection is closed
205 * Convert this connection to a {@link Connection}.
207 * Usually we will return the same instance if you call this method multiple times so you can
208 * consider this as a light-weighted operation.
210 Connection
toConnection();
213 * Retrieve an Hbck implementation to fix an HBase cluster. The returned Hbck is not guaranteed to
214 * be thread-safe. A new instance should be created by each thread. This is a lightweight
215 * operation. Pooling or caching of the returned Hbck instance is not recommended.
217 * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
219 * This will be used mostly by hbck tool.
220 * @return an Hbck instance for active master. Active master is fetched from the zookeeper.
222 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience
.HBCK
)
223 CompletableFuture
<Hbck
> getHbck();
226 * Retrieve an Hbck implementation to fix an HBase cluster. The returned Hbck is not guaranteed to
227 * be thread-safe. A new instance should be created by each thread. This is a lightweight
228 * operation. Pooling or caching of the returned Hbck instance is not recommended.
230 * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
232 * This will be used mostly by hbck tool. This may only be used to by pass getting registered
233 * master from ZK. In situations where ZK is not available or active master is not registered with
234 * ZK and user can get master address by other means, master can be explicitly specified.
235 * @param masterServer explicit {@link ServerName} for master server
236 * @return an Hbck instance for a specified master server
238 @InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience
.HBCK
)
239 Hbck
getHbck(ServerName masterServer
) throws IOException
;