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 com.google.protobuf.RpcChannel;
023import java.util.Arrays;
024import java.util.Collection;
025import java.util.EnumSet;
026import java.util.HashMap;
027import java.util.List;
028import java.util.Map;
029import java.util.Optional;
030import java.util.Set;
031import java.util.concurrent.CompletableFuture;
032import java.util.function.Function;
033import java.util.regex.Pattern;
034import java.util.stream.Collectors;
035import org.apache.hadoop.hbase.CacheEvictionStats;
036import org.apache.hadoop.hbase.ClusterMetrics;
037import org.apache.hadoop.hbase.ClusterMetrics.Option;
038import org.apache.hadoop.hbase.NamespaceDescriptor;
039import org.apache.hadoop.hbase.RegionMetrics;
040import org.apache.hadoop.hbase.ServerName;
041import org.apache.hadoop.hbase.TableName;
042import org.apache.hadoop.hbase.client.replication.TableCFs;
043import org.apache.hadoop.hbase.client.security.SecurityCapability;
044import org.apache.hadoop.hbase.quotas.QuotaFilter;
045import org.apache.hadoop.hbase.quotas.QuotaSettings;
046import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView;
047import org.apache.hadoop.hbase.replication.ReplicationPeerConfig;
048import org.apache.hadoop.hbase.replication.ReplicationPeerDescription;
049import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest;
050import org.apache.hadoop.hbase.security.access.Permission;
051import org.apache.hadoop.hbase.security.access.UserPermission;
052import org.apache.yetus.audience.InterfaceAudience;
053
054import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList;
055
056/**
057 * The asynchronous administrative API for HBase.
058 * @since 2.0.0
059 */
060@InterfaceAudience.Public
061public interface AsyncAdmin {
062
063  /**
064   * Check if a table exists.
065   * @param tableName Table to check.
066   * @return True if table exists already. The return value will be wrapped by a
067   *         {@link CompletableFuture}.
068   */
069  CompletableFuture<Boolean> tableExists(TableName tableName);
070
071  /**
072   * List all the userspace tables.
073   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
074   */
075  default CompletableFuture<List<TableDescriptor>> listTableDescriptors() {
076    return listTableDescriptors(false);
077  }
078
079  /**
080   * List all the tables.
081   * @param includeSysTables False to match only against userspace tables
082   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
083   */
084  CompletableFuture<List<TableDescriptor>> listTableDescriptors(boolean includeSysTables);
085
086  /**
087   * List all the tables matching the given pattern.
088   * @param pattern          The compiled regular expression to match against
089   * @param includeSysTables False to match only against userspace tables
090   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
091   */
092  CompletableFuture<List<TableDescriptor>> listTableDescriptors(Pattern pattern,
093    boolean includeSysTables);
094
095  /**
096   * List specific tables including system tables.
097   * @param tableNames the table list to match against
098   * @return - returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
099   */
100  CompletableFuture<List<TableDescriptor>> listTableDescriptors(List<TableName> tableNames);
101
102  /**
103   * Get list of table descriptors by namespace.
104   * @param name namespace name
105   * @return returns a list of TableDescriptors wrapped by a {@link CompletableFuture}.
106   */
107  CompletableFuture<List<TableDescriptor>> listTableDescriptorsByNamespace(String name);
108
109  /**
110   * List all enabled or disabled table descriptors
111   * @param isEnabled is true means return enabled table descriptors, false means return disabled
112   *                  table descriptors
113   * @return a list of table names wrapped by a {@link CompletableFuture}.
114   */
115  CompletableFuture<List<TableDescriptor>> listTableDescriptorsByState(boolean isEnabled);
116
117  /**
118   * List all of the names of userspace tables.
119   * @return a list of table names wrapped by a {@link CompletableFuture}.
120   * @see #listTableNames(Pattern, boolean)
121   */
122  default CompletableFuture<List<TableName>> listTableNames() {
123    return listTableNames(false);
124  }
125
126  /**
127   * List all of the names of tables.
128   * @param includeSysTables False to match only against userspace tables
129   * @return a list of table names wrapped by a {@link CompletableFuture}.
130   */
131  CompletableFuture<List<TableName>> listTableNames(boolean includeSysTables);
132
133  /**
134   * List all of the names of userspace tables.
135   * @param pattern          The regular expression to match against
136   * @param includeSysTables False to match only against userspace tables
137   * @return a list of table names wrapped by a {@link CompletableFuture}.
138   */
139  CompletableFuture<List<TableName>> listTableNames(Pattern pattern, boolean includeSysTables);
140
141  /**
142   * Get list of table names by namespace.
143   * @param name namespace name
144   * @return The list of table names in the namespace wrapped by a {@link CompletableFuture}.
145   */
146  CompletableFuture<List<TableName>> listTableNamesByNamespace(String name);
147
148  /**
149   * List all enabled or disabled table names
150   * @param isEnabled is true means return enabled table names, false means return disabled table
151   *                  names
152   * @return a list of table names wrapped by a {@link CompletableFuture}.
153   */
154  CompletableFuture<List<TableName>> listTableNamesByState(boolean isEnabled);
155
156  /**
157   * Method for getting the tableDescriptor
158   * @param tableName as a {@link TableName}
159   * @return the read-only tableDescriptor wrapped by a {@link CompletableFuture}.
160   */
161  CompletableFuture<TableDescriptor> getDescriptor(TableName tableName);
162
163  /**
164   * Creates a new table.
165   * @param desc table descriptor for table
166   */
167  CompletableFuture<Void> createTable(TableDescriptor desc);
168
169  /**
170   * Creates a new table with the specified number of regions. The start key specified will become
171   * the end key of the first region of the table, and the end key specified will become the start
172   * key of the last region of the table (the first region has a null start key and the last region
173   * has a null end key). BigInteger math will be used to divide the key range specified into enough
174   * segments to make the required number of total regions.
175   * @param desc       table descriptor for table
176   * @param startKey   beginning of key range
177   * @param endKey     end of key range
178   * @param numRegions the total number of regions to create
179   */
180  CompletableFuture<Void> createTable(TableDescriptor desc, byte[] startKey, byte[] endKey,
181    int numRegions);
182
183  /**
184   * Creates a new table with an initial set of empty regions defined by the specified split keys.
185   * The total number of regions created will be the number of split keys plus one. Note : Avoid
186   * passing empty split key.
187   * @param desc      table descriptor for table
188   * @param splitKeys array of split keys for the initial regions of the table
189   */
190  CompletableFuture<Void> createTable(TableDescriptor desc, byte[][] splitKeys);
191
192  /**
193   * Modify an existing table, more IRB friendly version.
194   * @param desc modified description of the table
195   */
196  default CompletableFuture<Void> modifyTable(TableDescriptor desc) {
197    return modifyTable(desc, true);
198  }
199
200  /**
201   * Modify an existing table, more IRB friendly version.
202   * @param desc          description of the table
203   * @param reopenRegions By default, 'modifyTable' reopens all regions, potentially causing a RIT
204   *                      (Region In Transition) storm in large tables. If set to 'false', regions
205   *                      will remain unaware of the modification until they are individually
206   *                      reopened. Please note that this may temporarily result in configuration
207   *                      inconsistencies among regions.
208   */
209  CompletableFuture<Void> modifyTable(TableDescriptor desc, boolean reopenRegions);
210
211  /**
212   * Change the store file tracker of the given table.
213   * @param tableName the table you want to change
214   * @param dstSFT    the destination store file tracker
215   */
216  CompletableFuture<Void> modifyTableStoreFileTracker(TableName tableName, String dstSFT);
217
218  /**
219   * Deletes a table.
220   * @param tableName name of table to delete
221   */
222  CompletableFuture<Void> deleteTable(TableName tableName);
223
224  /**
225   * Truncate a table.
226   * @param tableName      name of table to truncate
227   * @param preserveSplits True if the splits should be preserved
228   */
229  CompletableFuture<Void> truncateTable(TableName tableName, boolean preserveSplits);
230
231  /**
232   * Enable a table. The table has to be in disabled state for it to be enabled.
233   * @param tableName name of the table
234   */
235  CompletableFuture<Void> enableTable(TableName tableName);
236
237  /**
238   * Disable a table. The table has to be in enabled state for it to be disabled.
239   */
240  CompletableFuture<Void> disableTable(TableName tableName);
241
242  /**
243   * Check if a table is enabled.
244   * @param tableName name of table to check
245   * @return true if table is on-line. The return value will be wrapped by a
246   *         {@link CompletableFuture}.
247   */
248  CompletableFuture<Boolean> isTableEnabled(TableName tableName);
249
250  /**
251   * Check if a table is disabled.
252   * @param tableName name of table to check
253   * @return true if table is off-line. The return value will be wrapped by a
254   *         {@link CompletableFuture}.
255   */
256  CompletableFuture<Boolean> isTableDisabled(TableName tableName);
257
258  /**
259   * Check if a table is available.
260   * @param tableName name of table to check
261   * @return true if all regions of the table are available. The return value will be wrapped by a
262   *         {@link CompletableFuture}.
263   */
264  CompletableFuture<Boolean> isTableAvailable(TableName tableName);
265
266  /**
267   * Use this api to check if the table has been created with the specified number of splitkeys
268   * which was used while creating the given table. Note : If this api is used after a table's
269   * region gets splitted, the api may return false. The return value will be wrapped by a
270   * {@link CompletableFuture}.
271   * @param tableName name of table to check
272   * @param splitKeys keys to check if the table has been created with all split keys
273   * @deprecated Since 2.2.0. Will be removed in 3.0.0. Use {@link #isTableAvailable(TableName)}
274   */
275  @Deprecated
276  CompletableFuture<Boolean> isTableAvailable(TableName tableName, byte[][] splitKeys);
277
278  /**
279   * Add a column family to an existing table.
280   * @param tableName    name of the table to add column family to
281   * @param columnFamily column family descriptor of column family to be added
282   */
283  CompletableFuture<Void> addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily);
284
285  /**
286   * Delete a column family from a table.
287   * @param tableName    name of table
288   * @param columnFamily name of column family to be deleted
289   */
290  CompletableFuture<Void> deleteColumnFamily(TableName tableName, byte[] columnFamily);
291
292  /**
293   * Modify an existing column family on a table.
294   * @param tableName    name of table
295   * @param columnFamily new column family descriptor to use
296   */
297  CompletableFuture<Void> modifyColumnFamily(TableName tableName,
298    ColumnFamilyDescriptor columnFamily);
299
300  /**
301   * Change the store file tracker of the given table's given family.
302   * @param tableName the table you want to change
303   * @param family    the family you want to change
304   * @param dstSFT    the destination store file tracker
305   */
306  CompletableFuture<Void> modifyColumnFamilyStoreFileTracker(TableName tableName, byte[] family,
307    String dstSFT);
308
309  /**
310   * Create a new namespace.
311   * @param descriptor descriptor which describes the new namespace
312   */
313  CompletableFuture<Void> createNamespace(NamespaceDescriptor descriptor);
314
315  /**
316   * Modify an existing namespace.
317   * @param descriptor descriptor which describes the new namespace
318   */
319  CompletableFuture<Void> modifyNamespace(NamespaceDescriptor descriptor);
320
321  /**
322   * Delete an existing namespace. Only empty namespaces (no tables) can be removed.
323   * @param name namespace name
324   */
325  CompletableFuture<Void> deleteNamespace(String name);
326
327  /**
328   * Get a namespace descriptor by name
329   * @param name name of namespace descriptor
330   * @return A descriptor wrapped by a {@link CompletableFuture}.
331   */
332  CompletableFuture<NamespaceDescriptor> getNamespaceDescriptor(String name);
333
334  /**
335   * List available namespaces
336   * @return List of namespaces wrapped by a {@link CompletableFuture}.
337   */
338  CompletableFuture<List<String>> listNamespaces();
339
340  /**
341   * List available namespace descriptors
342   * @return List of descriptors wrapped by a {@link CompletableFuture}.
343   */
344  CompletableFuture<List<NamespaceDescriptor>> listNamespaceDescriptors();
345
346  /**
347   * Get all the online regions on a region server.
348   */
349  CompletableFuture<List<RegionInfo>> getRegions(ServerName serverName);
350
351  /**
352   * Get the regions of a given table.
353   */
354  CompletableFuture<List<RegionInfo>> getRegions(TableName tableName);
355
356  /**
357   * Flush a table.
358   * @param tableName table to flush
359   */
360  CompletableFuture<Void> flush(TableName tableName);
361
362  /**
363   * Flush the specified column family stores on all regions of the passed table. This runs as a
364   * synchronous operation.
365   * @param tableName    table to flush
366   * @param columnFamily column family within a table
367   */
368  CompletableFuture<Void> flush(TableName tableName, byte[] columnFamily);
369
370  /**
371   * Flush the specified column family stores on all regions of the passed table. This runs as a
372   * synchronous operation.
373   * @param tableName      table to flush
374   * @param columnFamilies column families within a table
375   */
376  CompletableFuture<Void> flush(TableName tableName, List<byte[]> columnFamilies);
377
378  /**
379   * Flush an individual region.
380   * @param regionName region to flush
381   */
382  CompletableFuture<Void> flushRegion(byte[] regionName);
383
384  /**
385   * Flush a column family within a region.
386   * @param regionName   region to flush
387   * @param columnFamily column family within a region. If not present, flush the region's all
388   *                     column families.
389   */
390  CompletableFuture<Void> flushRegion(byte[] regionName, byte[] columnFamily);
391
392  /**
393   * Flush all region on the region server.
394   * @param serverName server to flush
395   */
396  CompletableFuture<Void> flushRegionServer(ServerName serverName);
397
398  /**
399   * Compact a table. When the returned CompletableFuture is done, it only means the compact request
400   * was sent to HBase and may need some time to finish the compact operation. Throws
401   * {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
402   * @param tableName table to compact
403   */
404  default CompletableFuture<Void> compact(TableName tableName) {
405    return compact(tableName, CompactType.NORMAL);
406  }
407
408  /**
409   * Compact a column family within a table. When the returned CompletableFuture is done, it only
410   * means the compact request was sent to HBase and may need some time to finish the compact
411   * operation. Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
412   * @param tableName    table to compact
413   * @param columnFamily column family within a table. If not present, compact the table's all
414   *                     column families.
415   */
416  default CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily) {
417    return compact(tableName, columnFamily, CompactType.NORMAL);
418  }
419
420  /**
421   * Compact a table. When the returned CompletableFuture is done, it only means the compact request
422   * was sent to HBase and may need some time to finish the compact operation. Throws
423   * {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for normal compaction
424   * type.
425   * @param tableName   table to compact
426   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
427   */
428  CompletableFuture<Void> compact(TableName tableName, CompactType compactType);
429
430  /**
431   * Compact a column family within a table. When the returned CompletableFuture is done, it only
432   * means the compact request was sent to HBase and may need some time to finish the compact
433   * operation. Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
434   * normal compaction type.
435   * @param tableName    table to compact
436   * @param columnFamily column family within a table
437   * @param compactType  {@link org.apache.hadoop.hbase.client.CompactType}
438   */
439  CompletableFuture<Void> compact(TableName tableName, byte[] columnFamily,
440    CompactType compactType);
441
442  /**
443   * Compact an individual region. When the returned CompletableFuture is done, it only means the
444   * compact request was sent to HBase and may need some time to finish the compact operation.
445   * @param regionName region to compact
446   */
447  CompletableFuture<Void> compactRegion(byte[] regionName);
448
449  /**
450   * Compact a column family within a region. When the returned CompletableFuture is done, it only
451   * means the compact request was sent to HBase and may need some time to finish the compact
452   * operation.
453   * @param regionName   region to compact
454   * @param columnFamily column family within a region. If not present, compact the region's all
455   *                     column families.
456   */
457  CompletableFuture<Void> compactRegion(byte[] regionName, byte[] columnFamily);
458
459  /**
460   * Major compact a table. When the returned CompletableFuture is done, it only means the compact
461   * request was sent to HBase and may need some time to finish the compact operation. Throws
462   * {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
463   * @param tableName table to major compact
464   */
465  default CompletableFuture<Void> majorCompact(TableName tableName) {
466    return majorCompact(tableName, CompactType.NORMAL);
467  }
468
469  /**
470   * Major compact a column family within a table. When the returned CompletableFuture is done, it
471   * only means the compact request was sent to HBase and may need some time to finish the compact
472   * operation. Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for
473   * normal compaction. type.
474   * @param tableName    table to major compact
475   * @param columnFamily column family within a table. If not present, major compact the table's all
476   *                     column families.
477   */
478  default CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily) {
479    return majorCompact(tableName, columnFamily, CompactType.NORMAL);
480  }
481
482  /**
483   * Major compact a table. When the returned CompletableFuture is done, it only means the compact
484   * request was sent to HBase and may need some time to finish the compact operation. Throws
485   * {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found for normal compaction
486   * type.
487   * @param tableName   table to major compact
488   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
489   */
490  CompletableFuture<Void> majorCompact(TableName tableName, CompactType compactType);
491
492  /**
493   * Major compact a column family within a table. When the returned CompletableFuture is done, it
494   * only means the compact request was sent to HBase and may need some time to finish the compact
495   * operation. Throws {@link org.apache.hadoop.hbase.TableNotFoundException} if table not found.
496   * @param tableName    table to major compact
497   * @param columnFamily column family within a table. If not present, major compact the table's all
498   *                     column families.
499   * @param compactType  {@link org.apache.hadoop.hbase.client.CompactType}
500   */
501  CompletableFuture<Void> majorCompact(TableName tableName, byte[] columnFamily,
502    CompactType compactType);
503
504  /**
505   * Major compact a region. When the returned CompletableFuture is done, it only means the compact
506   * request was sent to HBase and may need some time to finish the compact operation.
507   * @param regionName region to major compact
508   */
509  CompletableFuture<Void> majorCompactRegion(byte[] regionName);
510
511  /**
512   * Major compact a column family within region. When the returned CompletableFuture is done, it
513   * only means the compact request was sent to HBase and may need some time to finish the compact
514   * operation.
515   * @param regionName   region to major compact
516   * @param columnFamily column family within a region. If not present, major compact the region's
517   *                     all column families.
518   */
519  CompletableFuture<Void> majorCompactRegion(byte[] regionName, byte[] columnFamily);
520
521  /**
522   * Compact all regions on the region server.
523   * @param serverName the region server name
524   */
525  CompletableFuture<Void> compactRegionServer(ServerName serverName);
526
527  /**
528   * Compact all regions on the region server.
529   * @param serverName the region server name
530   */
531  CompletableFuture<Void> majorCompactRegionServer(ServerName serverName);
532
533  /**
534   * Turn the Merge switch on or off.
535   * @param enabled enabled or not
536   * @return Previous switch value wrapped by a {@link CompletableFuture}
537   */
538  default CompletableFuture<Boolean> mergeSwitch(boolean enabled) {
539    return mergeSwitch(enabled, false);
540  }
541
542  /**
543   * Turn the Merge switch on or off.
544   * <p/>
545   * Notice that, the method itself is always non-blocking, which means it will always return
546   * immediately. The {@code drainMerges} parameter only effects when will we complete the returned
547   * {@link CompletableFuture}.
548   * @param enabled     enabled or not
549   * @param drainMerges If <code>true</code>, it waits until current merge() call, if outstanding,
550   *                    to return.
551   * @return Previous switch value wrapped by a {@link CompletableFuture}
552   */
553  CompletableFuture<Boolean> mergeSwitch(boolean enabled, boolean drainMerges);
554
555  /**
556   * Query the current state of the Merge switch.
557   * @return true if the switch is on, false otherwise. The return value will be wrapped by a
558   *         {@link CompletableFuture}
559   */
560  CompletableFuture<Boolean> isMergeEnabled();
561
562  /**
563   * Turn the Split switch on or off.
564   * @param enabled enabled or not
565   * @return Previous switch value wrapped by a {@link CompletableFuture}
566   */
567  default CompletableFuture<Boolean> splitSwitch(boolean enabled) {
568    return splitSwitch(enabled, false);
569  }
570
571  /**
572   * Turn the Split switch on or off.
573   * <p/>
574   * Notice that, the method itself is always non-blocking, which means it will always return
575   * immediately. The {@code drainSplits} parameter only effects when will we complete the returned
576   * {@link CompletableFuture}.
577   * @param enabled     enabled or not
578   * @param drainSplits If <code>true</code>, it waits until current split() call, if outstanding,
579   *                    to return.
580   * @return Previous switch value wrapped by a {@link CompletableFuture}
581   */
582  CompletableFuture<Boolean> splitSwitch(boolean enabled, boolean drainSplits);
583
584  /**
585   * Query the current state of the Split switch.
586   * @return true if the switch is on, false otherwise. The return value will be wrapped by a
587   *         {@link CompletableFuture}
588   */
589  CompletableFuture<Boolean> isSplitEnabled();
590
591  /**
592   * Merge two regions.
593   * @param nameOfRegionA encoded or full name of region a
594   * @param nameOfRegionB encoded or full name of region b
595   * @param forcible      true if do a compulsory merge, otherwise we will only merge two adjacent
596   *                      regions
597   * @deprecated since 2.3.0 and will be removed in 4.0.0.Use {@link #mergeRegions(List, boolean)}
598   *             instead.
599   */
600  @Deprecated
601  default CompletableFuture<Void> mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB,
602    boolean forcible) {
603    return mergeRegions(Arrays.asList(nameOfRegionA, nameOfRegionB), forcible);
604  }
605
606  /**
607   * Merge multiple regions (>=2).
608   * @param nameOfRegionsToMerge encoded or full name of daughter regions
609   * @param forcible             true if do a compulsory merge, otherwise we will only merge two
610   *                             adjacent regions
611   */
612  CompletableFuture<Void> mergeRegions(List<byte[]> nameOfRegionsToMerge, boolean forcible);
613
614  /**
615   * Split a table. The method will execute split action for each region in table.
616   * @param tableName table to split
617   */
618  CompletableFuture<Void> split(TableName tableName);
619
620  /**
621   * Split an individual region.
622   * @param regionName region to split
623   */
624  CompletableFuture<Void> splitRegion(byte[] regionName);
625
626  /**
627   * Split a table.
628   * @param tableName  table to split
629   * @param splitPoint the explicit position to split on
630   */
631  CompletableFuture<Void> split(TableName tableName, byte[] splitPoint);
632
633  /**
634   * Split an individual region.
635   * @param regionName region to split
636   * @param splitPoint the explicit position to split on. If not present, it will decide by region
637   *                   server.
638   */
639  CompletableFuture<Void> splitRegion(byte[] regionName, byte[] splitPoint);
640
641  /**
642   * Truncate an individual region.
643   * @param regionName region to truncate
644   */
645  CompletableFuture<Void> truncateRegion(byte[] regionName);
646
647  /**
648   * Assign an individual region.
649   * @param regionName Encoded or full name of region to assign.
650   */
651  CompletableFuture<Void> assign(byte[] regionName);
652
653  /**
654   * Unassign a region from current hosting regionserver. Region will then be assigned to a
655   * regionserver chosen at random. Region could be reassigned back to the same server. Use
656   * {@link #move(byte[], ServerName)} if you want to control the region movement.
657   * @param regionName Encoded or full name of region to unassign.
658   */
659  CompletableFuture<Void> unassign(byte[] regionName);
660
661  /**
662   * Unassign a region from current hosting regionserver. Region will then be assigned to a
663   * regionserver chosen at random. Region could be reassigned back to the same server. Use
664   * {@link #move(byte[], ServerName)} if you want to control the region movement.
665   * @param regionName Encoded or full name of region to unassign. Will clear any existing
666   *                   RegionPlan if one found.
667   * @param forcible   If true, force unassign (Will remove region from regions-in-transition too if
668   *                   present. If results in double assignment use hbck -fix to resolve. To be used
669   *                   by experts).
670   * @deprecated since 2.4.0 and will be removed in 4.0.0. Use {@link #unassign(byte[])} instead.
671   * @see <a href="https://issues.apache.org/jira/browse/HBASE-24875">HBASE-24875</a>
672   */
673  @Deprecated
674  default CompletableFuture<Void> unassign(byte[] regionName, boolean forcible) {
675    return unassign(regionName);
676  }
677
678  /**
679   * Offline specified region from master's in-memory state. It will not attempt to reassign the
680   * region as in unassign. This API can be used when a region not served by any region server and
681   * still online as per Master's in memory state. If this API is incorrectly used on active region
682   * then master will loose track of that region. This is a special method that should be used by
683   * experts or hbck.
684   * @param regionName Encoded or full name of region to offline
685   */
686  CompletableFuture<Void> offline(byte[] regionName);
687
688  /**
689   * Move the region <code>r</code> to a random server.
690   * @param regionName Encoded or full name of region to move.
691   */
692  CompletableFuture<Void> move(byte[] regionName);
693
694  /**
695   * Move the region <code>r</code> to <code>dest</code>.
696   * @param regionName     Encoded or full name of region to move.
697   * @param destServerName The servername of the destination regionserver. If not present, we'll
698   *                       assign to a random server. A server name is made of host, port and
699   *                       startcode. Here is an example:
700   *                       <code> host187.example.com,60020,1289493121758</code>
701   */
702  CompletableFuture<Void> move(byte[] regionName, ServerName destServerName);
703
704  /**
705   * Apply the new quota settings.
706   * @param quota the quota settings
707   */
708  CompletableFuture<Void> setQuota(QuotaSettings quota);
709
710  /**
711   * List the quotas based on the filter.
712   * @param filter the quota settings filter
713   * @return the QuotaSetting list, which wrapped by a CompletableFuture.
714   */
715  CompletableFuture<List<QuotaSettings>> getQuota(QuotaFilter filter);
716
717  /**
718   * Add a new replication peer for replicating data to slave cluster
719   * @param peerId     a short name that identifies the peer
720   * @param peerConfig configuration for the replication slave cluster
721   */
722  default CompletableFuture<Void> addReplicationPeer(String peerId,
723    ReplicationPeerConfig peerConfig) {
724    return addReplicationPeer(peerId, peerConfig, true);
725  }
726
727  /**
728   * Add a new replication peer for replicating data to slave cluster
729   * @param peerId     a short name that identifies the peer
730   * @param peerConfig configuration for the replication slave cluster
731   * @param enabled    peer state, true if ENABLED and false if DISABLED
732   */
733  CompletableFuture<Void> addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig,
734    boolean enabled);
735
736  /**
737   * Remove a peer and stop the replication
738   * @param peerId a short name that identifies the peer
739   */
740  CompletableFuture<Void> removeReplicationPeer(String peerId);
741
742  /**
743   * Restart the replication stream to the specified peer
744   * @param peerId a short name that identifies the peer
745   */
746  CompletableFuture<Void> enableReplicationPeer(String peerId);
747
748  /**
749   * Stop the replication stream to the specified peer
750   * @param peerId a short name that identifies the peer
751   */
752  CompletableFuture<Void> disableReplicationPeer(String peerId);
753
754  /**
755   * Returns the configured ReplicationPeerConfig for the specified peer
756   * @param peerId a short name that identifies the peer
757   * @return ReplicationPeerConfig for the peer wrapped by a {@link CompletableFuture}.
758   */
759  CompletableFuture<ReplicationPeerConfig> getReplicationPeerConfig(String peerId);
760
761  /**
762   * Update the peerConfig for the specified peer
763   * @param peerId     a short name that identifies the peer
764   * @param peerConfig new config for the peer
765   */
766  CompletableFuture<Void> updateReplicationPeerConfig(String peerId,
767    ReplicationPeerConfig peerConfig);
768
769  /**
770   * Append the replicable table-cf config of the specified peer
771   * @param peerId   a short that identifies the cluster
772   * @param tableCfs A map from tableName to column family names
773   */
774  CompletableFuture<Void> appendReplicationPeerTableCFs(String peerId,
775    Map<TableName, List<String>> tableCfs);
776
777  /**
778   * Remove some table-cfs from config of the specified peer
779   * @param peerId   a short name that identifies the cluster
780   * @param tableCfs A map from tableName to column family names
781   */
782  CompletableFuture<Void> removeReplicationPeerTableCFs(String peerId,
783    Map<TableName, List<String>> tableCfs);
784
785  /**
786   * Return a list of replication peers.
787   * @return a list of replication peers description. The return value will be wrapped by a
788   *         {@link CompletableFuture}.
789   */
790  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers();
791
792  /**
793   * Return a list of replication peers.
794   * @param pattern The compiled regular expression to match peer id
795   * @return a list of replication peers description. The return value will be wrapped by a
796   *         {@link CompletableFuture}.
797   */
798  CompletableFuture<List<ReplicationPeerDescription>> listReplicationPeers(Pattern pattern);
799
800  /**
801   * Find all table and column families that are replicated from this cluster
802   * @return the replicated table-cfs list of this cluster. The return value will be wrapped by a
803   *         {@link CompletableFuture}.
804   */
805  CompletableFuture<List<TableCFs>> listReplicatedTableCFs();
806
807  /**
808   * Enable a table's replication switch.
809   * @param tableName name of the table
810   */
811  CompletableFuture<Void> enableTableReplication(TableName tableName);
812
813  /**
814   * Disable a table's replication switch.
815   * @param tableName name of the table
816   */
817  CompletableFuture<Void> disableTableReplication(TableName tableName);
818
819  /**
820   * Check if a replication peer is enabled.
821   * @param peerId id of replication peer to check
822   * @return true if replication peer is enabled. The return value will be wrapped by a
823   *         {@link CompletableFuture}
824   */
825  CompletableFuture<Boolean> isReplicationPeerEnabled(String peerId);
826
827  /**
828   * Enable or disable replication peer modification.
829   * <p/>
830   * This is especially useful when you want to change the replication peer storage.
831   * @param on {@code true} means enable, otherwise disable
832   * @return the previous enable/disable state wrapped by a {@link CompletableFuture}
833   */
834  default CompletableFuture<Boolean> replicationPeerModificationSwitch(boolean on) {
835    return replicationPeerModificationSwitch(on, false);
836  }
837
838  /**
839   * Enable or disable replication peer modification.
840   * <p/>
841   * This is especially useful when you want to change the replication peer storage.
842   * @param on              {@code true} means enable, otherwise disable
843   * @param drainProcedures if {@code true}, will wait until all the running replication peer
844   *                        modification procedures finish
845   * @return the previous enable/disable state wrapped by a {@link CompletableFuture}
846   */
847  CompletableFuture<Boolean> replicationPeerModificationSwitch(boolean on, boolean drainProcedures);
848
849  /**
850   * Check whether replication peer modification is enabled.
851   * @return {@code true} if modification is enabled, otherwise {@code false}, wrapped by a
852   *         {@link CompletableFuture}
853   */
854  CompletableFuture<Boolean> isReplicationPeerModificationEnabled();
855
856  /**
857   * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be
858   * taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique
859   * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even
860   * a different type or with different parameters) will fail with a
861   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
862   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
863   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
864   * @param snapshotName name of the snapshot to be created
865   * @param tableName    name of the table for which snapshot is created
866   */
867  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName) {
868    return snapshot(snapshotName, tableName, SnapshotType.FLUSH);
869  }
870
871  /**
872   * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the
873   * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with
874   * different parameters) will fail with a
875   * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate
876   * naming. Snapshot names follow the same naming constraints as tables in HBase. See
877   * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
878   * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other
879   *                     snapshots stored on the cluster
880   * @param tableName    name of the table to snapshot
881   * @param type         type of snapshot to take
882   */
883  default CompletableFuture<Void> snapshot(String snapshotName, TableName tableName,
884    SnapshotType type) {
885    return snapshot(new SnapshotDescription(snapshotName, tableName, type));
886  }
887
888  /**
889   * Take a snapshot and wait for the server to complete that snapshot asynchronously. Only a single
890   * snapshot should be taken at a time for an instance of HBase, or results may be undefined (you
891   * can tell multiple HBase clusters to snapshot at the same time, but only one at a time for a
892   * single cluster). Snapshots are considered unique based on <b>the name of the snapshot</b>.
893   * Attempts to take a snapshot with the same name (even a different type or with different
894   * parameters) will fail with a {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException}
895   * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in
896   * HBase. See {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}.
897   * You should probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} unless you
898   * are sure about the type of snapshot that you want to take.
899   * @param snapshot snapshot to take
900   */
901  CompletableFuture<Void> snapshot(SnapshotDescription snapshot);
902
903  /**
904   * Check the current state of the passed snapshot. There are three possible states:
905   * <ol>
906   * <li>running - returns <tt>false</tt></li>
907   * <li>finished - returns <tt>true</tt></li>
908   * <li>finished with error - throws the exception that caused the snapshot to fail</li>
909   * </ol>
910   * The cluster only knows about the most recent snapshot. Therefore, if another snapshot has been
911   * run/started since the snapshot you are checking, you will receive an
912   * {@link org.apache.hadoop.hbase.snapshot.UnknownSnapshotException}.
913   * @param snapshot description of the snapshot to check
914   * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still
915   *         running
916   */
917  CompletableFuture<Boolean> isSnapshotFinished(SnapshotDescription snapshot);
918
919  /**
920   * Restore the specified snapshot on the original table. (The table must be disabled) If the
921   * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to true, a
922   * snapshot of the current table is taken before executing the restore operation. In case of
923   * restore failure, the failsafe snapshot will be restored. If the restore completes without
924   * problem the failsafe snapshot is deleted.
925   * @param snapshotName name of the snapshot to restore
926   */
927  CompletableFuture<Void> restoreSnapshot(String snapshotName);
928
929  /**
930   * Restore the specified snapshot on the original table. (The table must be disabled) If
931   * 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before
932   * executing the restore operation. In case of restore failure, the failsafe snapshot will be
933   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
934   * failsafe snapshot name is configurable by using the property
935   * "hbase.snapshot.restore.failsafe.name".
936   * @param snapshotName         name of the snapshot to restore
937   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
938   */
939  default CompletableFuture<Void> restoreSnapshot(String snapshotName,
940    boolean takeFailSafeSnapshot) {
941    return restoreSnapshot(snapshotName, takeFailSafeSnapshot, false);
942  }
943
944  /**
945   * Restore the specified snapshot on the original table. (The table must be disabled) If
946   * 'takeFailSafeSnapshot' is set to true, a snapshot of the current table is taken before
947   * executing the restore operation. In case of restore failure, the failsafe snapshot will be
948   * restored. If the restore completes without problem the failsafe snapshot is deleted. The
949   * failsafe snapshot name is configurable by using the property
950   * "hbase.snapshot.restore.failsafe.name".
951   * @param snapshotName         name of the snapshot to restore
952   * @param takeFailSafeSnapshot true if the failsafe snapshot should be taken
953   * @param restoreAcl           <code>true</code> to restore acl of snapshot
954   */
955  CompletableFuture<Void> restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot,
956    boolean restoreAcl);
957
958  /**
959   * Create a new table by cloning the snapshot content.
960   * @param snapshotName name of the snapshot to be cloned
961   * @param tableName    name of the table where the snapshot will be restored
962   */
963  default CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName) {
964    return cloneSnapshot(snapshotName, tableName, false);
965  }
966
967  /**
968   * Create a new table by cloning the snapshot content.
969   * @param snapshotName name of the snapshot to be cloned
970   * @param tableName    name of the table where the snapshot will be restored
971   * @param restoreAcl   <code>true</code> to restore acl of snapshot
972   */
973  default CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
974    boolean restoreAcl) {
975    return cloneSnapshot(snapshotName, tableName, restoreAcl, null);
976  }
977
978  /**
979   * Create a new table by cloning the snapshot content.
980   * @param snapshotName name of the snapshot to be cloned
981   * @param tableName    name of the table where the snapshot will be restored
982   * @param restoreAcl   <code>true</code> to restore acl of snapshot
983   * @param customSFT    specify the StroreFileTracker used for the table
984   */
985  CompletableFuture<Void> cloneSnapshot(String snapshotName, TableName tableName,
986    boolean restoreAcl, String customSFT);
987
988  /**
989   * List completed snapshots.
990   * @return a list of snapshot descriptors for completed snapshots wrapped by a
991   *         {@link CompletableFuture}
992   */
993  CompletableFuture<List<SnapshotDescription>> listSnapshots();
994
995  /**
996   * List all the completed snapshots matching the given pattern.
997   * @param pattern The compiled regular expression to match against
998   * @return - returns a List of SnapshotDescription wrapped by a {@link CompletableFuture}
999   */
1000  CompletableFuture<List<SnapshotDescription>> listSnapshots(Pattern pattern);
1001
1002  /**
1003   * List all the completed snapshots matching the given table name pattern.
1004   * @param tableNamePattern The compiled table name regular expression to match against
1005   * @return - returns a List of completed SnapshotDescription wrapped by a
1006   *         {@link CompletableFuture}
1007   */
1008  CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern);
1009
1010  /**
1011   * List all the completed snapshots matching the given table name regular expression and snapshot
1012   * name regular expression.
1013   * @param tableNamePattern    The compiled table name regular expression to match against
1014   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
1015   * @return - returns a List of completed SnapshotDescription wrapped by a
1016   *         {@link CompletableFuture}
1017   */
1018  CompletableFuture<List<SnapshotDescription>> listTableSnapshots(Pattern tableNamePattern,
1019    Pattern snapshotNamePattern);
1020
1021  /**
1022   * Delete an existing snapshot.
1023   * @param snapshotName name of the snapshot
1024   */
1025  CompletableFuture<Void> deleteSnapshot(String snapshotName);
1026
1027  /**
1028   * Delete all existing snapshots.
1029   */
1030  CompletableFuture<Void> deleteSnapshots();
1031
1032  /**
1033   * Delete existing snapshots whose names match the pattern passed.
1034   * @param pattern pattern for names of the snapshot to match
1035   */
1036  CompletableFuture<Void> deleteSnapshots(Pattern pattern);
1037
1038  /**
1039   * Delete all existing snapshots matching the given table name pattern.
1040   * @param tableNamePattern The compiled table name regular expression to match against
1041   */
1042  CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern);
1043
1044  /**
1045   * Delete all existing snapshots matching the given table name regular expression and snapshot
1046   * name regular expression.
1047   * @param tableNamePattern    The compiled table name regular expression to match against
1048   * @param snapshotNamePattern The compiled snapshot name regular expression to match against
1049   */
1050  CompletableFuture<Void> deleteTableSnapshots(Pattern tableNamePattern,
1051    Pattern snapshotNamePattern);
1052
1053  /**
1054   * Execute a distributed procedure on a cluster.
1055   * @param signature A distributed procedure is uniquely identified by its signature (default the
1056   *                  root ZK node name of the procedure).
1057   * @param instance  The instance name of the procedure. For some procedures, this parameter is
1058   *                  optional.
1059   * @param props     Property/Value pairs of properties passing to the procedure
1060   */
1061  CompletableFuture<Void> execProcedure(String signature, String instance,
1062    Map<String, String> props);
1063
1064  /**
1065   * Execute a distributed procedure on a cluster.
1066   * @param signature A distributed procedure is uniquely identified by its signature (default the
1067   *                  root ZK node name of the procedure).
1068   * @param instance  The instance name of the procedure. For some procedures, this parameter is
1069   *                  optional.
1070   * @param props     Property/Value pairs of properties passing to the procedure
1071   * @return data returned after procedure execution. null if no return data.
1072   */
1073  CompletableFuture<byte[]> execProcedureWithReturn(String signature, String instance,
1074    Map<String, String> props);
1075
1076  /**
1077   * Check the current state of the specified procedure. There are three possible states:
1078   * <ol>
1079   * <li>running - returns <tt>false</tt></li>
1080   * <li>finished - returns <tt>true</tt></li>
1081   * <li>finished with error - throws the exception that caused the procedure to fail</li>
1082   * </ol>
1083   * @param signature The signature that uniquely identifies a procedure
1084   * @param instance  The instance name of the procedure
1085   * @param props     Property/Value pairs of properties passing to the procedure
1086   * @return true if the specified procedure is finished successfully, false if it is still running.
1087   *         The value is wrapped by {@link CompletableFuture}
1088   */
1089  CompletableFuture<Boolean> isProcedureFinished(String signature, String instance,
1090    Map<String, String> props);
1091
1092  /**
1093   * Abort a procedure Do not use. Usually it is ignored but if not, it can do more damage than
1094   * good. See hbck2.
1095   * @param procId                ID of the procedure to abort
1096   * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted?
1097   * @return true if aborted, false if procedure already completed or does not exist. the value is
1098   *         wrapped by {@link CompletableFuture}
1099   * @deprecated since 2.1.1 and will be removed in 4.0.0.
1100   * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a>
1101   */
1102  @Deprecated
1103  CompletableFuture<Boolean> abortProcedure(long procId, boolean mayInterruptIfRunning);
1104
1105  /**
1106   * List procedures
1107   * @return procedure list JSON wrapped by {@link CompletableFuture}
1108   */
1109  CompletableFuture<String> getProcedures();
1110
1111  /**
1112   * List locks.
1113   * @return lock list JSON wrapped by {@link CompletableFuture}
1114   */
1115  CompletableFuture<String> getLocks();
1116
1117  /**
1118   * Mark region server(s) as decommissioned to prevent additional regions from getting assigned to
1119   * them. Optionally unload the regions on the servers. If there are multiple servers to be
1120   * decommissioned, decommissioning them at the same time can prevent wasteful region movements.
1121   * Region unloading is asynchronous.
1122   * @param servers The list of servers to decommission.
1123   * @param offload True to offload the regions from the decommissioned servers
1124   */
1125  CompletableFuture<Void> decommissionRegionServers(List<ServerName> servers, boolean offload);
1126
1127  /**
1128   * List region servers marked as decommissioned, which can not be assigned regions.
1129   * @return List of decommissioned region servers wrapped by {@link CompletableFuture}
1130   */
1131  CompletableFuture<List<ServerName>> listDecommissionedRegionServers();
1132
1133  /**
1134   * Remove decommission marker from a region server to allow regions assignments. Load regions onto
1135   * the server if a list of regions is given. Region loading is asynchronous.
1136   * @param server             The server to recommission.
1137   * @param encodedRegionNames Regions to load onto the server.
1138   */
1139  CompletableFuture<Void> recommissionRegionServer(ServerName server,
1140    List<byte[]> encodedRegionNames);
1141
1142  /** Returns cluster status wrapped by {@link CompletableFuture} */
1143  CompletableFuture<ClusterMetrics> getClusterMetrics();
1144
1145  /** Returns cluster status wrapped by {@link CompletableFuture} */
1146  CompletableFuture<ClusterMetrics> getClusterMetrics(EnumSet<Option> options);
1147
1148  /** Returns current master server name wrapped by {@link CompletableFuture} */
1149  default CompletableFuture<ServerName> getMaster() {
1150    return getClusterMetrics(EnumSet.of(Option.MASTER)).thenApply(ClusterMetrics::getMasterName);
1151  }
1152
1153  /** Returns current backup master list wrapped by {@link CompletableFuture} */
1154  default CompletableFuture<Collection<ServerName>> getBackupMasters() {
1155    return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS))
1156      .thenApply(ClusterMetrics::getBackupMasterNames);
1157  }
1158
1159  /** Returns current live region servers list wrapped by {@link CompletableFuture} */
1160  default CompletableFuture<Collection<ServerName>> getRegionServers() {
1161    return getClusterMetrics(EnumSet.of(Option.SERVERS_NAME))
1162      .thenApply(ClusterMetrics::getServersName);
1163  }
1164
1165  default CompletableFuture<Collection<ServerName>>
1166    getRegionServers(boolean excludeDecommissionedRS) {
1167    CompletableFuture<Collection<ServerName>> future = new CompletableFuture<>();
1168    addListener(
1169      getClusterMetrics(EnumSet.of(Option.SERVERS_NAME)).thenApply(ClusterMetrics::getServersName),
1170      (allServers, err) -> {
1171        if (err != null) {
1172          future.completeExceptionally(err);
1173        } else {
1174          if (!excludeDecommissionedRS) {
1175            future.complete(allServers);
1176          } else {
1177            addListener(listDecommissionedRegionServers(), (decomServers, decomErr) -> {
1178              if (decomErr != null) {
1179                future.completeExceptionally(decomErr);
1180              } else {
1181                future.complete(allServers.stream().filter(s -> !decomServers.contains(s))
1182                  .collect(ImmutableList.toImmutableList()));
1183              }
1184            });
1185          }
1186        }
1187      });
1188    return future;
1189  }
1190
1191  /** Returns a list of master coprocessors wrapped by {@link CompletableFuture} */
1192  default CompletableFuture<List<String>> getMasterCoprocessorNames() {
1193    return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS))
1194      .thenApply(ClusterMetrics::getMasterCoprocessorNames);
1195  }
1196
1197  /**
1198   * Get the info port of the current master if one is available.
1199   * @return master info port
1200   */
1201  default CompletableFuture<Integer> getMasterInfoPort() {
1202    return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT))
1203      .thenApply(ClusterMetrics::getMasterInfoPort);
1204  }
1205
1206  /**
1207   * Shuts down the HBase cluster.
1208   */
1209  CompletableFuture<Void> shutdown();
1210
1211  /**
1212   * Shuts down the current HBase master only.
1213   */
1214  CompletableFuture<Void> stopMaster();
1215
1216  /**
1217   * Stop the designated regionserver.
1218   */
1219  CompletableFuture<Void> stopRegionServer(ServerName serverName);
1220
1221  /**
1222   * Update the configuration and trigger an online config change on the regionserver.
1223   * @param serverName : The server whose config needs to be updated.
1224   */
1225  CompletableFuture<Void> updateConfiguration(ServerName serverName);
1226
1227  /**
1228   * Update the configuration and trigger an online config change on all the masters and
1229   * regionservers.
1230   */
1231  CompletableFuture<Void> updateConfiguration();
1232
1233  /**
1234   * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file.
1235   * <p>
1236   * When the returned CompletableFuture is done, it only means the rollWALWriter request was sent
1237   * to the region server and may need some time to finish the rollWALWriter operation. As a side
1238   * effect of this call, the named region server may schedule store flushes at the request of the
1239   * wal.
1240   * @param serverName The servername of the region server.
1241   */
1242  CompletableFuture<Void> rollWALWriter(ServerName serverName);
1243
1244  /**
1245   * Clear compacting queues on a region server.
1246   * @param serverName The servername of the region server.
1247   * @param queues     the set of queue name
1248   */
1249  CompletableFuture<Void> clearCompactionQueues(ServerName serverName, Set<String> queues);
1250
1251  /**
1252   * Get a list of {@link RegionMetrics} of all regions hosted on a region server.
1253   * @return list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
1254   */
1255  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName);
1256
1257  /**
1258   * Get a list of {@link RegionMetrics} of all regions hosted on a region server for a table.
1259   * @return a list of {@link RegionMetrics} wrapped by {@link CompletableFuture}
1260   */
1261  CompletableFuture<List<RegionMetrics>> getRegionMetrics(ServerName serverName,
1262    TableName tableName);
1263
1264  /**
1265   * Check whether master is in maintenance mode
1266   * @return true if master is in maintenance mode, false otherwise. The return value will be
1267   *         wrapped by a {@link CompletableFuture}
1268   */
1269  CompletableFuture<Boolean> isMasterInMaintenanceMode();
1270
1271  /**
1272   * Get the current compaction state of a table. It could be in a major compaction, a minor
1273   * compaction, both, or none.
1274   * @param tableName table to examine
1275   * @return the current compaction state wrapped by a {@link CompletableFuture}
1276   */
1277  default CompletableFuture<CompactionState> getCompactionState(TableName tableName) {
1278    return getCompactionState(tableName, CompactType.NORMAL);
1279  }
1280
1281  /**
1282   * Get the current compaction state of a table. It could be in a major compaction, a minor
1283   * compaction, both, or none.
1284   * @param tableName   table to examine
1285   * @param compactType {@link org.apache.hadoop.hbase.client.CompactType}
1286   * @return the current compaction state wrapped by a {@link CompletableFuture}
1287   */
1288  CompletableFuture<CompactionState> getCompactionState(TableName tableName,
1289    CompactType compactType);
1290
1291  /**
1292   * Get the current compaction state of region. It could be in a major compaction, a minor
1293   * compaction, both, or none.
1294   * @param regionName region to examine
1295   * @return the current compaction state wrapped by a {@link CompletableFuture}
1296   */
1297  CompletableFuture<CompactionState> getCompactionStateForRegion(byte[] regionName);
1298
1299  /**
1300   * Get the timestamp of the last major compaction for the passed table.
1301   * <p>
1302   * The timestamp of the oldest HFile resulting from a major compaction of that table, or not
1303   * present if no such HFile could be found.
1304   * @param tableName table to examine
1305   * @return the last major compaction timestamp wrapped by a {@link CompletableFuture}
1306   */
1307  CompletableFuture<Optional<Long>> getLastMajorCompactionTimestamp(TableName tableName);
1308
1309  /**
1310   * Get the timestamp of the last major compaction for the passed region.
1311   * <p>
1312   * The timestamp of the oldest HFile resulting from a major compaction of that region, or not
1313   * present if no such HFile could be found.
1314   * @param regionName region to examine
1315   * @return the last major compaction timestamp wrapped by a {@link CompletableFuture}
1316   */
1317  CompletableFuture<Optional<Long>> getLastMajorCompactionTimestampForRegion(byte[] regionName);
1318
1319  /**
1320   * Returns the list of supported security capabilities. The return value will be wrapped by a
1321   * {@link CompletableFuture}.
1322   */
1323  CompletableFuture<List<SecurityCapability>> getSecurityCapabilities();
1324
1325  /**
1326   * Turn the load balancer on or off.
1327   * @param on Set to <code>true</code> to enable, <code>false</code> to disable.
1328   * @return Previous balancer value wrapped by a {@link CompletableFuture}.
1329   */
1330  default CompletableFuture<Boolean> balancerSwitch(boolean on) {
1331    return balancerSwitch(on, false);
1332  }
1333
1334  /**
1335   * Turn the load balancer on or off.
1336   * <p/>
1337   * Notice that, the method itself is always non-blocking, which means it will always return
1338   * immediately. The {@code drainRITs} parameter only effects when will we complete the returned
1339   * {@link CompletableFuture}.
1340   * @param on        Set to <code>true</code> to enable, <code>false</code> to disable.
1341   * @param drainRITs If <code>true</code>, it waits until current balance() call, if outstanding,
1342   *                  to return.
1343   * @return Previous balancer value wrapped by a {@link CompletableFuture}.
1344   */
1345  CompletableFuture<Boolean> balancerSwitch(boolean on, boolean drainRITs);
1346
1347  /**
1348   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
1349   * reassignments. Can NOT run for various reasons. Check logs.
1350   * @return True if balancer ran, false otherwise. The return value will be wrapped by a
1351   *         {@link CompletableFuture}.
1352   */
1353  default CompletableFuture<Boolean> balance() {
1354    return balance(BalanceRequest.defaultInstance()).thenApply(BalanceResponse::isBalancerRan);
1355  }
1356
1357  /**
1358   * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the
1359   * reassignments. If there is region in transition, force parameter of true would still run
1360   * balancer. Can *not* run for other reasons. Check logs.
1361   * @param forcible whether we should force balance even if there is region in transition.
1362   * @return True if balancer ran, false otherwise. The return value will be wrapped by a
1363   *         {@link CompletableFuture}.
1364   * @deprecated Since 2.5.0. Will be removed in 4.0.0. Use {@link #balance(BalanceRequest)}
1365   *             instead.
1366   */
1367  default CompletableFuture<Boolean> balance(boolean forcible) {
1368    return balance(BalanceRequest.newBuilder().setIgnoreRegionsInTransition(forcible).build())
1369      .thenApply(BalanceResponse::isBalancerRan);
1370  }
1371
1372  /**
1373   * Invoke the balancer with the given balance request. The BalanceRequest defines how the balancer
1374   * will run. See {@link BalanceRequest} for more details.
1375   * @param request defines how the balancer should run
1376   * @return {@link BalanceResponse} with details about the results of the invocation.
1377   */
1378  CompletableFuture<BalanceResponse> balance(BalanceRequest request);
1379
1380  /**
1381   * Query the current state of the balancer.
1382   * @return true if the balance switch is on, false otherwise. The return value will be wrapped by
1383   *         a {@link CompletableFuture}.
1384   */
1385  CompletableFuture<Boolean> isBalancerEnabled();
1386
1387  /**
1388   * Set region normalizer on/off.
1389   * @param on whether normalizer should be on or off
1390   * @return Previous normalizer value wrapped by a {@link CompletableFuture}
1391   */
1392  CompletableFuture<Boolean> normalizerSwitch(boolean on);
1393
1394  /**
1395   * Query the current state of the region normalizer
1396   * @return true if region normalizer is on, false otherwise. The return value will be wrapped by a
1397   *         {@link CompletableFuture}
1398   */
1399  CompletableFuture<Boolean> isNormalizerEnabled();
1400
1401  /**
1402   * Invoke region normalizer. Can NOT run for various reasons. Check logs.
1403   * @return true if region normalizer ran, false otherwise. The return value will be wrapped by a
1404   *         {@link CompletableFuture}
1405   */
1406  default CompletableFuture<Boolean> normalize() {
1407    return normalize(new NormalizeTableFilterParams.Builder().build());
1408  }
1409
1410  /**
1411   * Invoke region normalizer. Can NOT run for various reasons. Check logs.
1412   * @param ntfp limit to tables matching the specified filter.
1413   * @return true if region normalizer ran, false otherwise. The return value will be wrapped by a
1414   *         {@link CompletableFuture}
1415   */
1416  CompletableFuture<Boolean> normalize(NormalizeTableFilterParams ntfp);
1417
1418  /**
1419   * Turn the cleaner chore on/off.
1420   * @return Previous cleaner state wrapped by a {@link CompletableFuture}
1421   */
1422  CompletableFuture<Boolean> cleanerChoreSwitch(boolean on);
1423
1424  /**
1425   * Query the current state of the cleaner chore.
1426   * @return true if cleaner chore is on, false otherwise. The return value will be wrapped by a
1427   *         {@link CompletableFuture}
1428   */
1429  CompletableFuture<Boolean> isCleanerChoreEnabled();
1430
1431  /**
1432   * Ask for cleaner chore to run.
1433   * @return true if cleaner chore ran, false otherwise. The return value will be wrapped by a
1434   *         {@link CompletableFuture}
1435   */
1436  CompletableFuture<Boolean> runCleanerChore();
1437
1438  /**
1439   * Turn the catalog janitor on/off.
1440   * @return the previous state wrapped by a {@link CompletableFuture}
1441   */
1442  CompletableFuture<Boolean> catalogJanitorSwitch(boolean on);
1443
1444  /**
1445   * Query on the catalog janitor state.
1446   * @return true if the catalog janitor is on, false otherwise. The return value will be wrapped by
1447   *         a {@link CompletableFuture}
1448   */
1449  CompletableFuture<Boolean> isCatalogJanitorEnabled();
1450
1451  /**
1452   * Ask for a scan of the catalog table.
1453   * @return the number of entries cleaned. The return value will be wrapped by a
1454   *         {@link CompletableFuture}
1455   */
1456  CompletableFuture<Integer> runCatalogJanitor();
1457
1458  /**
1459   * Execute the given coprocessor call on the master.
1460   * <p>
1461   * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
1462   * one line lambda expression, like:
1463   *
1464   * <pre>
1465   * channel -&gt; xxxService.newStub(channel)
1466   * </pre>
1467   *
1468   * @param stubMaker a delegation to the actual {@code newStub} call.
1469   * @param callable  a delegation to the actual protobuf rpc call. See the comment of
1470   *                  {@link ServiceCaller} for more details.
1471   * @param <S>       the type of the asynchronous stub
1472   * @param <R>       the type of the return value
1473   * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
1474   * @see ServiceCaller
1475   */
1476  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
1477    ServiceCaller<S, R> callable);
1478
1479  /**
1480   * Execute the given coprocessor call on the given region server.
1481   * <p>
1482   * The {@code stubMaker} is just a delegation to the {@code newStub} call. Usually it is only a
1483   * one line lambda expression, like:
1484   *
1485   * <pre>
1486   * channel -&gt; xxxService.newStub(channel)
1487   * </pre>
1488   *
1489   * @param stubMaker  a delegation to the actual {@code newStub} call.
1490   * @param callable   a delegation to the actual protobuf rpc call. See the comment of
1491   *                   {@link ServiceCaller} for more details.
1492   * @param serverName the given region server
1493   * @param <S>        the type of the asynchronous stub
1494   * @param <R>        the type of the return value
1495   * @return the return value of the protobuf rpc call, wrapped by a {@link CompletableFuture}.
1496   * @see ServiceCaller
1497   */
1498  <S, R> CompletableFuture<R> coprocessorService(Function<RpcChannel, S> stubMaker,
1499    ServiceCaller<S, R> callable, ServerName serverName);
1500
1501  /**
1502   * List all the dead region servers.
1503   */
1504  default CompletableFuture<List<ServerName>> listDeadServers() {
1505    return this.getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS))
1506      .thenApply(ClusterMetrics::getDeadServerNames);
1507  }
1508
1509  /**
1510   * List all the unknown region servers.
1511   */
1512  default CompletableFuture<List<ServerName>> listUnknownServers() {
1513    return this.getClusterMetrics(EnumSet.of(Option.UNKNOWN_SERVERS))
1514      .thenApply(ClusterMetrics::getUnknownServerNames);
1515  }
1516
1517  /**
1518   * Clear dead region servers from master.
1519   * @param servers list of dead region servers.
1520   * @return - returns a list of servers that not cleared wrapped by a {@link CompletableFuture}.
1521   */
1522  CompletableFuture<List<ServerName>> clearDeadServers(final List<ServerName> servers);
1523
1524  /**
1525   * Clear all the blocks corresponding to this table from BlockCache. For expert-admins. Calling
1526   * this API will drop all the cached blocks specific to a table from BlockCache. This can
1527   * significantly impact the query performance as the subsequent queries will have to retrieve the
1528   * blocks from underlying filesystem.
1529   * @param tableName table to clear block cache
1530   * @return CacheEvictionStats related to the eviction wrapped by a {@link CompletableFuture}.
1531   */
1532  CompletableFuture<CacheEvictionStats> clearBlockCache(final TableName tableName);
1533
1534  /**
1535   * Create a new table by cloning the existent table schema.
1536   * @param tableName      name of the table to be cloned
1537   * @param newTableName   name of the new table where the table will be created
1538   * @param preserveSplits True if the splits should be preserved
1539   */
1540  CompletableFuture<Void> cloneTableSchema(final TableName tableName, final TableName newTableName,
1541    final boolean preserveSplits);
1542
1543  /**
1544   * Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing
1545   * compactions. This state is ephemeral. The setting will be lost on restart. Compaction can also
1546   * be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled in
1547   * hbase-site.xml.
1548   * @param switchState     Set to <code>true</code> to enable, <code>false</code> to disable.
1549   * @param serverNamesList list of region servers.
1550   * @return Previous compaction states for region servers
1551   */
1552  CompletableFuture<Map<ServerName, Boolean>> compactionSwitch(boolean switchState,
1553    List<String> serverNamesList);
1554
1555  /**
1556   * Switch the rpc throttle enabled state.
1557   * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
1558   * @return Previous rpc throttle enabled value
1559   */
1560  CompletableFuture<Boolean> switchRpcThrottle(boolean enable);
1561
1562  /**
1563   * Get if the rpc throttle is enabled.
1564   * @return True if rpc throttle is enabled
1565   */
1566  CompletableFuture<Boolean> isRpcThrottleEnabled();
1567
1568  /**
1569   * Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota can be
1570   * exceeded if region server has availble quota.
1571   * @param enable Set to <code>true</code> to enable, <code>false</code> to disable.
1572   * @return Previous exceed throttle enabled value
1573   */
1574  CompletableFuture<Boolean> exceedThrottleQuotaSwitch(boolean enable);
1575
1576  /**
1577   * Fetches the table sizes on the filesystem as tracked by the HBase Master.
1578   */
1579  CompletableFuture<Map<TableName, Long>> getSpaceQuotaTableSizes();
1580
1581  /**
1582   * Fetches the observed {@link SpaceQuotaSnapshotView}s observed by a RegionServer.
1583   */
1584  CompletableFuture<? extends Map<TableName, ? extends SpaceQuotaSnapshotView>>
1585    getRegionServerSpaceQuotaSnapshots(ServerName serverName);
1586
1587  /**
1588   * Returns the Master's view of a quota on the given {@code namespace} or null if the Master has
1589   * no quota information on that namespace.
1590   */
1591  CompletableFuture<? extends SpaceQuotaSnapshotView>
1592    getCurrentSpaceQuotaSnapshot(String namespace);
1593
1594  /**
1595   * Returns the Master's view of a quota on the given {@code tableName} or null if the Master has
1596   * no quota information on that table.
1597   */
1598  CompletableFuture<? extends SpaceQuotaSnapshotView>
1599    getCurrentSpaceQuotaSnapshot(TableName tableName);
1600
1601  /**
1602   * Grants user specific permissions
1603   * @param userPermission           user name and the specific permission
1604   * @param mergeExistingPermissions If set to false, later granted permissions will override
1605   *                                 previous granted permissions. otherwise, it'll merge with
1606   *                                 previous granted permissions.
1607   */
1608  CompletableFuture<Void> grant(UserPermission userPermission, boolean mergeExistingPermissions);
1609
1610  /**
1611   * Revokes user specific permissions
1612   * @param userPermission user name and the specific permission
1613   */
1614  CompletableFuture<Void> revoke(UserPermission userPermission);
1615
1616  /**
1617   * Get the global/namespace/table permissions for user
1618   * @param getUserPermissionsRequest A request contains which user, global, namespace or table
1619   *                                  permissions needed
1620   * @return The user and permission list
1621   */
1622  CompletableFuture<List<UserPermission>>
1623    getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest);
1624
1625  /**
1626   * Check if the user has specific permissions
1627   * @param userName    the user name
1628   * @param permissions the specific permission list
1629   * @return True if user has the specific permissions
1630   */
1631  CompletableFuture<List<Boolean>> hasUserPermissions(String userName,
1632    List<Permission> permissions);
1633
1634  /**
1635   * Check if call user has specific permissions
1636   * @param permissions the specific permission list
1637   * @return True if user has the specific permissions
1638   */
1639  default CompletableFuture<List<Boolean>> hasUserPermissions(List<Permission> permissions) {
1640    return hasUserPermissions(null, permissions);
1641  }
1642
1643  /**
1644   * Turn on or off the auto snapshot cleanup based on TTL.
1645   * <p/>
1646   * Notice that, the method itself is always non-blocking, which means it will always return
1647   * immediately. The {@code sync} parameter only effects when will we complete the returned
1648   * {@link CompletableFuture}.
1649   * @param on   Set to <code>true</code> to enable, <code>false</code> to disable.
1650   * @param sync If <code>true</code>, it waits until current snapshot cleanup is completed, if
1651   *             outstanding.
1652   * @return Previous auto snapshot cleanup value wrapped by a {@link CompletableFuture}.
1653   */
1654  CompletableFuture<Boolean> snapshotCleanupSwitch(boolean on, boolean sync);
1655
1656  /**
1657   * Query the current state of the auto snapshot cleanup based on TTL.
1658   * @return true if the auto snapshot cleanup is enabled, false otherwise. The return value will be
1659   *         wrapped by a {@link CompletableFuture}.
1660   */
1661  CompletableFuture<Boolean> isSnapshotCleanupEnabled();
1662
1663  /**
1664   * Retrieves online slow RPC logs from the provided list of RegionServers
1665   * @param serverNames    Server names to get slowlog responses from
1666   * @param logQueryFilter filter to be used if provided
1667   * @return Online slowlog response list. The return value wrapped by a {@link CompletableFuture}
1668   * @deprecated since 2.4.0 and will be removed in 4.0.0. Use
1669   *             {@link #getLogEntries(Set, String, ServerType, int, Map)} instead.
1670   */
1671  @Deprecated
1672  default CompletableFuture<List<OnlineLogRecord>>
1673    getSlowLogResponses(final Set<ServerName> serverNames, final LogQueryFilter logQueryFilter) {
1674    String logType;
1675    if (LogQueryFilter.Type.LARGE_LOG.equals(logQueryFilter.getType())) {
1676      logType = "LARGE_LOG";
1677    } else {
1678      logType = "SLOW_LOG";
1679    }
1680    Map<String, Object> filterParams = new HashMap<>();
1681    filterParams.put("regionName", logQueryFilter.getRegionName());
1682    filterParams.put("clientAddress", logQueryFilter.getClientAddress());
1683    filterParams.put("tableName", logQueryFilter.getTableName());
1684    filterParams.put("userName", logQueryFilter.getUserName());
1685    filterParams.put("filterByOperator", logQueryFilter.getFilterByOperator().toString());
1686    CompletableFuture<List<LogEntry>> logEntries = getLogEntries(serverNames, logType,
1687      ServerType.REGION_SERVER, logQueryFilter.getLimit(), filterParams);
1688    return logEntries.thenApply(logEntryList -> logEntryList.stream()
1689      .map(logEntry -> (OnlineLogRecord) logEntry).collect(Collectors.toList()));
1690  }
1691
1692  /**
1693   * Clears online slow RPC logs from the provided list of RegionServers
1694   * @param serverNames Set of Server names to clean slowlog responses from
1695   * @return List of booleans representing if online slowlog response buffer is cleaned from each
1696   *         RegionServer. The return value wrapped by a {@link CompletableFuture}
1697   */
1698  CompletableFuture<List<Boolean>> clearSlowLogResponses(final Set<ServerName> serverNames);
1699
1700  /**
1701   * Retrieve recent online records from HMaster / RegionServers. Examples include slow/large RPC
1702   * logs, balancer decisions by master.
1703   * @param serverNames  servers to retrieve records from, useful in case of records maintained by
1704   *                     RegionServer as we can select specific server. In case of
1705   *                     servertype=MASTER, logs will only come from the currently active master.
1706   * @param logType      string representing type of log records
1707   * @param serverType   enum for server type: HMaster or RegionServer
1708   * @param limit        put a limit to list of records that server should send in response
1709   * @param filterParams additional filter params
1710   */
1711  CompletableFuture<List<LogEntry>> getLogEntries(Set<ServerName> serverNames, String logType,
1712    ServerType serverType, int limit, Map<String, Object> filterParams);
1713
1714  /**
1715   * Flush master local region
1716   */
1717  CompletableFuture<Void> flushMasterStore();
1718
1719  /**
1720   * Get the list of cached files
1721   */
1722  CompletableFuture<List<String>> getCachedFilesList(ServerName serverName);
1723}