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 com.google.protobuf.RpcChannel;
021
022import java.util.Collection;
023import java.util.EnumSet;
024import java.util.List;
025import java.util.Map;
026import java.util.Optional;
027import java.util.Set;
028import java.util.concurrent.CompletableFuture;
029import java.util.function.Function;
030import java.util.regex.Pattern;
031
032import org.apache.hadoop.hbase.CacheEvictionStats;
033import org.apache.hadoop.hbase.ClusterMetrics;
034import org.apache.hadoop.hbase.ClusterMetrics.Option;
035import org.apache.hadoop.hbase.NamespaceDescriptor;
036import org.apache.hadoop.hbase.RegionMetrics;
037import org.apache.hadoop.hbase.ServerName;
038import org.apache.hadoop.hbase.TableName;
039import org.apache.hadoop.hbase.client.replication.TableCFs;
040import org.apache.hadoop.hbase.client.security.SecurityCapability;
041import org.apache.hadoop.hbase.quotas.QuotaFilter;
042import org.apache.hadoop.hbase.quotas.QuotaSettings;
043import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
044import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
045import org.apache.yetus.audience.InterfaceAudience;
046
047/**
048 * The asynchronous administrative API for HBase.
049 * @since 2.0.0
050 */
051@InterfaceAudience.Public
052public interface AsyncAdmin {
053
054  /**
055   * @param tableName Table to check.
056   * @return True if table exists already. The return value will be wrapped by a
057   *         {@link CompletableFuture}.
058   */
059  CompletableFuture<Boolean> tableExists(TableName tableName);
060
061  /**
062   * List all the userspace tables.
063   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
064   */
065  default CompletableFuture<List<TableDescriptor>> listTableDescriptors() {
066    return listTableDescriptors(false);
067  }
068
069  /**
070   * List all the tables.
071   * @param includeSysTables False to match only against userspace tables
072   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
073   */
074  CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables);
075
076  /**
077   * List all the tables matching the given pattern.
078   * @param pattern The compiled regular expression to match against
079   * @param includeSysTables False to match only against userspace tables
080   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
081   */
082  CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
083      boolean includeSysTables);
084
085  /**
086   * Get list of table descriptors by namespace.
087   * @param name namespace name
088   * @return returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
089   */
090  CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name);
091
092  /**
093   * List all of the names of userspace tables.
094   * @return a list of table names wrapped by a {@link CompletableFuture}.
095   * @see #listTableNames(Pattern, boolean)
096   */
097  default CompletableFuture<List<TableName>> listTableNames() {
098    return listTableNames(false);
099  }
100
101  /**
102   * List all of the names of tables.
103   * @param includeSysTables False to match only against userspace tables
104   * @return a list of table names wrapped by a {@link CompletableFuture}.
105   */
106  CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables);
107
108  /**
109   * List all of the names of userspace tables.
110   * @param pattern The regular expression to match against
111   * @param includeSysTables False to match only against userspace tables
112   * @return a list of table names wrapped by a {@link CompletableFuture}.
113   */
114  CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables);
115
116  /**
117   * Get list of table names by namespace.
118   * @param name namespace name
119   * @return The list of table names in the namespace wrapped by a {@link CompletableFuture}.
120   */
121  CompletableFuture<List<TableName>> listTableNamesByNamespace(String name);
122
123  /**
124   * Method for getting the tableDescriptor
125   * @param tableName as a {@link TableName}
126   * @return the read-only tableDescriptor wrapped by a {@link CompletableFuture}.
127   */
128  CompletableFuture<TableDescriptor> getDescriptor(TableName tableName);
129
130  /**
131   * Creates a new table.
132   * @param desc table descriptor for table
133   */
134  CompletableFuture<Void> createTable(TableDescriptor desc);
135
136  /**
137   * Creates a new table with the specified number of regions. The start key specified will become
138   * the end key of the first region of the table, and the end key specified will become the start
139   * key of the last region of the table (the first region has a null start key and the last region
140   * has a null end key). BigInteger math will be used to divide the key range specified into enough
141   * segments to make the required number of total regions.
142   * @param desc table descriptor for table
143   * @param startKey beginning of key range
144   * @param endKey end of key range
145   * @param numRegions the total number of regions to create
146   */
147  CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
148      int numRegions);
149
150  /**
151   * Creates a new table with an initial set of empty regions defined by the specified split keys.
152   * The total number of regions created will be the number of split keys plus one.
153   * Note : Avoid passing empty split key.
154   * @param desc table descriptor for table
155   * @param splitKeys array of split keys for the initial regions of the table
156   */
157  CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys);
158
159  /**
160   * Modify an existing table, more IRB friendly version.
161   * @param desc modified description of the table
162   */
163  CompletableFuture<Void> modifyTable(TableDescriptor desc);
164
165  /**
166   * Deletes a table.
167   * @param tableName name of table to delete
168   */
169  CompletableFuture<Void> deleteTable(TableName tableName);
170
171  /**
172   * Truncate a table.
173   * @param tableName name of table to truncate
174   * @param preserveSplits True if the splits should be preserved
175   */
176  CompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits);
177
178  /**
179   * Enable a table. The table has to be in disabled state for it to be enabled.
180   * @param tableName name of the table
181   */
182  CompletableFuture<Void> enableTable(TableName tableName);
183
184  /**
185   * Disable a table. The table has to be in enabled state for it to be disabled.
186   * @param tableName
187   */
188  CompletableFuture<Void> disableTable(TableName tableName);
189
190  /**
191   * @param tableName name of table to check
192   * @return true if table is on-line. The return value will be wrapped by a
193   *         {@link CompletableFuture}.
194   */
195  CompletableFuture<Boolean> isTableEnabled(TableName tableName);
196
197  /**
198   * @param tableName name of table to check
199   * @return true if table is off-line. The return value will be wrapped by a
200   *         {@link CompletableFuture}.
201   */
202  CompletableFuture<Boolean> isTableDisabled(TableName tableName);
203
204  /**
205   * @param tableName name of table to check
206   * @return true if all regions of the table are available. The return value will be wrapped by a
207   *         {@link CompletableFuture}.
208   */
209  CompletableFuture<Boolean> isTableAvailable(TableName tableName);
210
211  /**
212   * Use this api to check if the table has been created with the specified number of splitkeys
213   * which was used while creating the given table. Note : If this api is used after a table's
214   * region gets splitted, the api may return false. The return value will be wrapped by a
215   * {@link CompletableFuture}.
216   * @param tableName name of table to check
217   * @param splitKeys keys to check if the table has been created with all split keys
218   */
219  CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys);
220
221  /**
222   * Add a column family to an existing table.
223   * @param tableName name of the table to add column family to
224   * @param columnFamily column family descriptor of column family to be added
225   */
226  CompletableFuture<Void> addColumnFamily(TableName tableName,
227      ColumnFamilyDescriptor columnFamily);
228
229  /**
230   * Delete a column family from a table.
231   * @param tableName name of table
232   * @param columnFamily name of column family to be deleted
233   */
234  CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily);
235
236  /**
237   * Modify an existing column family on a table.
238   * @param tableName name of table
239   * @param columnFamily new column family descriptor to use
240   */
241  CompletableFuture<Void> modifyColumnFamily(TableName tableName,
242      ColumnFamilyDescriptor columnFamily);
243
244  /**
245   * Create a new namespace.
246   * @param descriptor descriptor which describes the new namespace
247   */
248  CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor);
249
250  /**
251   * Modify an existing namespace.
252   * @param descriptor descriptor which describes the new namespace
253   */
254  CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor);
255
256  /**
257   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
258   * @param name namespace name
259   */
260  CompletableFuture<Void> deleteNamespace(String name);
261
262  /**
263   * Get a namespace descriptor by name
264   * @param name name of namespace descriptor
265   * @return A descriptor wrapped by a {@link CompletableFuture}.
266   */
267  CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name);
268
269  /**
270   * List available namespace descriptors
271   * @return List of descriptors wrapped by a {@link CompletableFuture}.
272   */
273  CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors();
274
275  /**
276   * Get all the online regions on a region server.
277   */
278  CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName);
279
280  /**
281   * Get the regions of a given table.
282   */
283  CompletableFuture<List<RegionInfo>> getRegions(TableName tableName);
284
285  /**
286   * Flush a table.
287   * @param tableName table to flush
288   */
289  CompletableFuture<Void> flush(TableName tableName);
290
291  /**
292   * Flush an individual region.
293   * @param regionName region to flush
294   */
295  CompletableFuture<Void> flushRegion(byte[] regionName);
296
297  /**
298   * Flush all region on the region server.
299   * @param serverName server to flush
300   */
301  CompletableFuture<Void> flushRegionServer(ServerName serverName);
302
303  /**
304   * Compact a table. When the returned CompletableFuture is done, it only means the compact request
305   * was sent to HBase and may need some time to finish the compact operation.
306   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
307   * @param tableName table to compact
308   */
309  default CompletableFuture<Void> compact(TableName tableName) {
310    return compact(tableName, CompactType.NORMAL);
311  }
312
313  /**
314   * Compact a column family within a table. When the returned CompletableFuture is done, it only
315   * means the compact request was sent to HBase and may need some time to finish the compact
316   * operation.
317   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
318   * @param tableName table to compact
319   * @param columnFamily column family within a table. If not present, compact the table's all
320   *          column families.
321   */
322  default CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily) {
323    return compact(tableName, columnFamily, CompactType.NORMAL);
324  }
325
326  /**
327   * Compact a table. When the returned CompletableFuture is done, it only means the compact request
328   * was sent to HBase and may need some time to finish the compact operation.
329   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
330   * normal compaction type.
331   * @param tableName table to compact
332   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
333   */
334  CompletableFuture<Void> compact(TableName tableName, CompactType compactType);
335
336  /**
337   * Compact a column family within a table. When the returned CompletableFuture is done, it only
338   * means the compact request was sent to HBase and may need some time to finish the compact
339   * operation.
340   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
341   * normal compaction type.
342   * @param tableName table to compact
343   * @param columnFamily column family within a table
344   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
345   */
346  CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily,
347      CompactType compactType);
348
349  /**
350   * Compact an individual region. When the returned CompletableFuture is done, it only means the
351   * compact request was sent to HBase and may need some time to finish the compact operation.
352   * @param regionName region to compact
353   */
354  CompletableFuture<Void> compactRegion(byte[] regionName);
355
356  /**
357   * Compact a column family within a region. When the returned CompletableFuture is done, it only
358   * means the compact request was sent to HBase and may need some time to finish the compact
359   * operation.
360   * @param regionName region to compact
361   * @param columnFamily column family within a region. If not present, compact the region's all
362   *          column families.
363   */
364  CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily);
365
366  /**
367   * Major compact a table. When the returned CompletableFuture is done, it only means the compact
368   * request was sent to HBase and may need some time to finish the compact operation.
369   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
370   * @param tableName table to major compact
371   */
372  default CompletableFuture<Void> majorCompact(TableName tableName) {
373    return majorCompact(tableName, CompactType.NORMAL);
374  }
375
376  /**
377   * Major compact a column family within a table. When the returned CompletableFuture is done, it
378   * only means the compact request was sent to HBase and may need some time to finish the compact
379   * operation.
380   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
381   * normal compaction. type.
382   * @param tableName table to major compact
383   * @param columnFamily column family within a table. If not present, major compact the table's all
384   *          column families.
385   */
386  default CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily) {
387    return majorCompact(tableName, columnFamily, CompactType.NORMAL);
388  }
389
390  /**
391   * Major compact a table. When the returned CompletableFuture is done, it only means the compact
392   * request was sent to HBase and may need some time to finish the compact operation.
393   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
394   * normal compaction type.
395   * @param tableName table to major compact
396   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
397   */
398  CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType);
399
400  /**
401   * Major compact a column family within a table. When the returned CompletableFuture is done, it
402   * only means the compact request was sent to HBase and may need some time to finish the compact
403   * operation.
404   * Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
405   * @param tableName table to major compact
406   * @param columnFamily column family within a table. If not present, major compact the table's all
407   *          column families.
408   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
409   */
410  CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily,
411      CompactType compactType);
412
413  /**
414   * Major compact a region. When the returned CompletableFuture is done, it only means the compact
415   * request was sent to HBase and may need some time to finish the compact operation.
416   * @param regionName region to major compact
417   */
418  CompletableFuture<Void> majorCompactRegion(byte[] regionName);
419
420  /**
421   * Major compact a column family within region. When the returned CompletableFuture is done, it
422   * only means the compact request was sent to HBase and may need some time to finish the compact
423   * operation.
424   * @param regionName region to major compact
425   * @param columnFamily column family within a region. If not present, major compact the region's
426   *          all column families.
427   */
428  CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily);
429
430  /**
431   * Compact all regions on the region server.
432   * @param serverName the region server name
433   */
434  CompletableFuture<Void> compactRegionServer(ServerName serverName);
435
436  /**
437   * Compact all regions on the region server.
438   * @param serverName the region server name
439   */
440  CompletableFuture<Void> majorCompactRegionServer(ServerName serverName);
441
442  /**
443   * Turn the Merge switch on or off.
444   * @param on
445   * @return Previous switch value wrapped by a {@link CompletableFuture}
446   */
447  CompletableFuture<Boolean> mergeSwitch(boolean on);
448
449  /**
450   * Query the current state of the Merge switch.
451   * @return true if the switch is on, false otherwise. The return value will be wrapped by a
452   *         {@link CompletableFuture}
453   */
454  CompletableFuture<Boolean> isMergeEnabled();
455
456  /**
457   * Turn the Split switch on or off.
458   * @param on
459   * @return Previous switch value wrapped by a {@link CompletableFuture}
460   */
461  CompletableFuture<Boolean> splitSwitch(boolean on);
462
463  /**
464   * Query the current state of the Split switch.
465   * @return true if the switch is on, false otherwise. The return value will be wrapped by a
466   *         {@link CompletableFuture}
467   */
468  CompletableFuture<Boolean> isSplitEnabled();
469
470  /**
471   * Merge two regions.
472   * @param nameOfRegionA encoded or full name of region a
473   * @param nameOfRegionB encoded or full name of region b
474   * @param forcible true if do a compulsory merge, otherwise we will only merge two adjacent
475   *          regions
476   */
477  CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
478      boolean forcible);
479
480  /**
481   * Split a table. The method will execute split action for each region in table.
482   * @param tableName table to split
483   */
484  CompletableFuture<Void> split(TableName tableName);
485
486  /**
487   * Split an individual region.
488   * @param regionName region to split
489   */
490  CompletableFuture<Void> splitRegion(byte[] regionName);
491
492  /**
493   * Split a table.
494   * @param tableName table to split
495   * @param splitPoint the explicit position to split on
496   */
497  CompletableFuture<Void> split(TableName tableName, byte[] splitPoint);
498
499  /**
500   * Split an individual region.
501   * @param regionName region to split
502   * @param splitPoint the explicit position to split on. If not present, it will decide by region
503   *          server.
504   */
505  CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint);
506
507  /**
508   * @param regionName Encoded or full name of region to assign.
509   */
510  CompletableFuture<Void> assign(byte[] regionName);
511
512  /**
513   * Unassign a region from current hosting regionserver. Region will then be assigned to a
514   * regionserver chosen at random. Region could be reassigned back to the same server. Use
515   * {@link #move(byte[], ServerName)} if you want to control the region movement.
516   * @param regionName Encoded or full name of region to unassign. Will clear any existing
517   *          RegionPlan if one found.
518   * @param forcible If true, force unassign (Will remove region from regions-in-transition too if
519   *          present. If results in double assignment use hbck -fix to resolve. To be used by
520   *          experts).
521   */
522  CompletableFuture<Void> unassign(byte[] regionName, boolean forcible);
523
524  /**
525   * Offline specified region from master's in-memory state. It will not attempt to reassign the
526   * region as in unassign. This API can be used when a region not served by any region server and
527   * still online as per Master's in memory state. If this API is incorrectly used on active region
528   * then master will loose track of that region. This is a special method that should be used by
529   * experts or hbck.
530   * @param regionName Encoded or full name of region to offline
531   */
532  CompletableFuture<Void> offline(byte[] regionName);
533
534  /**
535   * Move the region <code>r</code> to a random server.
536   * @param regionName Encoded or full name of region to move.
537   */
538  CompletableFuture<Void> move(byte[] regionName);
539
540  /**
541   * Move the region <code>r</code> to <code>dest</code>.
542   * @param regionName Encoded or full name of region to move.
543   * @param destServerName The servername of the destination regionserver. If not present, we'll
544   *          assign to a random server. A server name is made of host, port and startcode. Here is
545   *          an example: <code> host187.example.com,60020,1289493121758</code>
546   */
547  CompletableFuture<Void> move(byte[] regionName, ServerName destServerName);
548
549  /**
550   * Apply the new quota settings.
551   * @param quota the quota settings
552   */
553  CompletableFuture<Void> setQuota(QuotaSettings quota);
554
555  /**
556   * List the quotas based on the filter.
557   * @param filter the quota settings filter
558   * @return the QuotaSetting list, which wrapped by a CompletableFuture.
559   */
560  CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter);
561
562  /**
563   * Add a new replication peer for replicating data to slave cluster
564   * @param peerId a short name that identifies the peer
565   * @param peerConfig configuration for the replication slave cluster
566   */
567  default CompletableFuture<Void> addReplicationPeer(String peerId,
568      ReplicationPeerConfig peerConfig) {
569    return addReplicationPeer(peerId, peerConfig, true);
570  }
571
572  /**
573   * Add a new replication peer for replicating data to slave cluster
574   * @param peerId a short name that identifies the peer
575   * @param peerConfig configuration for the replication slave cluster
576   * @param enabled peer state, true if ENABLED and false if DISABLED
577   */
578  CompletableFuture<Void> addReplicationPeer(String peerId,
579      ReplicationPeerConfig peerConfig, boolean enabled);
580
581  /**
582   * Remove a peer and stop the replication
583   * @param peerId a short name that identifies the peer
584   */
585  CompletableFuture<Void> removeReplicationPeer(String peerId);
586
587  /**
588   * Restart the replication stream to the specified peer
589   * @param peerId a short name that identifies the peer
590   */
591  CompletableFuture<Void> enableReplicationPeer(String peerId);
592
593  /**
594   * Stop the replication stream to the specified peer
595   * @param peerId a short name that identifies the peer
596   */
597  CompletableFuture<Void> disableReplicationPeer(String peerId);
598
599  /**
600   * Returns the configured ReplicationPeerConfig for the specified peer
601   * @param peerId a short name that identifies the peer
602   * @return ReplicationPeerConfig for the peer wrapped by a {@link CompletableFuture}.
603   */
604  CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId);
605
606  /**
607   * Update the peerConfig for the specified peer
608   * @param peerId a short name that identifies the peer
609   * @param peerConfig new config for the peer
610   */
611  CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
612      ReplicationPeerConfig peerConfig);
613
614  /**
615   * Append the replicable table-cf config of the specified peer
616   * @param peerId a short that identifies the cluster
617   * @param tableCfs A map from tableName to column family names
618   */
619  CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
620      Map<TableName, List<String>> tableCfs);
621
622  /**
623   * Remove some table-cfs from config of the specified peer
624   * @param peerId a short name that identifies the cluster
625   * @param tableCfs A map from tableName to column family names
626   */
627  CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
628      Map<TableName, List<String>> tableCfs);
629
630  /**
631   * Return a list of replication peers.
632   * @return a list of replication peers description. The return value will be wrapped by a
633   *         {@link CompletableFuture}.
634   */
635  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers();
636
637  /**
638   * Return a list of replication peers.
639   * @param pattern The compiled regular expression to match peer id
640   * @return a list of replication peers description. The return value will be wrapped by a
641   *         {@link CompletableFuture}.
642   */
643  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern);
644
645  /**
646   * Find all table and column families that are replicated from this cluster
647   * @return the replicated table-cfs list of this cluster. The return value will be wrapped by a
648   *         {@link CompletableFuture}.
649   */
650  CompletableFuture<List<TableCFs>> listReplicatedTableCFs();
651
652  /**
653   * Enable a table's replication switch.
654   * @param tableName name of the table
655   */
656  CompletableFuture<Void> enableTableReplication(TableName tableName);
657
658  /**
659   * Disable a table's replication switch.
660   * @param tableName name of the table
661   */
662  CompletableFuture<Void> disableTableReplication(TableName tableName);
663
664  /**
665   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
666   * taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique
667   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
668   * a different type or with different parameters) will fail with a
669   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
670   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
671   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
672   * @param snapshotName name of the snapshot to be created
673   * @param tableName name of the table for which snapshot is created
674   */
675  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName) {
676    return snapshot(snapshotName, tableName, SnapshotType.FLUSH);
677  }
678
679  /**
680   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
681   * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with
682   * different parameters) will fail with a
683   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
684   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
685   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
686   * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
687   *          snapshots stored on the cluster
688   * @param tableName name of the table to snapshot
689   * @param type type of snapshot to take
690   */
691  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
692      SnapshotType type) {
693    return snapshot(new SnapshotDescription(snapshotName, tableName, type));
694  }
695
696  /**
697   * Take a snapshot and wait for the server to complete that snapshot asynchronously. Only a single
698   * snapshot should be taken at a time for an instance of HBase, or results may be undefined (you
699   * can tell multiple HBase clusters to snapshot at the same time, but only one at a time for a
700   * single cluster). Snapshots are considered unique based on <b>the name of the snapshot</b>.
701   * Attempts to take a snapshot with the same name (even a different type or with different
702   * parameters) will fail with a {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException}
703   * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
704   * HBase. See {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
705   * You should probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you
706   * are sure about the type of snapshot that you want to take.
707   * @param snapshot snapshot to take
708   */
709  CompletableFuture<Void> snapshot(SnapshotDescription snapshot);
710
711  /**
712   * Check the current state of the passed snapshot. There are three possible states:
713   * <ol>
714   * <li>running - returns <tt>false</tt></li>
715   * <li>finished - returns <tt>true</tt></li>
716   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
717   * </ol>
718   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
719   * run/started since the snapshot you are checking, you will receive an
720   * {@link org.apache.hadoop.hbase.snapshot.UnknownSnapshotException}.
721   * @param snapshot description of the snapshot to check
722   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
723   *         running
724   */
725  CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot);
726
727  /**
728   * Restore the specified snapshot on the original table. (The table must be disabled) If the
729   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a
730   * snapshot of the current table is taken before executing the restore operation. In case of
731   * restore failure, the failsafe snapshot will be restored. If the restore completes without
732   * problem the failsafe snapshot is deleted.
733   * @param snapshotName name of the snapshot to restore
734   */
735  CompletableFuture<Void> restoreSnapshot(String snapshotName);
736
737  /**
738   * Restore the specified snapshot on the original table. (The table must be disabled) If
739   * 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before
740   * executing the restore operation. In case of restore failure, the failsafe snapshot will be
741   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
742   * failsafe snapshot name is configurable by using the property
743   * "hbase.snapshot.restore.failsafe.name".
744   * @param snapshotName name of the snapshot to restore
745   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
746   */
747  CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot);
748
749  /**
750   * Create a new table by cloning the snapshot content.
751   * @param snapshotName name of the snapshot to be cloned
752   * @param tableName name of the table where the snapshot will be restored
753   */
754  CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName);
755
756  /**
757   * List completed snapshots.
758   * @return a list of snapshot descriptors for completed snapshots wrapped by a
759   *         {@link CompletableFuture}
760   */
761  CompletableFuture<List<SnapshotDescription>> listSnapshots();
762
763  /**
764   * List all the completed snapshots matching the given pattern.
765   * @param pattern The compiled regular expression to match against
766   * @return - returns a List of SnapshotDescription wrapped by a {@link CompletableFuture}
767   */
768  CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern);
769
770  /**
771   * List all the completed snapshots matching the given table name pattern.
772   * @param tableNamePattern The compiled table name regular expression to match against
773   * @return - returns a List of completed SnapshotDescription wrapped by a
774   *         {@link CompletableFuture}
775   */
776  CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern);
777
778  /**
779   * List all the completed snapshots matching the given table name regular expression and snapshot
780   * name regular expression.
781   * @param tableNamePattern The compiled table name regular expression to match against
782   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
783   * @return - returns a List of completed SnapshotDescription wrapped by a
784   *         {@link CompletableFuture}
785   */
786  CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern,
787      Pattern snapshotNamePattern);
788
789  /**
790   * Delete an existing snapshot.
791   * @param snapshotName name of the snapshot
792   */
793  CompletableFuture<Void> deleteSnapshot(String snapshotName);
794
795  /**
796   * Delete all existing snapshots.
797   */
798  CompletableFuture<Void> deleteSnapshots();
799
800  /**
801   * Delete existing snapshots whose names match the pattern passed.
802   * @param pattern pattern for names of the snapshot to match
803   */
804  CompletableFuture<Void> deleteSnapshots(Pattern pattern);
805
806  /**
807   * Delete all existing snapshots matching the given table name pattern.
808   * @param tableNamePattern The compiled table name regular expression to match against
809   */
810  CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern);
811
812  /**
813   * Delete all existing snapshots matching the given table name regular expression and snapshot
814   * name regular expression.
815   * @param tableNamePattern The compiled table name regular expression to match against
816   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
817   */
818  CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern,
819      Pattern snapshotNamePattern);
820
821  /**
822   * Execute a distributed procedure on a cluster.
823   * @param signature A distributed procedure is uniquely identified by its signature (default the
824   *          root ZK node name of the procedure).
825   * @param instance The instance name of the procedure. For some procedures, this parameter is
826   *          optional.
827   * @param props Property/Value pairs of properties passing to the procedure
828   */
829  CompletableFuture<Void> execProcedure(String signature, String instance,
830      Map<String, String> props);
831
832  /**
833   * Execute a distributed procedure on a cluster.
834   * @param signature A distributed procedure is uniquely identified by its signature (default the
835   *          root ZK node name of the procedure).
836   * @param instance The instance name of the procedure. For some procedures, this parameter is
837   *          optional.
838   * @param props Property/Value pairs of properties passing to the procedure
839   * @return data returned after procedure execution. null if no return data.
840   */
841  CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
842      Map<String, String> props);
843
844  /**
845   * Check the current state of the specified procedure. There are three possible states:
846   * <ol>
847   * <li>running - returns <tt>false</tt></li>
848   * <li>finished - returns <tt>true</tt></li>
849   * <li>finished with error - throws the exception that caused the procedure to fail</li>
850   * </ol>
851   * @param signature The signature that uniquely identifies a procedure
852   * @param instance The instance name of the procedure
853   * @param props Property/Value pairs of properties passing to the procedure
854   * @return true if the specified procedure is finished successfully, false if it is still running.
855   *         The value is wrapped by {@link CompletableFuture}
856   */
857  CompletableFuture<Boolean> isProcedureFinished(String signature, String instance,
858      Map<String, String> props);
859
860  /**
861   * Abort a procedure
862   * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2.
863   * @param procId ID of the procedure to abort
864   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
865   * @return true if aborted, false if procedure already completed or does not exist. the value is
866   *         wrapped by {@link CompletableFuture}
867   * @deprecated since 2.1.1 and will be removed in 4.0.0.
868   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a>
869   */
870  @Deprecated
871  CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning);
872
873  /**
874   * List procedures
875   * @return procedure list JSON wrapped by {@link CompletableFuture}
876   */
877  CompletableFuture<String> getProcedures();
878
879  /**
880   * List locks.
881   * @return lock list JSON wrapped by {@link CompletableFuture}
882   */
883  CompletableFuture<String> getLocks();
884
885  /**
886   * Mark region server(s) as decommissioned to prevent additional regions from getting
887   * assigned to them. Optionally unload the regions on the servers. If there are multiple servers
888   * to be decommissioned, decommissioning them at the same time can prevent wasteful region
889   * movements. Region unloading is asynchronous.
890   * @param servers The list of servers to decommission.
891   * @param offload True to offload the regions from the decommissioned servers
892   */
893  CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload);
894
895  /**
896   * List region servers marked as decommissioned, which can not be assigned regions.
897   * @return List of decommissioned region servers wrapped by {@link CompletableFuture}
898   */
899  CompletableFuture<List<ServerName>> listDecommissionedRegionServers();
900
901  /**
902   * Remove decommission marker from a region server to allow regions assignments. Load regions onto
903   * the server if a list of regions is given. Region loading is asynchronous.
904   * @param server The server to recommission.
905   * @param encodedRegionNames Regions to load onto the server.
906   */
907  CompletableFuture<Void> recommissionRegionServer(ServerName server,
908      List<byte[]> encodedRegionNames);
909
910  /**
911   * @return cluster status wrapped by {@link CompletableFuture}
912   */
913  CompletableFuture<ClusterMetrics> getClusterMetrics();
914
915  /**
916   * @return cluster status wrapped by {@link CompletableFuture}
917   */
918  CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options);
919
920  /**
921   * @return current master server name wrapped by {@link CompletableFuture}
922   */
923  default CompletableFuture<ServerName> getMaster() {
924    return getClusterMetrics(EnumSet.of(Option.MASTER)).thenApply(ClusterMetrics::getMasterName);
925  }
926
927  /**
928   * @return current backup master list wrapped by {@link CompletableFuture}
929   */
930  default CompletableFuture<Collection<ServerName>> getBackupMasters() {
931    return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS))
932      .thenApply(ClusterMetrics::getBackupMasterNames);
933  }
934
935  /**
936   * @return current live region servers list wrapped by {@link CompletableFuture}
937   */
938  default CompletableFuture<Collection<ServerName>> getRegionServers() {
939    return getClusterMetrics(EnumSet.of(Option.LIVE_SERVERS))
940      .thenApply(cm -> cm.getLiveServerMetrics().keySet());
941  }
942
943  /**
944   * @return a list of master coprocessors wrapped by {@link CompletableFuture}
945   */
946  default CompletableFuture<List<String>> getMasterCoprocessorNames() {
947    return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
948        .thenApply(ClusterMetrics::getMasterCoprocessorNames);
949  }
950
951  /**
952   * Get the info port of the current master if one is available.
953   * @return master info port
954   */
955  default CompletableFuture<Integer> getMasterInfoPort() {
956    return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).thenApply(
957      ClusterMetrics::getMasterInfoPort);
958  }
959
960  /**
961   * Shuts down the HBase cluster.
962   */
963  CompletableFuture<Void> shutdown();
964
965  /**
966   * Shuts down the current HBase master only.
967   */
968  CompletableFuture<Void> stopMaster();
969
970  /**
971   * Stop the designated regionserver.
972   * @param serverName
973   */
974  CompletableFuture<Void> stopRegionServer(ServerName serverName);
975
976  /**
977   * Update the configuration and trigger an online config change on the regionserver.
978   * @param serverName : The server whose config needs to be updated.
979   */
980  CompletableFuture<Void> updateConfiguration(ServerName serverName);
981
982  /**
983   * Update the configuration and trigger an online config change on all the masters and
984   * regionservers.
985   */
986  CompletableFuture<Void> updateConfiguration();
987
988  /**
989   * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
990   * <p>
991   * When the returned CompletableFuture is done, it only means the rollWALWriter request was sent
992   * to the region server and may need some time to finish the rollWALWriter operation. As a side
993   * effect of this call, the named region server may schedule store flushes at the request of the
994   * wal.
995   * @param serverName The servername of the region server.
996   */
997  CompletableFuture<Void> rollWALWriter(ServerName serverName);
998
999  /**
1000   * Clear compacting queues on a region server.
1001   * @param serverName
1002   * @param queues the set of queue name
1003   */
1004  CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues);
1005
1006  /**
1007   * Get a list of {@link RegionMetrics} of all regions hosted on a region seerver.
1008   * @param serverName
1009   * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
1010   */
1011  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName);
1012
1013  /**
1014   * Get a list of {@link RegionMetrics} of all regions hosted on a region seerver for a table.
1015   * @param serverName
1016   * @param tableName
1017   * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
1018   */
1019  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
1020    TableName tableName);
1021
1022  /**
1023   * Check whether master is in maintenance mode
1024   * @return true if master is in maintenance mode, false otherwise. The return value will be
1025   *         wrapped by a {@link CompletableFuture}
1026   */
1027  CompletableFuture<Boolean> isMasterInMaintenanceMode();
1028
1029  /**
1030   * Get the current compaction state of a table. It could be in a major compaction, a minor
1031   * compaction, both, or none.
1032   * @param tableName table to examine
1033   * @return the current compaction state wrapped by a {@link CompletableFuture}
1034   */
1035  default CompletableFuture<CompactionState> getCompactionState(TableName tableName) {
1036    return getCompactionState(tableName, CompactType.NORMAL);
1037  }
1038
1039  /**
1040   * Get the current compaction state of a table. It could be in a major compaction, a minor
1041   * compaction, both, or none.
1042   * @param tableName table to examine
1043   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
1044   * @return the current compaction state wrapped by a {@link CompletableFuture}
1045   */
1046  CompletableFuture<CompactionState> getCompactionState(TableName tableName,
1047      CompactType compactType);
1048
1049  /**
1050   * Get the current compaction state of region. It could be in a major compaction, a minor
1051   * compaction, both, or none.
1052   * @param regionName region to examine
1053   * @return the current compaction state wrapped by a {@link CompletableFuture}
1054   */
1055  CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName);
1056
1057  /**
1058   * Get the timestamp of the last major compaction for the passed table.
1059   * <p>
1060   * The timestamp of the oldest HFile resulting from a major compaction of that table, or not
1061   * present if no such HFile could be found.
1062   * @param tableName table to examine
1063   * @return the last major compaction timestamp wrapped by a {@link CompletableFuture}
1064   */
1065  CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName);
1066
1067  /**
1068   * Get the timestamp of the last major compaction for the passed region.
1069   * <p>
1070   * The timestamp of the oldest HFile resulting from a major compaction of that region, or not
1071   * present if no such HFile could be found.
1072   * @param regionName region to examine
1073   * @return the last major compaction timestamp wrapped by a {@link CompletableFuture}
1074   */
1075  CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName);
1076
1077  /**
1078   * @return the list of supported security capabilities. The return value will be wrapped by a
1079   *         {@link CompletableFuture}.
1080   */
1081  CompletableFuture<List<SecurityCapability>> getSecurityCapabilities();
1082
1083  /**
1084   * Turn the load balancer on or off.
1085   * @param on
1086   * @return Previous balancer value wrapped by a {@link CompletableFuture}.
1087   */
1088  CompletableFuture<Boolean> balancerSwitch(boolean on);
1089
1090  /**
1091   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
1092   * reassignments. Can NOT run for various reasons. Check logs.
1093   * @return True if balancer ran, false otherwise. The return value will be wrapped by a
1094   *         {@link CompletableFuture}.
1095   */
1096  default CompletableFuture<Boolean> balance() {
1097    return balance(false);
1098  }
1099
1100  /**
1101   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
1102   * reassignments. If there is region in transition, force parameter of true would still run
1103   * balancer. Can *not* run for other reasons. Check logs.
1104   * @param forcible whether we should force balance even if there is region in transition.
1105   * @return True if balancer ran, false otherwise. The return value will be wrapped by a
1106   *         {@link CompletableFuture}.
1107   */
1108  CompletableFuture<Boolean> balance(boolean forcible);
1109
1110  /**
1111   * Query the current state of the balancer.
1112   * @return true if the balance switch is on, false otherwise. The return value will be wrapped by a
1113   *         {@link CompletableFuture}.
1114   */
1115  CompletableFuture<Boolean> isBalancerEnabled();
1116
1117  /**
1118   * Set region normalizer on/off.
1119   * @param on whether normalizer should be on or off
1120   * @return Previous normalizer value wrapped by a {@link CompletableFuture}
1121   */
1122  CompletableFuture<Boolean> normalizerSwitch(boolean on);
1123
1124  /**
1125   * Query the current state of the region normalizer
1126   * @return true if region normalizer is on, false otherwise. The return value will be wrapped by a
1127   *         {@link CompletableFuture}
1128   */
1129  CompletableFuture<Boolean> isNormalizerEnabled();
1130
1131  /**
1132   * Invoke region normalizer. Can NOT run for various reasons. Check logs.
1133   * @return true if region normalizer ran, false otherwise. The return value will be wrapped by a
1134   *         {@link CompletableFuture}
1135   */
1136  CompletableFuture<Boolean> normalize();
1137
1138  /**
1139   * Turn the cleaner chore on/off.
1140   * @param on
1141   * @return Previous cleaner state wrapped by a {@link CompletableFuture}
1142   */
1143  CompletableFuture<Boolean> cleanerChoreSwitch(boolean on);
1144
1145  /**
1146   * Query the current state of the cleaner chore.
1147   * @return true if cleaner chore is on, false otherwise. The return value will be wrapped by
1148   *         a {@link CompletableFuture}
1149   */
1150  CompletableFuture<Boolean> isCleanerChoreEnabled();
1151
1152  /**
1153   * Ask for cleaner chore to run.
1154   * @return true if cleaner chore ran, false otherwise. The return value will be wrapped by a
1155   *         {@link CompletableFuture}
1156   */
1157  CompletableFuture<Boolean> runCleanerChore();
1158
1159  /**
1160   * Turn the catalog janitor on/off.
1161   * @param on
1162   * @return the previous state wrapped by a {@link CompletableFuture}
1163   */
1164  CompletableFuture<Boolean> catalogJanitorSwitch(boolean on);
1165
1166  /**
1167   * Query on the catalog janitor state.
1168   * @return true if the catalog janitor is on, false otherwise. The return value will be
1169   *         wrapped by a {@link CompletableFuture}
1170   */
1171  CompletableFuture<Boolean> isCatalogJanitorEnabled();
1172
1173  /**
1174   * Ask for a scan of the catalog table.
1175   * @return the number of entries cleaned. The return value will be wrapped by a
1176   *         {@link CompletableFuture}
1177   */
1178  CompletableFuture<Integer> runCatalogJanitor();
1179
1180  /**
1181   * Execute the given coprocessor call on the master.
1182   * <p>
1183   * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
1184   * one line lambda expression, like:
1185   *
1186   * <pre>
1187   * <code>
1188   * channel -> xxxService.newStub(channel)
1189   * </code>
1190   * </pre>
1191   * @param stubMaker a delegation to the actual {@code newStub} call.
1192   * @param callable a delegation to the actual protobuf rpc call. See the comment of
1193   *          {@link ServiceCaller} for more details.
1194   * @param <S> the type of the asynchronous stub
1195   * @param <R> the type of the return value
1196   * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
1197   * @see ServiceCaller
1198   */
1199  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
1200      ServiceCaller<S, R> callable);
1201
1202  /**
1203   * Execute the given coprocessor call on the given region server.
1204   * <p>
1205   * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
1206   * one line lambda expression, like:
1207   *
1208   * <pre>
1209   * <code>
1210   * channel -> xxxService.newStub(channel)
1211   * </code>
1212   * </pre>
1213   * @param stubMaker a delegation to the actual {@code newStub} call.
1214   * @param callable a delegation to the actual protobuf rpc call. See the comment of
1215   *          {@link ServiceCaller} for more details.
1216   * @param serverName the given region server
1217   * @param <S> the type of the asynchronous stub
1218   * @param <R> the type of the return value
1219   * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
1220   * @see ServiceCaller
1221   */
1222  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
1223    ServiceCaller<S, R> callable, ServerName serverName);
1224
1225  /**
1226   * List all the dead region servers.
1227   */
1228  default CompletableFuture<List<ServerName>> listDeadServers() {
1229    return this.getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS))
1230        .thenApply(ClusterMetrics::getDeadServerNames);
1231  }
1232
1233  /**
1234   * Clear dead region servers from master.
1235   * @param servers list of dead region servers.
1236   * @return - returns a list of servers that not cleared wrapped by a {@link CompletableFuture}.
1237   */
1238  CompletableFuture<List<ServerName>> clearDeadServers(final List<ServerName> servers);
1239
1240  /**
1241   * Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling
1242   * this API will drop all the cached blocks specific to a table from BlockCache. This can
1243   * significantly impact the query performance as the subsequent queries will have to retrieve the
1244   * blocks from underlying filesystem.
1245   * @param tableName table to clear block cache
1246   * @return CacheEvictionStats related to the eviction wrapped by a {@link CompletableFuture}.
1247   */
1248  CompletableFuture<CacheEvictionStats> clearBlockCache(final TableName tableName);
1249
1250  /**
1251   * Create a new table by cloning the existent table schema.
1252   *
1253   * @param tableName name of the table to be cloned
1254   * @param newTableName name of the new table where the table will be created
1255   * @param preserveSplits True if the splits should be preserved
1256   */
1257  CompletableFuture<Void>  cloneTableSchema(final TableName tableName,
1258      final TableName newTableName, final boolean preserveSplits);
1259}