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