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