001/**
002 *
003
004 * Licensed to the Apache Software Foundation (ASF) under one
005 * or more contributor license agreements.  See the NOTICE file
006 * distributed with this work for additional information
007 * regarding copyright ownership.  The ASF licenses this file
008 * to you under the Apache License, Version 2.0 (the
009 * "License"); you may not use this file except in compliance
010 * with the License.  You may obtain a copy of the License at
011 *
012 *     http://www.apache.org/licenses/LICENSE-2.0
013 *
014 * Unless required by applicable law or agreed to in writing, software
015 * distributed under the License is distributed on an "AS IS" BASIS,
016 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
017 * See the License for the specific language governing permissions and
018 * limitations under the License.
019 */
020package org.apache.hadoop.hbase.client;
021
022import java.io.IOException;
023import java.util.List;
024import org.apache.hadoop.conf.Configuration;
025import org.apache.hadoop.hbase.HRegionLocation;
026import org.apache.hadoop.hbase.MasterNotRunningException;
027import org.apache.hadoop.hbase.RegionLocations;
028import org.apache.hadoop.hbase.ServerName;
029import org.apache.hadoop.hbase.TableName;
030import org.apache.hadoop.hbase.ZooKeeperConnectionException;
031import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy;
032import org.apache.hadoop.hbase.ipc.RpcControllerFactory;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hadoop.hbase.shaded.protobuf.generated.AdminProtos.AdminService;
036import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.ClientService;
037
038/** Internal methods on Connection that should not be used by user code. */
039@InterfaceAudience.Private
040// NOTE: Although this class is public, this class is meant to be used directly from internal
041// classes and unit tests only.
042public interface ClusterConnection extends Connection {
043
044  /**
045   * Key for configuration in Configuration whose value is the class we implement making a
046   * new Connection instance.
047   */
048  String HBASE_CLIENT_CONNECTION_IMPL = "hbase.client.connection.impl";
049
050  /**
051   * @return - true if the master server is running
052   * @deprecated this has been deprecated without a replacement
053   */
054  @Deprecated
055  boolean isMasterRunning()
056      throws MasterNotRunningException, ZooKeeperConnectionException;
057
058  /**
059   * Use this api to check if the table has been created with the specified number of
060   * splitkeys which was used while creating the given table.
061   * Note : If this api is used after a table's region gets splitted, the api may return
062   * false.
063   * @param tableName
064   *          tableName
065   * @param splitKeys
066   *          splitKeys used while creating table
067   * @throws IOException
068   *           if a remote or network exception occurs
069   */
070  boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws
071      IOException;
072
073  /**
074   * A table that isTableEnabled == false and isTableDisabled == false
075   * is possible. This happens when a table has a lot of regions
076   * that must be processed.
077   * @param tableName table name
078   * @return true if the table is enabled, false otherwise
079   * @throws IOException if a remote or network exception occurs
080   */
081  boolean isTableEnabled(TableName tableName) throws IOException;
082
083  /**
084   * @param tableName table name
085   * @return true if the table is disabled, false otherwise
086   * @throws IOException if a remote or network exception occurs
087   */
088  boolean isTableDisabled(TableName tableName) throws IOException;
089
090  /**
091   * Retrieve TableState, represent current table state.
092   * @param tableName table state for
093   * @return state of the table
094   */
095  TableState getTableState(TableName tableName)  throws IOException;
096
097  /**
098   * Find the location of the region of <i>tableName</i> that <i>row</i>
099   * lives in.
100   * @param tableName name of the table <i>row</i> is in
101   * @param row row key you're trying to find the region of
102   * @return HRegionLocation that describes where to find the region in
103   *   question
104   * @throws IOException if a remote or network exception occurs
105   */
106  HRegionLocation locateRegion(final TableName tableName,
107      final byte [] row) throws IOException;
108
109  /**
110   * Allows flushing the region cache.
111   */
112  void clearRegionCache();
113
114  void cacheLocation(final TableName tableName, final RegionLocations location);
115
116  /**
117   * Allows flushing the region cache of all locations that pertain to
118   * <code>tableName</code>
119   * @param tableName Name of the table whose regions we are to remove from
120   *   cache.
121   */
122  void clearRegionCache(final TableName tableName);
123
124  /**
125   * Deletes cached locations for the specific region.
126   * @param location The location object for the region, to be purged from cache.
127   */
128  void deleteCachedRegionLocation(final HRegionLocation location);
129
130  /**
131   * Find the location of the region of <i>tableName</i> that <i>row</i>
132   * lives in, ignoring any value that might be in the cache.
133   * @param tableName name of the table <i>row</i> is in
134   * @param row row key you're trying to find the region of
135   * @return HRegionLocation that describes where to find the region in
136   *   question
137   * @throws IOException if a remote or network exception occurs
138   */
139  HRegionLocation relocateRegion(final TableName tableName,
140      final byte [] row) throws IOException;
141
142  /**
143   * Find the location of the region of <i>tableName</i> that <i>row</i>
144   * lives in, ignoring any value that might be in the cache.
145   * @param tableName name of the table <i>row</i> is in
146   * @param row row key you're trying to find the region of
147   * @param replicaId the replicaId of the region
148   * @return RegionLocations that describe where to find the region in
149   *   question
150   * @throws IOException if a remote or network exception occurs
151   */
152  RegionLocations relocateRegion(final TableName tableName,
153      final byte [] row, int replicaId) throws IOException;
154
155  /**
156   * Update the location cache. This is used internally by HBase, in most cases it should not be
157   *  used by the client application.
158   * @param tableName the table name
159   * @param regionName the region name
160   * @param rowkey the row
161   * @param exception the exception if any. Can be null.
162   * @param source the previous location
163   */
164  void updateCachedLocations(TableName tableName, byte[] regionName, byte[] rowkey,
165                                    Object exception, ServerName source);
166
167  /**
168   * Gets the location of the region of <i>regionName</i>.
169   * @param regionName name of the region to locate
170   * @return HRegionLocation that describes where to find the region in
171   *   question
172   * @throws IOException if a remote or network exception occurs
173   */
174  HRegionLocation locateRegion(final byte[] regionName)
175  throws IOException;
176
177  /**
178   * Gets the locations of all regions in the specified table, <i>tableName</i>.
179   * @param tableName table to get regions of
180   * @return list of region locations for all regions of table
181   * @throws IOException if IO failure occurs
182   */
183  List<HRegionLocation> locateRegions(final TableName tableName) throws IOException;
184
185  /**
186   * Gets the locations of all regions in the specified table, <i>tableName</i>.
187   * @param tableName table to get regions of
188   * @param useCache Should we use the cache to retrieve the region information.
189   * @param offlined True if we are to include offlined regions, false and we'll leave out offlined
190   *          regions from returned list.
191   * @return list of region locations for all regions of table
192   * @throws IOException if IO failure occurs
193   */
194  List<HRegionLocation> locateRegions(final TableName tableName,
195      final boolean useCache,
196      final boolean offlined) throws IOException;
197
198  /**
199   *
200   * @param tableName table to get regions of
201   * @param row the row
202   * @param useCache Should we use the cache to retrieve the region information.
203   * @param retry do we retry
204   * @return region locations for this row.
205   * @throws IOException if IO failure occurs
206   */
207  RegionLocations locateRegion(TableName tableName,
208                               byte[] row, boolean useCache, boolean retry) throws IOException;
209
210 /**
211  *
212  * @param tableName table to get regions of
213  * @param row the row
214  * @param useCache Should we use the cache to retrieve the region information.
215  * @param retry do we retry
216  * @param replicaId the replicaId for the region
217  * @return region locations for this row.
218  * @throws IOException if IO failure occurs
219  */
220  RegionLocations locateRegion(TableName tableName, byte[] row, boolean useCache, boolean retry,
221     int replicaId) throws IOException;
222
223  /**
224   * Returns a {@link MasterKeepAliveConnection} to the active master
225   */
226  MasterKeepAliveConnection getMaster() throws IOException;
227
228  /**
229   * Get the admin service for master.
230   */
231  AdminService.BlockingInterface getAdminForMaster() throws IOException;
232
233  /**
234   * Establishes a connection to the region server at the specified address.
235   * @param serverName the region server to connect to
236   * @return proxy for HRegionServer
237   * @throws IOException if a remote or network exception occurs
238   */
239  AdminService.BlockingInterface getAdmin(final ServerName serverName) throws IOException;
240
241  /**
242   * Establishes a connection to the region server at the specified address, and returns
243   * a region client protocol.
244   *
245   * @param serverName the region server to connect to
246   * @return ClientProtocol proxy for RegionServer
247   * @throws IOException if a remote or network exception occurs
248   *
249   */
250  ClientService.BlockingInterface getClient(final ServerName serverName) throws IOException;
251
252  /**
253   * Find region location hosting passed row
254   * @param tableName table name
255   * @param row Row to find.
256   * @param reload If true do not use cache, otherwise bypass.
257   * @return Location of row.
258   * @throws IOException if a remote or network exception occurs
259   */
260  HRegionLocation getRegionLocation(TableName tableName, byte[] row, boolean reload)
261      throws IOException;
262
263  /**
264   * Clear any caches that pertain to server name <code>sn</code>.
265   * @param sn A server name
266   */
267  void clearCaches(final ServerName sn);
268
269  /**
270   * @return Nonce generator for this ClusterConnection; may be null if disabled in configuration.
271   */
272  NonceGenerator getNonceGenerator();
273
274  /**
275   * @return Default AsyncProcess associated with this connection.
276   */
277  AsyncProcess getAsyncProcess();
278
279  /**
280   * Returns a new RpcRetryingCallerFactory from the given {@link Configuration}.
281   * This RpcRetryingCallerFactory lets the users create {@link RpcRetryingCaller}s which can be
282   * intercepted with the configured {@link RetryingCallerInterceptor}
283   * @param conf configuration
284   * @return RpcRetryingCallerFactory
285   */
286  RpcRetryingCallerFactory getNewRpcRetryingCallerFactory(Configuration conf);
287
288  /**
289   * @return Connection's RpcRetryingCallerFactory instance
290   */
291  RpcRetryingCallerFactory getRpcRetryingCallerFactory();
292
293  /**
294   * @return Connection's RpcControllerFactory instance
295   */
296  RpcControllerFactory getRpcControllerFactory();
297
298  /**
299   * @return a ConnectionConfiguration object holding parsed configuration values
300   */
301  ConnectionConfiguration getConnectionConfiguration();
302
303  /**
304   * @return the current statistics tracker associated with this connection
305   */
306  ServerStatisticTracker getStatisticsTracker();
307
308  /**
309   * @return the configured client backoff policy
310   */
311  ClientBackoffPolicy getBackoffPolicy();
312
313  /**
314   * @return the MetricsConnection instance associated with this connection.
315   */
316  MetricsConnection getConnectionMetrics();
317
318  /**
319   * @return true when this connection uses a {@link org.apache.hadoop.hbase.codec.Codec} and so
320   *         supports cell blocks.
321   */
322  boolean hasCellBlockSupport();
323
324  /**
325   * Retrieve an Hbck implementation to fix an HBase cluster.
326   * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
327   * each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
328   * is not recommended.
329   * <br>
330   * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
331   *<br>
332   * This will be used mostly by hbck tool.
333   *
334   * @return an Hbck instance for active master. Active master is fetched from the zookeeper.
335   */
336  Hbck getHbck() throws IOException;
337
338  /**
339   * Retrieve an Hbck implementation to fix an HBase cluster.
340   * The returned Hbck is not guaranteed to be thread-safe. A new instance should be created by
341   * each thread. This is a lightweight operation. Pooling or caching of the returned Hbck instance
342   * is not recommended.
343   * <br>
344   * The caller is responsible for calling {@link Hbck#close()} on the returned Hbck instance.
345   *<br>
346   * This will be used mostly by hbck tool. This may only be used to by pass getting
347   * registered master from ZK. In situations where ZK is not available or active master is not
348   * registered with ZK and user can get master address by other means, master can be explicitly
349   * specified.
350   *
351   * @param masterServer explicit {@link ServerName} for master server
352   * @return an Hbck instance for a specified master server
353   */
354  Hbck getHbck(ServerName masterServer) throws IOException;
355}