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.get; 021 022import java.io.Closeable; 023import java.io.IOException; 024import java.util.Collection; 025import java.util.EnumSet; 026import java.util.HashMap; 027import java.util.List; 028import java.util.Map; 029import java.util.Set; 030import java.util.concurrent.Future; 031import java.util.concurrent.TimeUnit; 032import java.util.regex.Pattern; 033import java.util.stream.Collectors; 034import org.apache.hadoop.conf.Configuration; 035import org.apache.hadoop.hbase.Abortable; 036import org.apache.hadoop.hbase.CacheEvictionStats; 037import org.apache.hadoop.hbase.ClusterMetrics; 038import org.apache.hadoop.hbase.ClusterMetrics.Option; 039import org.apache.hadoop.hbase.ClusterStatus; 040import org.apache.hadoop.hbase.HRegionInfo; 041import org.apache.hadoop.hbase.HTableDescriptor; 042import org.apache.hadoop.hbase.NamespaceDescriptor; 043import org.apache.hadoop.hbase.NamespaceNotFoundException; 044import org.apache.hadoop.hbase.RegionMetrics; 045import org.apache.hadoop.hbase.ServerName; 046import org.apache.hadoop.hbase.TableExistsException; 047import org.apache.hadoop.hbase.TableName; 048import org.apache.hadoop.hbase.TableNotFoundException; 049import org.apache.hadoop.hbase.client.replication.ReplicationPeerConfigUtil; 050import org.apache.hadoop.hbase.client.replication.TableCFs; 051import org.apache.hadoop.hbase.client.security.SecurityCapability; 052import org.apache.hadoop.hbase.ipc.CoprocessorRpcChannel; 053import org.apache.hadoop.hbase.quotas.QuotaFilter; 054import org.apache.hadoop.hbase.quotas.QuotaRetriever; 055import org.apache.hadoop.hbase.quotas.QuotaSettings; 056import org.apache.hadoop.hbase.quotas.SpaceQuotaSnapshotView; 057import org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException; 058import org.apache.hadoop.hbase.replication.ReplicationException; 059import org.apache.hadoop.hbase.replication.ReplicationPeerConfig; 060import org.apache.hadoop.hbase.replication.ReplicationPeerDescription; 061import org.apache.hadoop.hbase.security.access.GetUserPermissionsRequest; 062import org.apache.hadoop.hbase.security.access.Permission; 063import org.apache.hadoop.hbase.security.access.UserPermission; 064import org.apache.hadoop.hbase.snapshot.HBaseSnapshotException; 065import org.apache.hadoop.hbase.snapshot.RestoreSnapshotException; 066import org.apache.hadoop.hbase.snapshot.SnapshotCreationException; 067import org.apache.hadoop.hbase.snapshot.UnknownSnapshotException; 068import org.apache.hadoop.hbase.util.Bytes; 069import org.apache.hadoop.hbase.util.Pair; 070import org.apache.yetus.audience.InterfaceAudience; 071 072import org.apache.hbase.thirdparty.com.google.common.collect.ImmutableList; 073 074/** 075 * The administrative API for HBase. Obtain an instance from {@link Connection#getAdmin()} and 076 * call {@link #close()} when done. 077 * <p>Admin can be used to create, drop, list, enable and disable and otherwise modify tables, 078 * as well as perform other administrative operations. 079 * 080 * @see ConnectionFactory 081 * @see Connection 082 * @see Table 083 * @since 0.99.0 084 */ 085@InterfaceAudience.Public 086public interface Admin extends Abortable, Closeable { 087 088 /** 089 * Return the operation timeout for a rpc call. 090 * @see #getSyncWaitTimeout() 091 */ 092 int getOperationTimeout(); 093 094 /** 095 * Return the blocking wait time for an asynchronous operation. Can be configured by 096 * {@code hbase.client.sync.wait.timeout.msec}. 097 * <p/> 098 * For several operations, such as createTable, deleteTable, etc, the rpc call will finish right 099 * after we schedule a procedure at master side, so the timeout will not be controlled by the 100 * above {@link #getOperationTimeout()}. And timeout value here tells you how much time we will 101 * wait until the procedure at master side is finished. 102 * <p/> 103 * In general, you can consider that the implementation for XXXX method is just a 104 * XXXXAsync().get(getSyncWaitTimeout(), TimeUnit.MILLISECONDS). 105 * @see #getOperationTimeout() 106 */ 107 int getSyncWaitTimeout(); 108 109 @Override 110 void abort(String why, Throwable e); 111 112 @Override 113 boolean isAborted(); 114 115 /** 116 * @return Connection used by this object. 117 */ 118 Connection getConnection(); 119 120 /** 121 * @param tableName Table to check. 122 * @return <code>true</code> if table exists already. 123 * @throws IOException if a remote or network exception occurs 124 */ 125 boolean tableExists(TableName tableName) throws IOException; 126 127 /** 128 * List all the userspace tables. 129 * 130 * @return an array of read-only HTableDescriptors 131 * @throws IOException if a remote or network exception occurs 132 * @deprecated since 2.0 version and will be removed in 3.0 version. 133 * Use {@link #listTableDescriptors()}. 134 * @see #listTableDescriptors() 135 */ 136 @Deprecated 137 HTableDescriptor[] listTables() throws IOException; 138 139 /** 140 * List all the userspace tables. 141 * 142 * @return a list of TableDescriptors 143 * @throws IOException if a remote or network exception occurs 144 */ 145 List<TableDescriptor> listTableDescriptors() throws IOException; 146 147 /** 148 * List all the userspace tables that match the given pattern. 149 * 150 * @param pattern The compiled regular expression to match against 151 * @return an array of read-only HTableDescriptors 152 * @throws IOException if a remote or network exception occurs 153 * @see #listTables() 154 * @deprecated since 2.0 version and will be removed in 3.0 version. 155 * Use {@link #listTableDescriptors(java.util.regex.Pattern)}. 156 * @see #listTableDescriptors(Pattern) 157 */ 158 @Deprecated 159 HTableDescriptor[] listTables(Pattern pattern) throws IOException; 160 161 /** 162 * List all the userspace tables that match the given pattern. 163 * 164 * @param pattern The compiled regular expression to match against 165 * @return a list of TableDescriptors 166 * @throws IOException if a remote or network exception occurs 167 * @see #listTables() 168 */ 169 default List<TableDescriptor> listTableDescriptors(Pattern pattern) throws IOException { 170 return listTableDescriptors(pattern, false); 171 } 172 173 /** 174 * List all the userspace tables matching the given regular expression. 175 * 176 * @param regex The regular expression to match against 177 * @return a list of read-only HTableDescriptors 178 * @throws IOException if a remote or network exception occurs 179 * @see #listTableDescriptors(Pattern) 180 * @deprecated since 2.0 version and will be removed in 3.0 version. Use 181 * {@link #listTableDescriptors(Pattern)} instead. 182 */ 183 @Deprecated 184 HTableDescriptor[] listTables(String regex) throws IOException; 185 186 /** 187 * List all the tables matching the given pattern. 188 * 189 * @param pattern The compiled regular expression to match against 190 * @param includeSysTables <code>false</code> to match only against userspace tables 191 * @return an array of read-only HTableDescriptors 192 * @throws IOException if a remote or network exception occurs 193 * @see #listTables() 194 * @deprecated since 2.0 version and will be removed in 3.0 version. 195 * Use {@link #listTableDescriptors(java.util.regex.Pattern, boolean)}. 196 * @see #listTableDescriptors(java.util.regex.Pattern, boolean) 197 */ 198 @Deprecated 199 HTableDescriptor[] listTables(Pattern pattern, boolean includeSysTables) 200 throws IOException; 201 202 /** 203 * List all the tables matching the given pattern. 204 * 205 * @param pattern The compiled regular expression to match against 206 * @param includeSysTables <code>false</code> to match only against userspace tables 207 * @return a list of TableDescriptors 208 * @throws IOException if a remote or network exception occurs 209 * @see #listTables() 210 */ 211 List<TableDescriptor> listTableDescriptors(Pattern pattern, boolean includeSysTables) 212 throws IOException; 213 214 /** 215 * List all the tables matching the given pattern. 216 * 217 * @param regex The regular expression to match against 218 * @param includeSysTables <code>false</code> to match only against userspace tables 219 * @return an array of read-only HTableDescriptors 220 * @throws IOException if a remote or network exception occurs 221 * @see #listTables(java.util.regex.Pattern, boolean) 222 * @deprecated since 2.0 version and will be removed in 3.0 version. 223 * Use {@link #listTableDescriptors(Pattern, boolean)}. 224 */ 225 @Deprecated 226 HTableDescriptor[] listTables(String regex, boolean includeSysTables) 227 throws IOException; 228 229 /** 230 * List all of the names of userspace tables. 231 * 232 * @return TableName[] table names 233 * @throws IOException if a remote or network exception occurs 234 */ 235 TableName[] listTableNames() throws IOException; 236 237 /** 238 * List all of the names of userspace tables. 239 * @param pattern The regular expression to match against 240 * @return array of table names 241 * @throws IOException if a remote or network exception occurs 242 */ 243 default TableName[] listTableNames(Pattern pattern) throws IOException { 244 return listTableNames(pattern, false); 245 } 246 247 /** 248 * List all of the names of userspace tables. 249 * @param regex The regular expression to match against 250 * @return TableName[] table names 251 * @throws IOException if a remote or network exception occurs 252 * @deprecated since 2.0 version and will be removed in 3.0 version. Use 253 * {@link #listTableNames(Pattern)} instead. 254 */ 255 @Deprecated 256 TableName[] listTableNames(String regex) throws IOException; 257 258 /** 259 * List all of the names of userspace tables. 260 * @param pattern The regular expression to match against 261 * @param includeSysTables <code>false</code> to match only against userspace tables 262 * @return TableName[] table names 263 * @throws IOException if a remote or network exception occurs 264 */ 265 TableName[] listTableNames(Pattern pattern, boolean includeSysTables) 266 throws IOException; 267 268 /** 269 * List all of the names of userspace tables. 270 * @param regex The regular expression to match against 271 * @param includeSysTables <code>false</code> to match only against userspace tables 272 * @return TableName[] table names 273 * @throws IOException if a remote or network exception occurs 274 * @deprecated since 2.0 version and will be removed in 3.0 version. Use 275 * {@link #listTableNames(Pattern, boolean)} instead. 276 */ 277 @Deprecated 278 TableName[] listTableNames(String regex, boolean includeSysTables) 279 throws IOException; 280 281 /** 282 * Get a table descriptor. 283 * 284 * @param tableName as a {@link TableName} 285 * @return the read-only tableDescriptor 286 * @throws org.apache.hadoop.hbase.TableNotFoundException 287 * @throws IOException if a remote or network exception occurs 288 * @deprecated since 2.0 version and will be removed in 3.0 version. 289 * Use {@link #getDescriptor(TableName)}. 290 */ 291 @Deprecated 292 HTableDescriptor getTableDescriptor(TableName tableName) 293 throws TableNotFoundException, IOException; 294 295 /** 296 * Get a table descriptor. 297 * 298 * @param tableName as a {@link TableName} 299 * @return the tableDescriptor 300 * @throws org.apache.hadoop.hbase.TableNotFoundException 301 * @throws IOException if a remote or network exception occurs 302 */ 303 TableDescriptor getDescriptor(TableName tableName) 304 throws TableNotFoundException, IOException; 305 306 /** 307 * Creates a new table. Synchronous operation. 308 * 309 * @param desc table descriptor for table 310 * @throws IllegalArgumentException if the table name is reserved 311 * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running 312 * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent 313 * threads, the table may have been created between test-for-existence and attempt-at-creation). 314 * @throws IOException if a remote or network exception occurs 315 */ 316 default void createTable(TableDescriptor desc) throws IOException { 317 get(createTableAsync(desc), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 318 } 319 320 /** 321 * Creates a new table with the specified number of regions. The start key specified will become 322 * the end key of the first region of the table, and the end key specified will become the start 323 * key of the last region of the table (the first region has a null start key and the last region 324 * has a null end key). BigInteger math will be used to divide the key range specified into enough 325 * segments to make the required number of total regions. Synchronous operation. 326 * 327 * @param desc table descriptor for table 328 * @param startKey beginning of key range 329 * @param endKey end of key range 330 * @param numRegions the total number of regions to create 331 * @throws IllegalArgumentException if the table name is reserved 332 * @throws IOException if a remote or network exception occurs 333 * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running 334 * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent 335 * threads, the table may have been created between test-for-existence and attempt-at-creation). 336 */ 337 void createTable(TableDescriptor desc, byte[] startKey, byte[] endKey, int numRegions) 338 throws IOException; 339 340 /** 341 * Creates a new table with an initial set of empty regions defined by the specified split keys. 342 * The total number of regions created will be the number of split keys plus one. Synchronous 343 * operation. Note : Avoid passing empty split key. 344 * 345 * @param desc table descriptor for table 346 * @param splitKeys array of split keys for the initial regions of the table 347 * @throws IllegalArgumentException if the table name is reserved, if the split keys are repeated 348 * and if the split key has empty byte array. 349 * @throws org.apache.hadoop.hbase.MasterNotRunningException if master is not running 350 * @throws org.apache.hadoop.hbase.TableExistsException if table already exists (If concurrent 351 * threads, the table may have been created between test-for-existence and attempt-at-creation). 352 * @throws IOException if a remote or network exception occurs 353 */ 354 default void createTable(TableDescriptor desc, byte[][] splitKeys) throws IOException { 355 get(createTableAsync(desc, splitKeys), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 356 } 357 358 /** 359 * Creates a new table but does not block and wait for it to come online. You can use 360 * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 361 * ExecutionException if there was an error while executing the operation or TimeoutException in 362 * case the wait timeout was not long enough to allow the operation to complete. 363 * <p/> 364 * Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split 365 * key has empty byte array. 366 * @param desc table descriptor for table 367 * @throws IOException if a remote or network exception occurs 368 * @return the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the 369 * operation to complete. 370 */ 371 Future<Void> createTableAsync(TableDescriptor desc) throws IOException; 372 373 /** 374 * Creates a new table but does not block and wait for it to come online. You can use 375 * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 376 * ExecutionException if there was an error while executing the operation or TimeoutException in 377 * case the wait timeout was not long enough to allow the operation to complete. 378 * <p/> 379 * Throws IllegalArgumentException Bad table name, if the split keys are repeated and if the split 380 * key has empty byte array. 381 * @param desc table descriptor for table 382 * @param splitKeys keys to check if the table has been created with all split keys 383 * @throws IOException if a remote or network exception occurs 384 * @return the result of the async creation. You can use Future.get(long, TimeUnit) to wait on the 385 * operation to complete. 386 */ 387 Future<Void> createTableAsync(TableDescriptor desc, byte[][] splitKeys) throws IOException; 388 389 /** 390 * Deletes a table. Synchronous operation. 391 * @param tableName name of table to delete 392 * @throws IOException if a remote or network exception occurs 393 */ 394 default void deleteTable(TableName tableName) throws IOException { 395 get(deleteTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 396 } 397 398 /** 399 * Deletes the table but does not block and wait for it to be completely removed. 400 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 401 * It may throw ExecutionException if there was an error while executing the operation 402 * or TimeoutException in case the wait timeout was not long enough to allow the 403 * operation to complete. 404 * 405 * @param tableName name of table to delete 406 * @throws IOException if a remote or network exception occurs 407 * @return the result of the async delete. You can use Future.get(long, TimeUnit) 408 * to wait on the operation to complete. 409 */ 410 Future<Void> deleteTableAsync(TableName tableName) throws IOException; 411 412 /** 413 * Deletes tables matching the passed in pattern and wait on completion. Warning: Use this method 414 * carefully, there is no prompting and the effect is immediate. Consider using {@link 415 * #listTableDescriptors(Pattern)} 416 * and {@link #deleteTable(org.apache.hadoop.hbase.TableName)} 417 * 418 * @param regex The regular expression to match table names against 419 * @return Table descriptors for tables that couldn't be deleted. 420 * The return htds are read-only 421 * @throws IOException if a remote or network exception occurs 422 * @see #deleteTables(java.util.regex.Pattern) 423 * @see #deleteTable(org.apache.hadoop.hbase.TableName) 424 * @deprecated since 2.0 version and will be removed in 3.0 version 425 * This is just a trivial helper method without any magic. 426 * Consider using {@link #listTableDescriptors(Pattern)} 427 * and {@link #deleteTable(TableName)} 428 */ 429 @Deprecated 430 HTableDescriptor[] deleteTables(String regex) throws IOException; 431 432 /** 433 * Delete tables matching the passed in pattern and wait on completion. Warning: Use this method 434 * carefully, there is no prompting and the effect is immediate. Consider using {@link 435 * #listTableDescriptors(java.util.regex.Pattern)} and 436 * {@link #deleteTable(org.apache.hadoop.hbase.TableName)} 437 * 438 * @param pattern The pattern to match table names against 439 * @return Table descriptors for tables that couldn't be deleted 440 * The return htds are read-only 441 * @throws IOException if a remote or network exception occurs 442 * @deprecated since 2.0 version and will be removed in 3.0 version 443 * This is just a trivial helper method without any magic. 444 * Consider using {@link #listTableDescriptors(java.util.regex.Pattern)} 445 * and {@link #deleteTable(TableName)} 446 */ 447 @Deprecated 448 HTableDescriptor[] deleteTables(Pattern pattern) throws IOException; 449 450 /** 451 * Truncate a table. 452 * Synchronous operation. 453 * 454 * @param tableName name of table to truncate 455 * @param preserveSplits <code>true</code> if the splits should be preserved 456 * @throws IOException if a remote or network exception occurs 457 */ 458 default void truncateTable(TableName tableName, boolean preserveSplits) throws IOException { 459 get(truncateTableAsync(tableName, preserveSplits), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 460 } 461 462 /** 463 * Truncate the table but does not block and wait for it to be completely enabled. You can use 464 * Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 465 * ExecutionException if there was an error while executing the operation or TimeoutException in 466 * case the wait timeout was not long enough to allow the operation to complete. 467 * @param tableName name of table to delete 468 * @param preserveSplits <code>true</code> if the splits should be preserved 469 * @throws IOException if a remote or network exception occurs 470 * @return the result of the async truncate. You can use Future.get(long, TimeUnit) to wait on the 471 * operation to complete. 472 */ 473 Future<Void> truncateTableAsync(TableName tableName, boolean preserveSplits) 474 throws IOException; 475 476 /** 477 * Enable a table. May timeout. Use {@link #enableTableAsync(org.apache.hadoop.hbase.TableName)} 478 * and {@link #isTableEnabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in 479 * disabled state for it to be enabled. 480 * @param tableName name of the table 481 * @throws IOException if a remote or network exception occurs There could be couple types of 482 * IOException TableNotFoundException means the table doesn't exist. 483 * TableNotDisabledException means the table isn't in disabled state. 484 * @see #isTableEnabled(org.apache.hadoop.hbase.TableName) 485 * @see #disableTable(org.apache.hadoop.hbase.TableName) 486 * @see #enableTableAsync(org.apache.hadoop.hbase.TableName) 487 */ 488 default void enableTable(TableName tableName) throws IOException { 489 get(enableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 490 } 491 492 /** 493 * Enable the table but does not block and wait for it to be completely enabled. 494 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 495 * It may throw ExecutionException if there was an error while executing the operation 496 * or TimeoutException in case the wait timeout was not long enough to allow the 497 * operation to complete. 498 * 499 * @param tableName name of table to delete 500 * @throws IOException if a remote or network exception occurs 501 * @return the result of the async enable. You can use Future.get(long, TimeUnit) 502 * to wait on the operation to complete. 503 */ 504 Future<Void> enableTableAsync(TableName tableName) throws IOException; 505 506 /** 507 * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method 508 * carefully, there is no prompting and the effect is immediate. Consider using {@link 509 * #listTableDescriptors(Pattern)} and {@link #enableTable(org.apache.hadoop.hbase.TableName)} 510 * 511 * @param regex The regular expression to match table names against 512 * @throws IOException if a remote or network exception occurs 513 * @return Table descriptors for tables that couldn't be enabled. 514 * The return HTDs are read-only. 515 * @see #enableTables(java.util.regex.Pattern) 516 * @see #enableTable(org.apache.hadoop.hbase.TableName) 517 * @deprecated since 2.0 version and will be removed in 3.0 version 518 * This is just a trivial helper method without any magic. 519 * Consider using {@link #listTableDescriptors(Pattern)} 520 * and {@link #enableTable(org.apache.hadoop.hbase.TableName)} 521 */ 522 @Deprecated 523 HTableDescriptor[] enableTables(String regex) throws IOException; 524 525 /** 526 * Enable tables matching the passed in pattern and wait on completion. Warning: Use this method 527 * carefully, there is no prompting and the effect is immediate. Consider using {@link 528 * #listTableDescriptors(java.util.regex.Pattern)} and 529 * {@link #enableTable(org.apache.hadoop.hbase.TableName)} 530 * 531 * @param pattern The pattern to match table names against 532 * @throws IOException if a remote or network exception occurs 533 * @return Table descriptors for tables that couldn't be enabled. 534 * The return HTDs are read-only. 535 * @deprecated since 2.0 version and will be removed in 3.0 version 536 * This is just a trivial helper method without any magic. 537 * Consider using {@link #listTableDescriptors(java.util.regex.Pattern)} 538 * and {@link #enableTable(org.apache.hadoop.hbase.TableName)} 539 */ 540 @Deprecated 541 HTableDescriptor[] enableTables(Pattern pattern) throws IOException; 542 543 /** 544 * Disable the table but does not block and wait for it to be completely disabled. 545 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 546 * It may throw ExecutionException if there was an error while executing the operation 547 * or TimeoutException in case the wait timeout was not long enough to allow the 548 * operation to complete. 549 * 550 * @param tableName name of table to delete 551 * @throws IOException if a remote or network exception occurs 552 * @return the result of the async disable. You can use Future.get(long, TimeUnit) 553 * to wait on the operation to complete. 554 */ 555 Future<Void> disableTableAsync(TableName tableName) throws IOException; 556 557 /** 558 * Disable table and wait on completion. May timeout eventually. Use 559 * {@link #disableTableAsync(org.apache.hadoop.hbase.TableName)} and 560 * {@link #isTableDisabled(org.apache.hadoop.hbase.TableName)} instead. The table has to be in 561 * enabled state for it to be disabled. 562 * @param tableName 563 * @throws IOException There could be couple types of IOException TableNotFoundException means the 564 * table doesn't exist. TableNotEnabledException means the table isn't in enabled state. 565 */ 566 default void disableTable(TableName tableName) throws IOException { 567 get(disableTableAsync(tableName), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 568 } 569 570 /** 571 * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method 572 * carefully, there is no prompting and the effect is immediate. Consider using {@link 573 * #listTableDescriptors(Pattern)} and {@link #disableTable(org.apache.hadoop.hbase.TableName)} 574 * 575 * @param regex The regular expression to match table names against 576 * @return Table descriptors for tables that couldn't be disabled 577 * The return htds are read-only 578 * @throws IOException if a remote or network exception occurs 579 * @see #disableTables(java.util.regex.Pattern) 580 * @see #disableTable(org.apache.hadoop.hbase.TableName) 581 * @deprecated since 2.0 version and will be removed in 3.0 version 582 * This is just a trivial helper method without any magic. 583 * Consider using {@link #listTableDescriptors(Pattern)} 584 * and {@link #disableTable(org.apache.hadoop.hbase.TableName)} 585 */ 586 @Deprecated 587 HTableDescriptor[] disableTables(String regex) throws IOException; 588 589 /** 590 * Disable tables matching the passed in pattern and wait on completion. Warning: Use this method 591 * carefully, there is no prompting and the effect is immediate. Consider using {@link 592 * #listTableDescriptors(java.util.regex.Pattern)} and 593 * {@link #disableTable(org.apache.hadoop.hbase.TableName)} 594 * 595 * @param pattern The pattern to match table names against 596 * @return Table descriptors for tables that couldn't be disabled 597 * The return htds are read-only 598 * @throws IOException if a remote or network exception occurs 599 * @deprecated since 2.0 version and will be removed in 3.0 version 600 * This is just a trivial helper method without any magic. 601 * Consider using {@link #listTableDescriptors(java.util.regex.Pattern)} 602 * and {@link #disableTable(org.apache.hadoop.hbase.TableName)} 603 */ 604 @Deprecated 605 HTableDescriptor[] disableTables(Pattern pattern) throws IOException; 606 607 /** 608 * @param tableName name of table to check 609 * @return <code>true</code> if table is on-line 610 * @throws IOException if a remote or network exception occurs 611 */ 612 boolean isTableEnabled(TableName tableName) throws IOException; 613 614 /** 615 * @param tableName name of table to check 616 * @return <code>true</code> if table is off-line 617 * @throws IOException if a remote or network exception occurs 618 */ 619 boolean isTableDisabled(TableName tableName) throws IOException; 620 621 /** 622 * @param tableName name of table to check 623 * @return <code>true</code> if all regions of the table are available 624 * @throws IOException if a remote or network exception occurs 625 */ 626 boolean isTableAvailable(TableName tableName) throws IOException; 627 628 /** 629 * Use this api to check if the table has been created with the specified number of splitkeys 630 * which was used while creating the given table. Note : If this api is used after a table's 631 * region gets splitted, the api may return <code>false</code>. 632 * 633 * @param tableName name of table to check 634 * @param splitKeys keys to check if the table has been created with all split keys 635 * @throws IOException if a remote or network excpetion occurs 636 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #isTableAvailable(TableName)} 637 */ 638 @Deprecated 639 boolean isTableAvailable(TableName tableName, byte[][] splitKeys) throws IOException; 640 641 /** 642 * Get the status of an <code>alter</code> (a.k.a <code>modify</code>) command - indicates how 643 * many regions have received the updated schema Asynchronous operation. 644 * 645 * @param tableName TableName instance 646 * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are 647 * yet to be updated Pair.getSecond() is the total number of regions of the table 648 * @throws IOException if a remote or network exception occurs 649 * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future 650 * on an operation. 651 */ 652 @Deprecated 653 Pair<Integer, Integer> getAlterStatus(TableName tableName) throws IOException; 654 655 /** 656 * Get the status of <code>alter</code> (a.k.a <code>modify</code>) command - indicates how many 657 * regions have received the updated schema Asynchronous operation. 658 * 659 * @param tableName name of the table to get the status of 660 * @return Pair indicating the number of regions updated Pair.getFirst() is the regions that are 661 * yet to be updated Pair.getSecond() is the total number of regions of the table 662 * @throws IOException if a remote or network exception occurs 663 * @deprecated Since 2.0.0. Will be removed in 3.0.0. No longer needed now you get a Future 664 * on an operation. 665 */ 666 @Deprecated 667 Pair<Integer, Integer> getAlterStatus(byte[] tableName) throws IOException; 668 669 /** 670 * Add a column family to an existing table. Synchronous operation. 671 * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it 672 * returns a {@link Future} from which you can learn whether success or failure. 673 * 674 * @param tableName name of the table to add column family to 675 * @param columnFamily column family descriptor of column family to be added 676 * @throws IOException if a remote or network exception occurs 677 * @deprecated As of release 2.0.0. 678 * This will be removed in HBase 3.0.0. 679 * Use {@link #addColumnFamily(TableName, ColumnFamilyDescriptor)}. 680 */ 681 @Deprecated 682 default void addColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) 683 throws IOException { 684 addColumnFamily(tableName, columnFamily); 685 } 686 687 /** 688 * Add a column family to an existing table. Synchronous operation. 689 * Use {@link #addColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it 690 * returns a {@link Future} from which you can learn whether success or failure. 691 * 692 * @param tableName name of the table to add column family to 693 * @param columnFamily column family descriptor of column family to be added 694 * @throws IOException if a remote or network exception occurs 695 */ 696 default void addColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) 697 throws IOException { 698 get(addColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 699 } 700 701 /** 702 * Add a column family to an existing table. Asynchronous operation. 703 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 704 * It may throw ExecutionException if there was an error while executing the operation 705 * or TimeoutException in case the wait timeout was not long enough to allow the 706 * operation to complete. 707 * 708 * @param tableName name of the table to add column family to 709 * @param columnFamily column family descriptor of column family to be added 710 * @throws IOException if a remote or network exception occurs 711 * @return the result of the async add column family. You can use Future.get(long, TimeUnit) to 712 * wait on the operation to complete. 713 */ 714 Future<Void> addColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) 715 throws IOException; 716 717 /** 718 * Delete a column family from a table. Synchronous operation. 719 * Use {@link #deleteColumnFamily(TableName, byte[])} instead because it 720 * returns a {@link Future} from which you can learn whether success or failure. 721 * 722 * @param tableName name of table 723 * @param columnFamily name of column family to be deleted 724 * @throws IOException if a remote or network exception occurs 725 * @deprecated As of release 2.0.0. 726 * This will be removed in HBase 3.0.0. 727 * Use {@link #deleteColumnFamily(TableName, byte[])}}. 728 */ 729 @Deprecated 730 void deleteColumn(TableName tableName, byte[] columnFamily) throws IOException; 731 732 /** 733 * Delete a column family from a table. Synchronous operation. 734 * Use {@link #deleteColumnFamily(TableName, byte[])} instead because it 735 * returns a {@link Future} from which you can learn whether success or failure. 736 * @param tableName name of table 737 * @param columnFamily name of column family to be deleted 738 * @throws IOException if a remote or network exception occurs 739 */ 740 default void deleteColumnFamily(TableName tableName, byte[] columnFamily) throws IOException { 741 get(deleteColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), 742 TimeUnit.MILLISECONDS); 743 } 744 745 /** 746 * Delete a column family from a table. Asynchronous operation. 747 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 748 * It may throw ExecutionException if there was an error while executing the operation 749 * or TimeoutException in case the wait timeout was not long enough to allow the 750 * operation to complete. 751 * 752 * @param tableName name of table 753 * @param columnFamily name of column family to be deleted 754 * @throws IOException if a remote or network exception occurs 755 * @return the result of the async delete column family. You can use Future.get(long, TimeUnit) to 756 * wait on the operation to complete. 757 */ 758 Future<Void> deleteColumnFamilyAsync(TableName tableName, byte[] columnFamily) 759 throws IOException; 760 761 /** 762 * Modify an existing column family on a table. Synchronous operation. Use 763 * {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it returns 764 * a {@link Future} from which you can learn whether success or failure. 765 * @param tableName name of table 766 * @param columnFamily new column family descriptor to use 767 * @throws IOException if a remote or network exception occurs 768 * @deprecated As of release 2.0.0. 769 * This will be removed in HBase 3.0.0. 770 * Use {@link #modifyColumnFamily(TableName, ColumnFamilyDescriptor)}. 771 */ 772 @Deprecated 773 default void modifyColumn(TableName tableName, ColumnFamilyDescriptor columnFamily) 774 throws IOException { 775 modifyColumnFamily(tableName, columnFamily); 776 } 777 778 /** 779 * Modify an existing column family on a table. Synchronous operation. 780 * Use {@link #modifyColumnFamilyAsync(TableName, ColumnFamilyDescriptor)} instead because it 781 * returns a {@link Future} from which you can learn whether success or failure. 782 * @param tableName name of table 783 * @param columnFamily new column family descriptor to use 784 * @throws IOException if a remote or network exception occurs 785 */ 786 default void modifyColumnFamily(TableName tableName, ColumnFamilyDescriptor columnFamily) 787 throws IOException { 788 get(modifyColumnFamilyAsync(tableName, columnFamily), getSyncWaitTimeout(), 789 TimeUnit.MILLISECONDS); 790 } 791 792 /** 793 * Modify an existing column family on a table. Asynchronous operation. 794 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 795 * It may throw ExecutionException if there was an error while executing the operation 796 * or TimeoutException in case the wait timeout was not long enough to allow the 797 * operation to complete. 798 * 799 * @param tableName name of table 800 * @param columnFamily new column family descriptor to use 801 * @throws IOException if a remote or network exception occurs 802 * @return the result of the async modify column family. You can use Future.get(long, TimeUnit) to 803 * wait on the operation to complete. 804 */ 805 Future<Void> modifyColumnFamilyAsync(TableName tableName, ColumnFamilyDescriptor columnFamily) 806 throws IOException; 807 808 /** 809 * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins. 810 * 811 * @param regionname region name to close 812 * @param serverName Deprecated. Not used. 813 * @throws IOException if a remote or network exception occurs 814 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. 815 * Use {@link #unassign(byte[], boolean)}. 816 */ 817 @Deprecated 818 void closeRegion(String regionname, String serverName) throws IOException; 819 820 /** 821 * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins. 822 * 823 * @param regionname region name to close 824 * @param serverName Deprecated. Not used. 825 * @throws IOException if a remote or network exception occurs 826 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. 827 * Use {@link #unassign(byte[], boolean)}. 828 */ 829 @Deprecated 830 void closeRegion(byte[] regionname, String serverName) throws IOException; 831 832 /** 833 * Uses {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins. 834 * 835 * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name 836 * suffix: e.g. if regionname is 837 * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>, 838 * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>. 839 * @param serverName Deprecated. Not used. 840 * @return Deprecated. Returns <code>true</code> always. 841 * @throws IOException if a remote or network exception occurs 842 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. 843 * Use {@link #unassign(byte[], boolean)}. 844 */ 845 @Deprecated 846 boolean closeRegionWithEncodedRegionName(String encodedRegionName, String serverName) 847 throws IOException; 848 849 /** 850 * Used {@link #unassign(byte[], boolean)} to unassign the region. For expert-admins. 851 * 852 * @param sn Deprecated. Not used. 853 * @throws IOException if a remote or network exception occurs 854 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 855 * (<a href="https://issues.apache.org/jira/browse/HBASE-18231">HBASE-18231</a>). 856 * Use {@link #unassign(byte[], boolean)}. 857 */ 858 @Deprecated 859 void closeRegion(final ServerName sn, final HRegionInfo hri) throws IOException; 860 861 /** 862 * Get all the online regions on a region server. 863 * @throws IOException if a remote or network exception occurs 864 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 865 * (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>). 866 * Use {@link #getRegions(ServerName sn)}. 867 */ 868 @Deprecated 869 List<HRegionInfo> getOnlineRegions(ServerName sn) throws IOException; 870 871 /** 872 * Get all the online regions on a region server. 873 * 874 * @return List of {@link RegionInfo} 875 * @throws IOException if a remote or network exception occurs 876 */ 877 List<RegionInfo> getRegions(ServerName serverName) throws IOException; 878 879 /** 880 * Flush a table. Synchronous operation. 881 * 882 * @param tableName table to flush 883 * @throws IOException if a remote or network exception occurs 884 */ 885 void flush(TableName tableName) throws IOException; 886 887 /** 888 * Flush the specified column family stores on all regions of the passed table. 889 * This runs as a synchronous operation. 890 * 891 * @param tableName table to flush 892 * @param columnFamily column family within a table 893 * @throws IOException if a remote or network exception occurs 894 */ 895 void flush(TableName tableName, byte[] columnFamily) throws IOException; 896 897 /** 898 * Flush an individual region. Synchronous operation. 899 * 900 * @param regionName region to flush 901 * @throws IOException if a remote or network exception occurs 902 */ 903 void flushRegion(byte[] regionName) throws IOException; 904 905 /** 906 * Flush a column family within a region. Synchronous operation. 907 * 908 * @param regionName region to flush 909 * @param columnFamily column family within a region 910 * @throws IOException if a remote or network exception occurs 911 */ 912 void flushRegion(byte[] regionName, byte[] columnFamily) throws IOException; 913 914 /** 915 * Flush all regions on the region server. Synchronous operation. 916 * @param serverName the region server name to flush 917 * @throws IOException if a remote or network exception occurs 918 */ 919 void flushRegionServer(ServerName serverName) throws IOException; 920 921 /** 922 * Compact a table. Asynchronous operation in that this method requests that a 923 * Compaction run and then it returns. It does not wait on the completion of Compaction 924 * (it can take a while). 925 * 926 * @param tableName table to compact 927 * @throws IOException if a remote or network exception occurs 928 */ 929 void compact(TableName tableName) throws IOException; 930 931 /** 932 * Compact an individual region. Asynchronous operation in that this method requests that a 933 * Compaction run and then it returns. It does not wait on the completion of Compaction 934 * (it can take a while). 935 * 936 * @param regionName region to compact 937 * @throws IOException if a remote or network exception occurs 938 */ 939 void compactRegion(byte[] regionName) throws IOException; 940 941 /** 942 * Compact a column family within a table. Asynchronous operation in that this method requests 943 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 944 * (it can take a while). 945 * 946 * @param tableName table to compact 947 * @param columnFamily column family within a table 948 * @throws IOException if a remote or network exception occurs 949 */ 950 void compact(TableName tableName, byte[] columnFamily) 951 throws IOException; 952 953 /** 954 * Compact a column family within a region. Asynchronous operation in that this method requests 955 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 956 * (it can take a while). 957 * 958 * @param regionName region to compact 959 * @param columnFamily column family within a region 960 * @throws IOException if a remote or network exception occurs 961 */ 962 void compactRegion(byte[] regionName, byte[] columnFamily) 963 throws IOException; 964 965 /** 966 * Compact a table. Asynchronous operation in that this method requests that a 967 * Compaction run and then it returns. It does not wait on the completion of Compaction 968 * (it can take a while). 969 * 970 * @param tableName table to compact 971 * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} 972 * @throws IOException if a remote or network exception occurs 973 * @throws InterruptedException 974 */ 975 void compact(TableName tableName, CompactType compactType) 976 throws IOException, InterruptedException; 977 978 /** 979 * Compact a column family within a table. Asynchronous operation in that this method 980 * requests that a Compaction run and then it returns. It does not wait on the 981 * completion of Compaction (it can take a while). 982 * 983 * @param tableName table to compact 984 * @param columnFamily column family within a table 985 * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} 986 * @throws IOException if not a mob column family or if a remote or network exception occurs 987 * @throws InterruptedException 988 */ 989 void compact(TableName tableName, byte[] columnFamily, CompactType compactType) 990 throws IOException, InterruptedException; 991 992 /** 993 * Major compact a table. Asynchronous operation in that this method requests 994 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 995 * (it can take a while). 996 * 997 * @param tableName table to major compact 998 * @throws IOException if a remote or network exception occurs 999 */ 1000 void majorCompact(TableName tableName) throws IOException; 1001 1002 /** 1003 * Major compact a table or an individual region. Asynchronous operation in that this method requests 1004 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 1005 * (it can take a while). 1006 * 1007 * @param regionName region to major compact 1008 * @throws IOException if a remote or network exception occurs 1009 */ 1010 void majorCompactRegion(byte[] regionName) throws IOException; 1011 1012 /** 1013 * Major compact a column family within a table. Asynchronous operation in that this method requests 1014 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 1015 * (it can take a while). 1016 * 1017 * @param tableName table to major compact 1018 * @param columnFamily column family within a table 1019 * @throws IOException if a remote or network exception occurs 1020 */ 1021 void majorCompact(TableName tableName, byte[] columnFamily) 1022 throws IOException; 1023 1024 /** 1025 * Major compact a column family within region. Asynchronous operation in that this method requests 1026 * that a Compaction run and then it returns. It does not wait on the completion of Compaction 1027 * (it can take a while). 1028 * 1029 * @param regionName egion to major compact 1030 * @param columnFamily column family within a region 1031 * @throws IOException if a remote or network exception occurs 1032 */ 1033 void majorCompactRegion(byte[] regionName, byte[] columnFamily) 1034 throws IOException; 1035 1036 /** 1037 * Major compact a table. Asynchronous operation in that this method requests that a 1038 * Compaction run and then it returns. It does not wait on the completion of Compaction 1039 * (it can take a while). 1040 * 1041 * @param tableName table to compact 1042 * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} 1043 * @throws IOException if a remote or network exception occurs 1044 * @throws InterruptedException 1045 */ 1046 void majorCompact(TableName tableName, CompactType compactType) 1047 throws IOException, InterruptedException; 1048 1049 /** 1050 * Major compact a column family within a table. Asynchronous operation in that this method requests that a 1051 * Compaction run and then it returns. It does not wait on the completion of Compaction 1052 * (it can take a while). 1053 * 1054 * @param tableName table to compact 1055 * @param columnFamily column family within a table 1056 * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} 1057 * @throws IOException if not a mob column family or if a remote or network exception occurs 1058 * @throws InterruptedException 1059 */ 1060 void majorCompact(TableName tableName, byte[] columnFamily, CompactType compactType) 1061 throws IOException, InterruptedException; 1062 1063 /** 1064 * Compact all regions on the region server. Asynchronous operation in that this method requests 1065 * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it 1066 * can take a while). 1067 * @param sn the region server name 1068 * @param major if it's major compaction 1069 * @throws IOException if a remote or network exception occurs 1070 * @throws InterruptedException 1071 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. Use 1072 * {@link #compactRegionServer(ServerName)} or 1073 * {@link #majorCompactRegionServer(ServerName)}. 1074 */ 1075 @Deprecated 1076 default void compactRegionServer(ServerName sn, boolean major) throws IOException, 1077 InterruptedException { 1078 if (major) { 1079 majorCompactRegionServer(sn); 1080 } else { 1081 compactRegionServer(sn); 1082 } 1083 } 1084 1085 /** 1086 * Turn the compaction on or off. Disabling compactions will also interrupt any currently ongoing 1087 * compactions. This state is ephemeral. The setting will be lost on restart. Compaction 1088 * can also be enabled/disabled by modifying configuration hbase.regionserver.compaction.enabled 1089 * in hbase-site.xml. 1090 * 1091 * @param switchState Set to <code>true</code> to enable, <code>false</code> to disable. 1092 * @param serverNamesList list of region servers. 1093 * @return Previous compaction states for region servers 1094 */ 1095 Map<ServerName, Boolean> compactionSwitch(boolean switchState, List<String> serverNamesList) 1096 throws IOException; 1097 1098 /** 1099 * Compact all regions on the region server. Asynchronous operation in that this method requests 1100 * that a Compaction run and then it returns. It does not wait on the completion of Compaction (it 1101 * can take a while). 1102 * @param serverName the region server name 1103 * @throws IOException if a remote or network exception occurs 1104 */ 1105 void compactRegionServer(ServerName serverName) throws IOException; 1106 1107 /** 1108 * Major compact all regions on the region server. Asynchronous operation in that this method 1109 * requests that a Compaction run and then it returns. It does not wait on the completion of 1110 * Compaction (it can take a while). 1111 * @param serverName the region server name 1112 * @throws IOException if a remote or network exception occurs 1113 */ 1114 void majorCompactRegionServer(ServerName serverName) throws IOException; 1115 1116 /** 1117 * Move the region <code>encodedRegionName</code> to a random server. 1118 * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name 1119 * suffix: e.g. if regionname is 1120 * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>, 1121 * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>. 1122 * @throws IOException if we can't find a region named <code>encodedRegionName</code> 1123 */ 1124 void move(byte[] encodedRegionName) throws IOException; 1125 1126 /** 1127 * Move the region <code>rencodedRegionName</code> to <code>destServerName</code>. 1128 * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name 1129 * suffix: e.g. if regionname is 1130 * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>, 1131 * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>. 1132 * @param destServerName The servername of the destination regionserver. If passed the empty byte 1133 * array we'll assign to a random server. A server name is made of host, port and 1134 * startcode. Here is an example: <code> host187.example.com,60020,1289493121758</code> 1135 * @throws IOException if we can't find a region named <code>encodedRegionName</code> 1136 * @deprecated since 2.2.0 and will be removed in 4.0.0. Use {@link #move(byte[], ServerName)} 1137 * instead. And if you want to move the region to a random server, please use 1138 * {@link #move(byte[])}. 1139 * @see <a href="https://issues.apache.org/jira/browse/HBASE-22108">HBASE-22108</a> 1140 */ 1141 @Deprecated 1142 default void move(byte[] encodedRegionName, byte[] destServerName) throws IOException { 1143 if (destServerName == null || destServerName.length == 0) { 1144 move(encodedRegionName); 1145 } else { 1146 move(encodedRegionName, ServerName.valueOf(Bytes.toString(destServerName))); 1147 } 1148 } 1149 1150 /** 1151 * Move the region <code>rencodedRegionName</code> to <code>destServerName</code>. 1152 * @param encodedRegionName The encoded region name; i.e. the hash that makes up the region name 1153 * suffix: e.g. if regionname is 1154 * <code>TestTable,0094429456,1289497600452.527db22f95c8a9e0116f0cc13c680396.</code>, 1155 * then the encoded region name is: <code>527db22f95c8a9e0116f0cc13c680396</code>. 1156 * @param destServerName The servername of the destination regionserver. A server name is made of 1157 * host, port and startcode. Here is an example: 1158 * <code> host187.example.com,60020,1289493121758</code> 1159 * @throws IOException if we can't find a region named <code>encodedRegionName</code> 1160 */ 1161 void move(byte[] encodedRegionName, ServerName destServerName) throws IOException; 1162 1163 /** 1164 * Assign a Region. 1165 * @param regionName Region name to assign. 1166 * @throws IOException if a remote or network exception occurs 1167 */ 1168 void assign(byte[] regionName) throws IOException; 1169 1170 /** 1171 * Unassign a Region. 1172 * @param regionName Region name to assign. 1173 * @throws IOException if a remote or network exception occurs 1174 */ 1175 void unassign(byte[] regionName) throws IOException; 1176 1177 /** 1178 * Unassign a region from current hosting regionserver. Region will then be assigned to a 1179 * regionserver chosen at random. Region could be reassigned back to the same server. Use {@link 1180 * #move(byte[], ServerName)} if you want to control the region movement. 1181 * 1182 * @param regionName Region to unassign. Will clear any existing RegionPlan if one found. 1183 * @param force If <code>true</code>, force unassign (Will remove region from regions-in-transition too if 1184 * present. If results in double assignment use hbck -fix to resolve. To be used by experts). 1185 * @throws IOException if a remote or network exception occurs 1186 * @deprecated since 2.4.0 and will be removed in 4.0.0. Use {@link #unassign(byte[])} 1187 * instead. 1188 * @see <a href="https://issues.apache.org/jira/browse/HBASE-24875">HBASE-24875</a> 1189 */ 1190 @Deprecated 1191 default void unassign(byte[] regionName, boolean force) throws IOException { 1192 unassign(regionName); 1193 } 1194 1195 /** 1196 * Offline specified region from master's in-memory state. It will not attempt to reassign the 1197 * region as in unassign. This API can be used when a region not served by any region server and 1198 * still online as per Master's in memory state. If this API is incorrectly used on active region 1199 * then master will loose track of that region. This is a special method that should be used by 1200 * experts or hbck. 1201 * 1202 * @param regionName Region to offline. 1203 * @throws IOException if a remote or network exception occurs 1204 */ 1205 void offline(byte[] regionName) throws IOException; 1206 1207 /** 1208 * Turn the load balancer on or off. 1209 * 1210 * @param synchronous If <code>true</code>, it waits until current balance() call, if 1211 * outstanding, to return. 1212 * @return Previous balancer value 1213 * @throws IOException if a remote or network exception occurs 1214 * @deprecated Since 2.0.0. Will be removed in 3.0.0. 1215 * Use {@link #balancerSwitch(boolean, boolean)} instead. 1216 */ 1217 @Deprecated 1218 default boolean setBalancerRunning(boolean on, boolean synchronous) throws IOException { 1219 return balancerSwitch(on, synchronous); 1220 } 1221 1222 /** 1223 * Turn the load balancer on or off. 1224 * @param onOrOff Set to <code>true</code> to enable, <code>false</code> to disable. 1225 * @param synchronous If <code>true</code>, it waits until current balance() call, if 1226 * outstanding, to return. 1227 * @return Previous balancer value 1228 * @throws IOException if a remote or network exception occurs 1229 */ 1230 boolean balancerSwitch(boolean onOrOff, boolean synchronous) 1231 throws IOException; 1232 1233 /** 1234 * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the 1235 * reassignments. Can NOT run for various reasons. Check logs. 1236 * 1237 * @return <code>true</code> if balancer ran, <code>false</code> otherwise. 1238 * @throws IOException if a remote or network exception occurs 1239 * @deprecated Since 2.0.0. Will be removed in 3.0.0. 1240 * Use {@link #balance()} instead. 1241 */ 1242 @Deprecated 1243 default boolean balancer() throws IOException { 1244 return balance(); 1245 } 1246 1247 /** 1248 * Invoke the balancer. Will run the balancer and if regions to move, it will go ahead and do the 1249 * reassignments. Can NOT run for various reasons. Check logs. 1250 * 1251 * @return <code>true</code> if balancer ran, <code>false</code> otherwise. 1252 * @throws IOException if a remote or network exception occurs 1253 */ 1254 boolean balance() throws IOException; 1255 1256 /** 1257 * Invoke the balancer. Will run the balancer and if regions to move, it will 1258 * go ahead and do the reassignments. If there is region in transition, force parameter of true 1259 * would still run balancer. Can *not* run for other reasons. Check 1260 * logs. 1261 * @param force whether we should force balance even if there is region in transition 1262 * @return <code>true</code> if balancer ran, <code>false</code> otherwise. 1263 * @throws IOException if a remote or network exception occurs 1264 * @deprecated Since 2.0.0. Will be removed in 3.0.0. 1265 * Use {@link #balance(boolean)} instead. 1266 */ 1267 @Deprecated 1268 default boolean balancer(boolean force) throws IOException { 1269 return balance(force); 1270 } 1271 1272 /** 1273 * Invoke the balancer. Will run the balancer and if regions to move, it will 1274 * go ahead and do the reassignments. If there is region in transition, force parameter of true 1275 * would still run balancer. Can *not* run for other reasons. Check 1276 * logs. 1277 * @param force whether we should force balance even if there is region in transition 1278 * @return <code>true</code> if balancer ran, <code>false</code> otherwise. 1279 * @throws IOException if a remote or network exception occurs 1280 */ 1281 boolean balance(boolean force) throws IOException; 1282 1283 /** 1284 * Query the current state of the balancer. 1285 * 1286 * @return <code>true</code> if the balancer is enabled, <code>false</code> otherwise. 1287 * @throws IOException if a remote or network exception occurs 1288 */ 1289 boolean isBalancerEnabled() throws IOException; 1290 1291 /** 1292 * Clear all the blocks corresponding to this table from BlockCache. For expert-admins. 1293 * Calling this API will drop all the cached blocks specific to a table from BlockCache. 1294 * This can significantly impact the query performance as the subsequent queries will 1295 * have to retrieve the blocks from underlying filesystem. 1296 * 1297 * @param tableName table to clear block cache 1298 * @return CacheEvictionStats related to the eviction 1299 * @throws IOException if a remote or network exception occurs 1300 */ 1301 CacheEvictionStats clearBlockCache(final TableName tableName) throws IOException; 1302 1303 /** 1304 * Invoke region normalizer. Can NOT run for various reasons. Check logs. 1305 * This is a non-blocking invocation to region normalizer. If return value is true, it means 1306 * the request was submitted successfully. We need to check logs for the details of which regions 1307 * were split/merged. 1308 * 1309 * @return {@code true} if region normalizer ran, {@code false} otherwise. 1310 * @throws IOException if a remote or network exception occurs 1311 */ 1312 default boolean normalize() throws IOException { 1313 return normalize(new NormalizeTableFilterParams.Builder().build()); 1314 } 1315 1316 /** 1317 * Invoke region normalizer. Can NOT run for various reasons. Check logs. 1318 * This is a non-blocking invocation to region normalizer. If return value is true, it means 1319 * the request was submitted successfully. We need to check logs for the details of which regions 1320 * were split/merged. 1321 * 1322 * @param ntfp limit to tables matching the specified filter. 1323 * @return {@code true} if region normalizer ran, {@code false} otherwise. 1324 * @throws IOException if a remote or network exception occurs 1325 */ 1326 boolean normalize(NormalizeTableFilterParams ntfp) throws IOException; 1327 1328 /** 1329 * Query the current state of the region normalizer. 1330 * 1331 * @return <code>true</code> if region normalizer is enabled, <code>false</code> otherwise. 1332 * @throws IOException if a remote or network exception occurs 1333 */ 1334 boolean isNormalizerEnabled() throws IOException; 1335 1336 /** 1337 * Turn region normalizer on or off. 1338 * 1339 * @return Previous normalizer value 1340 * @throws IOException if a remote or network exception occurs 1341 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #normalizerSwitch(boolean)}} 1342 * instead. 1343 */ 1344 @Deprecated 1345 default boolean setNormalizerRunning(boolean on) throws IOException { 1346 return normalizerSwitch(on); 1347 } 1348 1349 /** 1350 * Turn region normalizer on or off. 1351 * 1352 * @return Previous normalizer value 1353 * @throws IOException if a remote or network exception occurs 1354 */ 1355 boolean normalizerSwitch (boolean on) throws IOException; 1356 1357 /** 1358 * Enable/Disable the catalog janitor. 1359 * 1360 * @param enable if <code>true</code> enables the catalog janitor 1361 * @return the previous state 1362 * @throws IOException if a remote or network exception occurs 1363 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #catalogJanitorSwitch(boolean)}} 1364 * instead. 1365 */ 1366 @Deprecated 1367 default boolean enableCatalogJanitor(boolean enable) throws IOException { 1368 return catalogJanitorSwitch(enable); 1369 } 1370 1371 /** 1372 * Enable/Disable the catalog janitor/ 1373 * 1374 * @param onOrOff if <code>true</code> enables the catalog janitor 1375 * @return the previous state 1376 * @throws IOException if a remote or network exception occurs 1377 */ 1378 boolean catalogJanitorSwitch(boolean onOrOff) throws IOException; 1379 1380 /** 1381 * Ask for a scan of the catalog table. 1382 * 1383 * @return the number of entries cleaned. Returns -1 if previous run is in progress. 1384 * @throws IOException if a remote or network exception occurs 1385 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #runCatalogJanitor()}} 1386 * instead. 1387 */ 1388 @Deprecated 1389 default int runCatalogScan() throws IOException { 1390 return runCatalogJanitor(); 1391 } 1392 1393 /** 1394 * Ask for a scan of the catalog table. 1395 * 1396 * @return the number of entries cleaned 1397 * @throws IOException if a remote or network exception occurs 1398 */ 1399 int runCatalogJanitor() throws IOException; 1400 1401 /** 1402 * Query on the catalog janitor state (Enabled/Disabled?). 1403 * 1404 * @throws IOException if a remote or network exception occurs 1405 */ 1406 boolean isCatalogJanitorEnabled() throws IOException; 1407 1408 /** 1409 * Enable/Disable the cleaner chore. 1410 * 1411 * @param on if <code>true</code> enables the cleaner chore 1412 * @return the previous state 1413 * @throws IOException if a remote or network exception occurs 1414 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #cleanerChoreSwitch(boolean)}} 1415 * instead. 1416 */ 1417 @Deprecated 1418 default boolean setCleanerChoreRunning(boolean on) throws IOException { 1419 return cleanerChoreSwitch(on); 1420 } 1421 1422 /** 1423 * Enable/Disable the cleaner chore. 1424 * 1425 * @param onOrOff if <code>true</code> enables the cleaner chore 1426 * @return the previous state 1427 * @throws IOException if a remote or network exception occurs 1428 */ 1429 boolean cleanerChoreSwitch(boolean onOrOff) throws IOException; 1430 1431 /** 1432 * Ask for cleaner chore to run. 1433 * 1434 * @return <code>true</code> if cleaner chore ran, <code>false</code> otherwise 1435 * @throws IOException if a remote or network exception occurs 1436 */ 1437 boolean runCleanerChore() throws IOException; 1438 1439 /** 1440 * Query on the cleaner chore state (Enabled/Disabled?). 1441 * 1442 * @throws IOException if a remote or network exception occurs 1443 */ 1444 boolean isCleanerChoreEnabled() throws IOException; 1445 1446 /** 1447 * Merge two regions. Asynchronous operation. 1448 * 1449 * @param nameOfRegionA encoded or full name of region a 1450 * @param nameOfRegionB encoded or full name of region b 1451 * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two 1452 * adjacent regions 1453 * @throws IOException if a remote or network exception occurs 1454 * @deprecated Since 2.0. Will be removed in 3.0. Use 1455 * {@link #mergeRegionsAsync(byte[], byte[], boolean)} instead. 1456 */ 1457 @Deprecated 1458 void mergeRegions(byte[] nameOfRegionA, byte[] nameOfRegionB, 1459 boolean forcible) throws IOException; 1460 1461 /** 1462 * Merge two regions. Asynchronous operation. 1463 * @param nameOfRegionA encoded or full name of region a 1464 * @param nameOfRegionB encoded or full name of region b 1465 * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge two 1466 * adjacent regions 1467 * @throws IOException if a remote or network exception occurs 1468 * @deprecated since 2.3.0 and will be removed in 4.0.0. Multi-region merge feature is now 1469 * supported. Use {@link #mergeRegionsAsync(byte[][], boolean)} instead. 1470 */ 1471 @Deprecated 1472 default Future<Void> mergeRegionsAsync(byte[] nameOfRegionA, byte[] nameOfRegionB, 1473 boolean forcible) throws IOException { 1474 byte[][] nameofRegionsToMerge = new byte[2][]; 1475 nameofRegionsToMerge[0] = nameOfRegionA; 1476 nameofRegionsToMerge[1] = nameOfRegionB; 1477 return mergeRegionsAsync(nameofRegionsToMerge, forcible); 1478 } 1479 1480 /** 1481 * Merge multiple regions (>=2). Asynchronous operation. 1482 * @param nameofRegionsToMerge encoded or full name of daughter regions 1483 * @param forcible <code>true</code> if do a compulsory merge, otherwise we will only merge 1484 * adjacent regions 1485 * @throws IOException if a remote or network exception occurs 1486 */ 1487 Future<Void> mergeRegionsAsync(byte[][] nameofRegionsToMerge, boolean forcible) 1488 throws IOException; 1489 1490 /** 1491 * Split a table. The method will execute split action for each region in table. 1492 * Asynchronous operation. 1493 * @param tableName table to split 1494 * @throws IOException if a remote or network exception occurs 1495 */ 1496 void split(TableName tableName) throws IOException; 1497 1498 /** 1499 * Split an individual region. Asynchronous operation. 1500 * 1501 * @param regionName region to split 1502 * @throws IOException if a remote or network exception occurs 1503 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. 1504 * Use {@link #splitRegionAsync(byte[], byte[])}. 1505 */ 1506 @Deprecated 1507 void splitRegion(byte[] regionName) throws IOException; 1508 1509 /** 1510 * Split a table. Asynchronous operation. 1511 * 1512 * @param tableName table to split 1513 * @param splitPoint the explicit position to split on 1514 * @throws IOException if a remote or network exception occurs 1515 */ 1516 void split(TableName tableName, byte[] splitPoint) throws IOException; 1517 1518 /** 1519 * Split an individual region. Asynchronous operation. 1520 * 1521 * @param regionName region to split 1522 * @param splitPoint the explicit position to split on 1523 * @throws IOException if a remote or network exception occurs 1524 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. 1525 * Use {@link #splitRegionAsync(byte[], byte[])}. 1526 */ 1527 @Deprecated 1528 void splitRegion(byte[] regionName, byte[] splitPoint) 1529 throws IOException; 1530 1531 /** 1532 * Split an individual region. Asynchronous operation. 1533 * @param regionName region to split 1534 * @throws IOException if a remote or network exception occurs 1535 */ 1536 Future<Void> splitRegionAsync(byte[] regionName) throws IOException; 1537 1538 /** 1539 * Split an individual region. Asynchronous operation. 1540 * @param regionName region to split 1541 * @param splitPoint the explicit position to split on 1542 * @throws IOException if a remote or network exception occurs 1543 */ 1544 Future<Void> splitRegionAsync(byte[] regionName, byte[] splitPoint) throws IOException; 1545 1546 /** 1547 * Modify an existing table, more IRB friendly version. 1548 * @param tableName name of table. 1549 * @param td modified description of the table 1550 * @throws IOException if a remote or network exception occurs 1551 * @deprecated since 2.0 version and will be removed in 3.0 version. use 1552 * {@link #modifyTable(TableDescriptor)} 1553 */ 1554 @Deprecated 1555 default void modifyTable(TableName tableName, TableDescriptor td) throws IOException { 1556 if (!tableName.equals(td.getTableName())) { 1557 throw new IllegalArgumentException("the specified table name '" + tableName + 1558 "' doesn't match with the HTD one: " + td.getTableName()); 1559 } 1560 modifyTable(td); 1561 } 1562 1563 /** 1564 * Modify an existing table, more IRB friendly version. 1565 * @param td modified description of the table 1566 * @throws IOException if a remote or network exception occurs 1567 */ 1568 default void modifyTable(TableDescriptor td) throws IOException { 1569 get(modifyTableAsync(td), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 1570 } 1571 1572 /** 1573 * Modify an existing table, more IRB friendly version. Asynchronous operation. This means that 1574 * it may be a while before your schema change is updated across all of the table. 1575 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 1576 * It may throw ExecutionException if there was an error while executing the operation 1577 * or TimeoutException in case the wait timeout was not long enough to allow the 1578 * operation to complete. 1579 * 1580 * @param tableName name of table. 1581 * @param td modified description of the table 1582 * @throws IOException if a remote or network exception occurs 1583 * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the 1584 * operation to complete 1585 * @deprecated since 2.0 version and will be removed in 3.0 version. 1586 * use {@link #modifyTableAsync(TableDescriptor)} 1587 */ 1588 @Deprecated 1589 default Future<Void> modifyTableAsync(TableName tableName, TableDescriptor td) 1590 throws IOException { 1591 if (!tableName.equals(td.getTableName())) { 1592 throw new IllegalArgumentException("the specified table name '" + tableName + 1593 "' doesn't match with the HTD one: " + td.getTableName()); 1594 } 1595 return modifyTableAsync(td); 1596 } 1597 1598 /** 1599 * Modify an existing table, more IRB (ruby) friendly version. Asynchronous operation. This means that 1600 * it may be a while before your schema change is updated across all of the table. 1601 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 1602 * It may throw ExecutionException if there was an error while executing the operation 1603 * or TimeoutException in case the wait timeout was not long enough to allow the 1604 * operation to complete. 1605 * 1606 * @param td description of the table 1607 * @throws IOException if a remote or network exception occurs 1608 * @return the result of the async modify. You can use Future.get(long, TimeUnit) to wait on the 1609 * operation to complete 1610 */ 1611 Future<Void> modifyTableAsync(TableDescriptor td) throws IOException; 1612 1613 /** 1614 * Shuts down the HBase cluster. 1615 * <p/> 1616 * Notice that, a success shutdown call may ends with an error since the remote server has already 1617 * been shutdown. 1618 * @throws IOException if a remote or network exception occurs 1619 */ 1620 void shutdown() throws IOException; 1621 1622 /** 1623 * Shuts down the current HBase master only. Does not shutdown the cluster. 1624 * <p/> 1625 * Notice that, a success stopMaster call may ends with an error since the remote server has 1626 * already been shutdown. 1627 * @throws IOException if a remote or network exception occurs 1628 * @see #shutdown() 1629 */ 1630 void stopMaster() throws IOException; 1631 1632 /** 1633 * Check whether Master is in maintenance mode. 1634 * 1635 * @throws IOException if a remote or network exception occurs 1636 */ 1637 boolean isMasterInMaintenanceMode() throws IOException; 1638 1639 /** 1640 * Stop the designated regionserver. 1641 * 1642 * @param hostnamePort Hostname and port delimited by a <code>:</code> as in 1643 * <code>example.org:1234</code> 1644 * @throws IOException if a remote or network exception occurs 1645 */ 1646 void stopRegionServer(String hostnamePort) throws IOException; 1647 1648 /** 1649 * Get whole cluster status, containing status about: 1650 * <pre> 1651 * hbase version 1652 * cluster id 1653 * primary/backup master(s) 1654 * master's coprocessors 1655 * live/dead regionservers 1656 * balancer 1657 * regions in transition 1658 * </pre> 1659 * @return cluster status 1660 * @throws IOException if a remote or network exception occurs 1661 * @deprecated since 2.0 version and will be removed in 3.0 version. 1662 * use {@link #getClusterMetrics()} 1663 */ 1664 @Deprecated 1665 default ClusterStatus getClusterStatus() throws IOException { 1666 return new ClusterStatus(getClusterMetrics()); 1667 } 1668 1669 /** 1670 * Get whole cluster metrics, containing status about: 1671 * <pre> 1672 * hbase version 1673 * cluster id 1674 * primary/backup master(s) 1675 * master's coprocessors 1676 * live/dead regionservers 1677 * balancer 1678 * regions in transition 1679 * </pre> 1680 * @return cluster metrics 1681 * @throws IOException if a remote or network exception occurs 1682 */ 1683 default ClusterMetrics getClusterMetrics() throws IOException { 1684 return getClusterMetrics(EnumSet.allOf(ClusterMetrics.Option.class)); 1685 } 1686 1687 /** 1688 * Get cluster status with a set of {@link Option} to get desired status. 1689 * @return cluster status 1690 * @throws IOException if a remote or network exception occurs 1691 */ 1692 ClusterMetrics getClusterMetrics(EnumSet<Option> options) throws IOException; 1693 1694 /** 1695 * @return current master server name 1696 * @throws IOException if a remote or network exception occurs 1697 */ 1698 default ServerName getMaster() throws IOException { 1699 return getClusterMetrics(EnumSet.of(Option.MASTER)).getMasterName(); 1700 } 1701 1702 /** 1703 * @return current backup master list 1704 * @throws IOException if a remote or network exception occurs 1705 */ 1706 default Collection<ServerName> getBackupMasters() throws IOException { 1707 return getClusterMetrics(EnumSet.of(Option.BACKUP_MASTERS)).getBackupMasterNames(); 1708 } 1709 1710 /** 1711 * @return current live region servers list 1712 * @throws IOException if a remote or network exception occurs 1713 */ 1714 default Collection<ServerName> getRegionServers() throws IOException { 1715 return getRegionServers(false); 1716 } 1717 1718 /** 1719 * Retrieve all current live region servers including decommissioned 1720 * if excludeDecommissionedRS is false, else non-decommissioned ones only 1721 * 1722 * @param excludeDecommissionedRS should we exclude decommissioned RS nodes 1723 * @return all current live region servers including/excluding decommissioned hosts 1724 * @throws IOException if a remote or network exception occurs 1725 */ 1726 default Collection<ServerName> getRegionServers(boolean excludeDecommissionedRS) 1727 throws IOException { 1728 List<ServerName> allServers = 1729 getClusterMetrics(EnumSet.of(Option.SERVERS_NAME)).getServersName(); 1730 if (!excludeDecommissionedRS) { 1731 return allServers; 1732 } 1733 List<ServerName> decommissionedRegionServers = listDecommissionedRegionServers(); 1734 return allServers.stream() 1735 .filter(s -> !decommissionedRegionServers.contains(s)) 1736 .collect(ImmutableList.toImmutableList()); 1737 } 1738 1739 /** 1740 * Get {@link RegionMetrics} of all regions hosted on a regionserver. 1741 * 1742 * @param serverName region server from which {@link RegionMetrics} is required. 1743 * @return a {@link RegionMetrics} list of all regions hosted on a region server 1744 * @throws IOException if a remote or network exception occurs 1745 */ 1746 default List<RegionMetrics> getRegionMetrics(ServerName serverName) throws IOException { 1747 return getRegionMetrics(serverName, null); 1748 } 1749 1750 /** 1751 * Get {@link RegionMetrics} of all regions hosted on a regionserver for a table. 1752 * 1753 * @param serverName region server from which {@link RegionMetrics} is required. 1754 * @param tableName get {@link RegionMetrics} of regions belonging to the table 1755 * @return region metrics map of all regions of a table hosted on a region server 1756 * @throws IOException if a remote or network exception occurs 1757 */ 1758 List<RegionMetrics> getRegionMetrics(ServerName serverName, 1759 TableName tableName) throws IOException; 1760 1761 /** 1762 * @return Configuration used by the instance. 1763 */ 1764 Configuration getConfiguration(); 1765 1766 /** 1767 * Create a new namespace. Blocks until namespace has been successfully created or an exception is 1768 * thrown. 1769 * @param descriptor descriptor which describes the new namespace. 1770 * @throws IOException if a remote or network exception occurs 1771 */ 1772 default void createNamespace(NamespaceDescriptor descriptor) throws IOException { 1773 get(createNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 1774 } 1775 1776 /** 1777 * Create a new namespace. 1778 * @param descriptor descriptor which describes the new namespace 1779 * @return the result of the async create namespace operation. Use Future.get(long, TimeUnit) to 1780 * wait on the operation to complete. 1781 * @throws IOException if a remote or network exception occurs 1782 */ 1783 Future<Void> createNamespaceAsync(NamespaceDescriptor descriptor) throws IOException; 1784 1785 /** 1786 * Modify an existing namespace. Blocks until namespace has been successfully modified or an 1787 * exception is thrown. 1788 * @param descriptor descriptor which describes the new namespace 1789 * @throws IOException if a remote or network exception occurs 1790 */ 1791 default void modifyNamespace(NamespaceDescriptor descriptor) throws IOException { 1792 get(modifyNamespaceAsync(descriptor), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 1793 } 1794 1795 /** 1796 * Modify an existing namespace. 1797 * @param descriptor descriptor which describes the new namespace 1798 * @return the result of the async modify namespace operation. Use Future.get(long, TimeUnit) to 1799 * wait on the operation to complete. 1800 * @throws IOException if a remote or network exception occurs 1801 */ 1802 Future<Void> modifyNamespaceAsync(NamespaceDescriptor descriptor) throws IOException; 1803 1804 /** 1805 * Delete an existing namespace. Only empty namespaces (no tables) can be removed. Blocks until 1806 * namespace has been successfully deleted or an exception is thrown. 1807 * @param name namespace name 1808 * @throws IOException if a remote or network exception occurs 1809 */ 1810 default void deleteNamespace(String name) throws IOException { 1811 get(deleteNamespaceAsync(name), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 1812 } 1813 1814 /** 1815 * Delete an existing namespace. Only empty namespaces (no tables) can be removed. 1816 * @param name namespace name 1817 * @return the result of the async delete namespace operation. Use Future.get(long, TimeUnit) to 1818 * wait on the operation to complete. 1819 * @throws IOException if a remote or network exception occurs 1820 */ 1821 Future<Void> deleteNamespaceAsync(String name) throws IOException; 1822 1823 /** 1824 * Get a namespace descriptor by name. 1825 * @param name name of namespace descriptor 1826 * @return A descriptor 1827 * @throws org.apache.hadoop.hbase.NamespaceNotFoundException 1828 * @throws IOException if a remote or network exception occurs 1829 */ 1830 NamespaceDescriptor getNamespaceDescriptor(String name) 1831 throws NamespaceNotFoundException, IOException; 1832 1833 /** 1834 * List available namespaces 1835 * 1836 * @return List of namespace names 1837 * @throws IOException if a remote or network exception occurs 1838 */ 1839 String[] listNamespaces() throws IOException; 1840 1841 /** 1842 * List available namespace descriptors 1843 * 1844 * @return List of descriptors 1845 * @throws IOException if a remote or network exception occurs 1846 */ 1847 NamespaceDescriptor[] listNamespaceDescriptors() 1848 throws IOException; 1849 1850 /** 1851 * Get list of table descriptors by namespace. 1852 * 1853 * @param name namespace name 1854 * @return HTD[] the read-only tableDescriptors 1855 * @throws IOException if a remote or network exception occurs 1856 * @deprecated since 2.0 version and will be removed in 3.0 version. 1857 * use {@link #listTableDescriptorsByNamespace(byte[])} 1858 */ 1859 @Deprecated 1860 HTableDescriptor[] listTableDescriptorsByNamespace(String name) 1861 throws IOException; 1862 1863 /** 1864 * Get list of table descriptors by namespace. 1865 * @param name namespace name 1866 * @return returns a list of TableDescriptors 1867 * @throws IOException if a remote or network exception occurs 1868 */ 1869 List<TableDescriptor> listTableDescriptorsByNamespace(byte[] name) throws IOException; 1870 1871 /** 1872 * Get list of table names by namespace. 1873 * @param name namespace name 1874 * @return The list of table names in the namespace 1875 * @throws IOException if a remote or network exception occurs 1876 */ 1877 TableName[] listTableNamesByNamespace(String name) throws IOException; 1878 1879 /** 1880 * Get the regions of a given table. 1881 * 1882 * @param tableName the name of the table 1883 * @return List of {@link HRegionInfo}. 1884 * @throws IOException if a remote or network exception occurs 1885 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 1886 * (<a href="https://issues.apache.org/jira/browse/HBASE-17980">HBASE-17980</a>). 1887 * Use {@link #getRegions(TableName)}. 1888 */ 1889 @Deprecated 1890 List<HRegionInfo> getTableRegions(TableName tableName) 1891 throws IOException; 1892 1893 /** 1894 * Get the regions of a given table. 1895 * 1896 * @param tableName the name of the table 1897 * @return List of {@link RegionInfo}. 1898 * @throws IOException if a remote or network exception occurs 1899 */ 1900 List<RegionInfo> getRegions(TableName tableName) throws IOException; 1901 1902 @Override 1903 void close() throws IOException; 1904 1905 /** 1906 * Get tableDescriptors. 1907 * 1908 * @param tableNames List of table names 1909 * @return HTD[] the read-only tableDescriptors 1910 * @throws IOException if a remote or network exception occurs 1911 * @deprecated since 2.0 version and will be removed in 3.0 version. 1912 * use {@link #listTableDescriptors(List)} 1913 */ 1914 @Deprecated 1915 HTableDescriptor[] getTableDescriptorsByTableName(List<TableName> tableNames) 1916 throws IOException; 1917 1918 /** 1919 * Get tableDescriptors. 1920 * 1921 * @param tableNames List of table names 1922 * @return returns a list of TableDescriptors 1923 * @throws IOException if a remote or network exception occurs 1924 */ 1925 List<TableDescriptor> listTableDescriptors(List<TableName> tableNames) 1926 throws IOException; 1927 1928 /** 1929 * Get tableDescriptors. 1930 * 1931 * @param names List of table names 1932 * @return HTD[] the read-only tableDescriptors 1933 * @throws IOException if a remote or network exception occurs 1934 * @deprecated since 2.0 version and will be removed in 3.0 version. 1935 * use {@link #listTableDescriptors(List)} 1936 */ 1937 @Deprecated 1938 HTableDescriptor[] getTableDescriptors(List<String> names) 1939 throws IOException; 1940 1941 /** 1942 * Abort a procedure. 1943 * <p/> 1944 * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2. 1945 * @param procId ID of the procedure to abort 1946 * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? 1947 * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does 1948 * not exist 1949 * @throws IOException if a remote or network exception occurs 1950 * @deprecated since 2.1.1 and will be removed in 4.0.0. 1951 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a> 1952 */ 1953 @Deprecated 1954 default boolean abortProcedure(long procId, boolean mayInterruptIfRunning) throws IOException { 1955 return get(abortProcedureAsync(procId, mayInterruptIfRunning), getSyncWaitTimeout(), 1956 TimeUnit.MILLISECONDS); 1957 } 1958 1959 /** 1960 * Abort a procedure but does not block and wait for completion. 1961 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. 1962 * It may throw ExecutionException if there was an error while executing the operation 1963 * or TimeoutException in case the wait timeout was not long enough to allow the 1964 * operation to complete. 1965 * Do not use. Usually it is ignored but if not, it can do more damage than good. See hbck2. 1966 * 1967 * @param procId ID of the procedure to abort 1968 * @param mayInterruptIfRunning if the proc completed at least one step, should it be aborted? 1969 * @return <code>true</code> if aborted, <code>false</code> if procedure already completed or does not exist 1970 * @throws IOException if a remote or network exception occurs 1971 * @deprecated since 2.1.1 and will be removed in 4.0.0. 1972 * @see <a href="https://issues.apache.org/jira/browse/HBASE-21223">HBASE-21223</a> 1973 */ 1974 @Deprecated 1975 Future<Boolean> abortProcedureAsync( 1976 long procId, 1977 boolean mayInterruptIfRunning) throws IOException; 1978 1979 /** 1980 * Get procedures. 1981 * @return procedure list in JSON 1982 * @throws IOException if a remote or network exception occurs 1983 */ 1984 String getProcedures() throws IOException; 1985 1986 /** 1987 * Get locks. 1988 * @return lock list in JSON 1989 * @throws IOException if a remote or network exception occurs 1990 */ 1991 String getLocks() throws IOException; 1992 1993 /** 1994 * Roll the log writer. I.e. for filesystem based write ahead logs, start writing to a new file. 1995 * 1996 * Note that the actual rolling of the log writer is asynchronous and may not be complete when 1997 * this method returns. As a side effect of this call, the named region server may schedule 1998 * store flushes at the request of the wal. 1999 * 2000 * @param serverName The servername of the regionserver. 2001 * @throws IOException if a remote or network exception occurs 2002 * @throws org.apache.hadoop.hbase.regionserver.wal.FailedLogCloseException 2003 */ 2004 void rollWALWriter(ServerName serverName) throws IOException, FailedLogCloseException; 2005 2006 /** 2007 * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames(). 2008 * @return an array of master coprocessors 2009 * @throws IOException if a remote or network exception occurs 2010 * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames() 2011 * @deprecated since 2.0 version and will be removed in 3.0 version. 2012 * use {@link #getMasterCoprocessorNames()} 2013 */ 2014 @Deprecated 2015 default String[] getMasterCoprocessors() throws IOException { 2016 return getMasterCoprocessorNames().stream().toArray(size -> new String[size]); 2017 } 2018 2019 /** 2020 * Helper that delegates to getClusterMetrics().getMasterCoprocessorNames(). 2021 * @return an array of master coprocessors 2022 * @throws IOException if a remote or network exception occurs 2023 * @see org.apache.hadoop.hbase.ClusterMetrics#getMasterCoprocessorNames() 2024 */ 2025 default List<String> getMasterCoprocessorNames() throws IOException { 2026 return getClusterMetrics(EnumSet.of(Option.MASTER_COPROCESSORS)) 2027 .getMasterCoprocessorNames(); 2028 } 2029 2030 /** 2031 * Get the current compaction state of a table. It could be in a major compaction, a minor 2032 * compaction, both, or none. 2033 * 2034 * @param tableName table to examine 2035 * @return the current compaction state 2036 * @throws IOException if a remote or network exception occurs 2037 */ 2038 CompactionState getCompactionState(TableName tableName) throws IOException; 2039 2040 /** 2041 * Get the current compaction state of a table. It could be in a compaction, or none. 2042 * 2043 * @param tableName table to examine 2044 * @param compactType {@link org.apache.hadoop.hbase.client.CompactType} 2045 * @return the current compaction state 2046 * @throws IOException if a remote or network exception occurs 2047 */ 2048 CompactionState getCompactionState(TableName tableName, 2049 CompactType compactType) throws IOException; 2050 2051 /** 2052 * Get the current compaction state of region. It could be in a major compaction, a minor 2053 * compaction, both, or none. 2054 * 2055 * @param regionName region to examine 2056 * @return the current compaction state 2057 * @throws IOException if a remote or network exception occurs 2058 */ 2059 CompactionState getCompactionStateForRegion(byte[] regionName) throws IOException; 2060 2061 /** 2062 * Get the timestamp of the last major compaction for the passed table 2063 * 2064 * The timestamp of the oldest HFile resulting from a major compaction of that table, 2065 * or 0 if no such HFile could be found. 2066 * 2067 * @param tableName table to examine 2068 * @return the last major compaction timestamp or 0 2069 * @throws IOException if a remote or network exception occurs 2070 */ 2071 long getLastMajorCompactionTimestamp(TableName tableName) throws IOException; 2072 2073 /** 2074 * Get the timestamp of the last major compaction for the passed region. 2075 * 2076 * The timestamp of the oldest HFile resulting from a major compaction of that region, 2077 * or 0 if no such HFile could be found. 2078 * 2079 * @param regionName region to examine 2080 * @return the last major compaction timestamp or 0 2081 * @throws IOException if a remote or network exception occurs 2082 */ 2083 long getLastMajorCompactionTimestampForRegion(byte[] regionName) throws IOException; 2084 2085 /** 2086 * Take a snapshot for the given table. If the table is enabled, a FLUSH-type snapshot will be 2087 * taken. If the table is disabled, an offline snapshot is taken. Snapshots are considered unique 2088 * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even 2089 * a different type or with different parameters) will fail with a 2090 * {@link org.apache.hadoop.hbase.snapshot.SnapshotCreationException} indicating the duplicate 2091 * naming. Snapshot names follow the same naming constraints as tables in HBase. See 2092 * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. 2093 * @param snapshotName name of the snapshot to be created 2094 * @param tableName name of the table for which snapshot is created 2095 * @throws IOException if a remote or network exception occurs 2096 * @throws org.apache.hadoop.hbase.snapshot.SnapshotCreationException if snapshot creation failed 2097 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2098 */ 2099 default void snapshot(String snapshotName, TableName tableName) 2100 throws IOException, SnapshotCreationException, IllegalArgumentException { 2101 snapshot(snapshotName, tableName, SnapshotType.FLUSH); 2102 } 2103 2104 /** 2105 * Create a timestamp consistent snapshot for the given table. Snapshots are considered unique 2106 * based on <b>the name of the snapshot</b>. Attempts to take a snapshot with the same name (even 2107 * different type or with different parameters) will fail with a {@link SnapshotCreationException} 2108 * indicating the duplicate naming. Snapshot names follow the same naming constraints as tables in 2109 * HBase. 2110 * @param snapshotName name of the snapshot to be created 2111 * @param tableName name of the table for which snapshot is created 2112 * @throws IOException if a remote or network exception occurs 2113 * @throws SnapshotCreationException if snapshot creation failed 2114 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2115 * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #snapshot(String, TableName)} 2116 * instead. 2117 */ 2118 @Deprecated 2119 default void snapshot(byte[] snapshotName, TableName tableName) 2120 throws IOException, SnapshotCreationException, IllegalArgumentException { 2121 snapshot(Bytes.toString(snapshotName), tableName); 2122 } 2123 2124 /** 2125 * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the 2126 * snapshot</b>. Attempts to take a snapshot with the same name (even a different type or with 2127 * different parameters) will fail with a {@link SnapshotCreationException} indicating the 2128 * duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See 2129 * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. 2130 * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other 2131 * snapshots stored on the cluster 2132 * @param tableName name of the table to snapshot 2133 * @param type type of snapshot to take 2134 * @throws IOException we fail to reach the master 2135 * @throws SnapshotCreationException if snapshot creation failed 2136 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2137 */ 2138 default void snapshot(String snapshotName, TableName tableName, SnapshotType type) 2139 throws IOException, SnapshotCreationException, IllegalArgumentException { 2140 snapshot(new SnapshotDescription(snapshotName, tableName, type)); 2141 } 2142 2143 /** 2144 * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the 2145 * snapshot</b>. Snapshots are taken sequentially even when requested concurrently, across 2146 * all tables. Attempts to take a snapshot with the same name (even a different type or with 2147 * different parameters) will fail with a {@link SnapshotCreationException} indicating the 2148 * duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See 2149 * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. 2150 * Snapshot can live with ttl seconds. 2151 * 2152 * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other 2153 * snapshots stored on the cluster 2154 * @param tableName name of the table to snapshot 2155 * @param type type of snapshot to take 2156 * @param snapshotProps snapshot additional properties e.g. TTL 2157 * @throws IOException we fail to reach the master 2158 * @throws SnapshotCreationException if snapshot creation failed 2159 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2160 */ 2161 default void snapshot(String snapshotName, TableName tableName, SnapshotType type, 2162 Map<String, Object> snapshotProps) throws IOException, 2163 SnapshotCreationException, IllegalArgumentException { 2164 snapshot(new SnapshotDescription(snapshotName, tableName, type, snapshotProps)); 2165 } 2166 2167 /** 2168 * Create typed snapshot of the table. Snapshots are considered unique based on <b>the name of the 2169 * snapshot</b>. Snapshots are taken sequentially even when requested concurrently, across 2170 * all tables. Attempts to take a snapshot with the same name (even a different type or with 2171 * different parameters) will fail with a {@link SnapshotCreationException} indicating the 2172 * duplicate naming. Snapshot names follow the same naming constraints as tables in HBase. See 2173 * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. 2174 * Snapshot can live with ttl seconds. 2175 * 2176 * @param snapshotName name to give the snapshot on the filesystem. Must be unique from all other 2177 * snapshots stored on the cluster 2178 * @param tableName name of the table to snapshot 2179 * @param snapshotProps snapshot additional properties e.g. TTL 2180 * @throws IOException we fail to reach the master 2181 * @throws SnapshotCreationException if snapshot creation failed 2182 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2183 */ 2184 default void snapshot(String snapshotName, TableName tableName, 2185 Map<String, Object> snapshotProps) throws IOException, 2186 SnapshotCreationException, IllegalArgumentException { 2187 snapshot(new SnapshotDescription(snapshotName, tableName, SnapshotType.FLUSH, snapshotProps)); 2188 } 2189 2190 /** 2191 * Take a snapshot and wait for the server to complete that snapshot (blocking). Snapshots are 2192 * considered unique based on <b>the name of the snapshot</b>. Snapshots are taken sequentially 2193 * even when requested concurrently, across all tables. Attempts to take a snapshot with the same 2194 * name (even a different type or with different parameters) will fail with a 2195 * {@link SnapshotCreationException} indicating the duplicate naming. Snapshot names follow the 2196 * same naming constraints as tables in HBase. See 2197 * {@link org.apache.hadoop.hbase.TableName#isLegalFullyQualifiedTableName(byte[])}. You should 2198 * probably use {@link #snapshot(String, org.apache.hadoop.hbase.TableName)} or 2199 * {@link #snapshot(byte[], org.apache.hadoop.hbase.TableName)} unless you are sure about the type 2200 * of snapshot that you want to take. 2201 * @param snapshot snapshot to take 2202 * @throws IOException or we lose contact with the master. 2203 * @throws SnapshotCreationException if snapshot failed to be taken 2204 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2205 */ 2206 void snapshot(SnapshotDescription snapshot) 2207 throws IOException, SnapshotCreationException, IllegalArgumentException; 2208 2209 /** 2210 * Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a 2211 * single snapshot should be taken at a time, or results may be undefined. 2212 * 2213 * @param snapshot snapshot to take 2214 * @throws IOException if the snapshot did not succeed or we lose contact with the master. 2215 * @throws SnapshotCreationException if snapshot creation failed 2216 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2217 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 2218 * {@link #snapshotAsync(SnapshotDescription)} instead. 2219 */ 2220 @Deprecated 2221 @SuppressWarnings("FutureReturnValueIgnored") 2222 default void takeSnapshotAsync(SnapshotDescription snapshot) 2223 throws IOException, SnapshotCreationException { 2224 snapshotAsync(snapshot); 2225 } 2226 2227 /** 2228 * Take a snapshot without waiting for the server to complete that snapshot (asynchronous) Only a 2229 * single snapshot should be taken at a time, or results may be undefined. 2230 * @param snapshot snapshot to take 2231 * @throws IOException if the snapshot did not succeed or we lose contact with the master. 2232 * @throws SnapshotCreationException if snapshot creation failed 2233 * @throws IllegalArgumentException if the snapshot request is formatted incorrectly 2234 */ 2235 Future<Void> snapshotAsync(SnapshotDescription snapshot) 2236 throws IOException, SnapshotCreationException; 2237 2238 /** 2239 * Check the current state of the passed snapshot. There are three possible states: <ol> 2240 * <li>running - returns <tt>false</tt></li> <li>finished - returns <tt>true</tt></li> 2241 * <li>finished with error - throws the exception that caused the snapshot to fail</li> </ol> The 2242 * cluster only knows about the most recent snapshot. Therefore, if another snapshot has been 2243 * run/started since the snapshot you are checking, you will receive an {@link 2244 * org.apache.hadoop.hbase.snapshot.UnknownSnapshotException}. 2245 * 2246 * @param snapshot description of the snapshot to check 2247 * @return <tt>true</tt> if the snapshot is completed, <tt>false</tt> if the snapshot is still 2248 * running 2249 * @throws IOException if we have a network issue 2250 * @throws org.apache.hadoop.hbase.snapshot.HBaseSnapshotException if the snapshot failed 2251 * @throws org.apache.hadoop.hbase.snapshot.UnknownSnapshotException if the requested snapshot is 2252 * unknown 2253 */ 2254 boolean isSnapshotFinished(SnapshotDescription snapshot) 2255 throws IOException, HBaseSnapshotException, UnknownSnapshotException; 2256 2257 /** 2258 * Restore the specified snapshot on the original table. (The table must be disabled) If the 2259 * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to 2260 * <code>true</code>, a snapshot of the current table is taken before executing the restore 2261 * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore 2262 * completes without problem the failsafe snapshot is deleted. 2263 * @param snapshotName name of the snapshot to restore 2264 * @throws IOException if a remote or network exception occurs 2265 * @throws org.apache.hadoop.hbase.snapshot.RestoreSnapshotException if snapshot failed to be 2266 * restored 2267 * @throws IllegalArgumentException if the restore request is formatted incorrectly 2268 * @deprecated since 2.3.0, will be removed in 3.0.0. Use {@link #restoreSnapshot(String)} 2269 * instead. 2270 */ 2271 @Deprecated 2272 default void restoreSnapshot(byte[] snapshotName) throws IOException, RestoreSnapshotException { 2273 restoreSnapshot(Bytes.toString(snapshotName)); 2274 } 2275 2276 /** 2277 * Restore the specified snapshot on the original table. (The table must be disabled) If the 2278 * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to 2279 * <code>true</code>, a snapshot of the current table is taken before executing the restore 2280 * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore 2281 * completes without problem the failsafe snapshot is deleted. 2282 * @param snapshotName name of the snapshot to restore 2283 * @throws IOException if a remote or network exception occurs 2284 * @throws RestoreSnapshotException if snapshot failed to be restored 2285 * @throws IllegalArgumentException if the restore request is formatted incorrectly 2286 */ 2287 void restoreSnapshot(String snapshotName) throws IOException, RestoreSnapshotException; 2288 2289 /** 2290 * Restore the specified snapshot on the original table. (The table must be disabled) If the 2291 * "hbase.snapshot.restore.take.failsafe.snapshot" configuration property is set to 2292 * <code>true</code>, a snapshot of the current table is taken before executing the restore 2293 * operation. In case of restore failure, the failsafe snapshot will be restored. If the restore 2294 * completes without problem the failsafe snapshot is deleted. 2295 * @param snapshotName name of the snapshot to restore 2296 * @throws IOException if a remote or network exception occurs 2297 * @throws RestoreSnapshotException if snapshot failed to be restored 2298 * @return the result of the async restore snapshot. You can use Future.get(long, TimeUnit) to 2299 * wait on the operation to complete. 2300 * @deprecated since 2.3.0, will be removed in 3.0.0. The implementation does not take care of the 2301 * failsafe property, so do not use it any more. 2302 */ 2303 @Deprecated 2304 Future<Void> restoreSnapshotAsync(String snapshotName) 2305 throws IOException, RestoreSnapshotException; 2306 2307 /** 2308 * Restore the specified snapshot on the original table. (The table must be disabled) If 2309 * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken 2310 * before executing the restore operation. In case of restore failure, the failsafe snapshot will 2311 * be restored. If the restore completes without problem the failsafe snapshot is deleted. The 2312 * failsafe snapshot name is configurable by using the property 2313 * "hbase.snapshot.restore.failsafe.name". 2314 * @param snapshotName name of the snapshot to restore 2315 * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken 2316 * @throws IOException if a remote or network exception occurs 2317 * @throws RestoreSnapshotException if snapshot failed to be restored 2318 * @throws IllegalArgumentException if the restore request is formatted incorrectly 2319 * @deprecated since 2.3.0, will be removed in 3.0.0. Use 2320 * {@link #restoreSnapshot(String, boolean)} instead. 2321 */ 2322 @Deprecated 2323 default void restoreSnapshot(byte[] snapshotName, boolean takeFailSafeSnapshot) 2324 throws IOException, RestoreSnapshotException { 2325 restoreSnapshot(Bytes.toString(snapshotName), takeFailSafeSnapshot); 2326 } 2327 2328 /** 2329 * Restore the specified snapshot on the original table. (The table must be disabled) If 2330 * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken 2331 * before executing the restore operation. In case of restore failure, the failsafe snapshot will 2332 * be restored. If the restore completes without problem the failsafe snapshot is deleted. The 2333 * failsafe snapshot name is configurable by using the property 2334 * "hbase.snapshot.restore.failsafe.name". 2335 * @param snapshotName name of the snapshot to restore 2336 * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken 2337 * @throws IOException if a remote or network exception occurs 2338 * @throws RestoreSnapshotException if snapshot failed to be restored 2339 * @throws IllegalArgumentException if the restore request is formatted incorrectly 2340 */ 2341 default void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot) 2342 throws IOException, RestoreSnapshotException { 2343 restoreSnapshot(snapshotName, takeFailSafeSnapshot, false); 2344 } 2345 2346 /** 2347 * Restore the specified snapshot on the original table. (The table must be disabled) If 2348 * 'takeFailSafeSnapshot' is set to <code>true</code>, a snapshot of the current table is taken 2349 * before executing the restore operation. In case of restore failure, the failsafe snapshot will 2350 * be restored. If the restore completes without problem the failsafe snapshot is deleted. The 2351 * failsafe snapshot name is configurable by using the property 2352 * "hbase.snapshot.restore.failsafe.name". 2353 * @param snapshotName name of the snapshot to restore 2354 * @param takeFailSafeSnapshot <code>true</code> if the failsafe snapshot should be taken 2355 * @param restoreAcl <code>true</code> to restore acl of snapshot 2356 * @throws IOException if a remote or network exception occurs 2357 * @throws RestoreSnapshotException if snapshot failed to be restored 2358 * @throws IllegalArgumentException if the restore request is formatted incorrectly 2359 */ 2360 void restoreSnapshot(String snapshotName, boolean takeFailSafeSnapshot, 2361 boolean restoreAcl) throws IOException, RestoreSnapshotException; 2362 2363 /** 2364 * Create a new table by cloning the snapshot content. 2365 * @param snapshotName name of the snapshot to be cloned 2366 * @param tableName name of the table where the snapshot will be restored 2367 * @throws IOException if a remote or network exception occurs 2368 * @throws TableExistsException if table to be created already exists 2369 * @throws RestoreSnapshotException if snapshot failed to be cloned 2370 * @throws IllegalArgumentException if the specified table has not a valid name 2371 * @deprecated since 2.3.0, will be removed in 3.0.0. Use 2372 * {@link #cloneSnapshot(String, TableName)} instead. 2373 */ 2374 @Deprecated 2375 default void cloneSnapshot(byte[] snapshotName, TableName tableName) 2376 throws IOException, TableExistsException, RestoreSnapshotException { 2377 cloneSnapshot(Bytes.toString(snapshotName), tableName); 2378 } 2379 2380 /** 2381 * Create a new table by cloning the snapshot content. 2382 * @param snapshotName name of the snapshot to be cloned 2383 * @param tableName name of the table where the snapshot will be restored 2384 * @throws IOException if a remote or network exception occurs 2385 * @throws TableExistsException if table to be created already exists 2386 * @throws RestoreSnapshotException if snapshot failed to be cloned 2387 * @throws IllegalArgumentException if the specified table has not a valid name 2388 */ 2389 default void cloneSnapshot(String snapshotName, TableName tableName) 2390 throws IOException, TableExistsException, RestoreSnapshotException { 2391 cloneSnapshot(snapshotName, tableName, false); 2392 } 2393 2394 /** 2395 * Create a new table by cloning the snapshot content. 2396 * @param snapshotName name of the snapshot to be cloned 2397 * @param tableName name of the table where the snapshot will be restored 2398 * @param restoreAcl <code>true</code> to clone acl into newly created table 2399 * @throws IOException if a remote or network exception occurs 2400 * @throws TableExistsException if table to be created already exists 2401 * @throws RestoreSnapshotException if snapshot failed to be cloned 2402 * @throws IllegalArgumentException if the specified table has not a valid name 2403 */ 2404 default void cloneSnapshot(String snapshotName, TableName tableName, boolean restoreAcl) 2405 throws IOException, TableExistsException, RestoreSnapshotException { 2406 get(cloneSnapshotAsync(snapshotName, tableName, restoreAcl), getSyncWaitTimeout(), 2407 TimeUnit.MILLISECONDS); 2408 } 2409 2410 /** 2411 * Create a new table by cloning the snapshot content, but does not block and wait for it to be 2412 * completely cloned. You can use Future.get(long, TimeUnit) to wait on the operation to complete. 2413 * It may throw ExecutionException if there was an error while executing the operation or 2414 * TimeoutException in case the wait timeout was not long enough to allow the operation to 2415 * complete. 2416 * @param snapshotName name of the snapshot to be cloned 2417 * @param tableName name of the table where the snapshot will be restored 2418 * @throws IOException if a remote or network exception occurs 2419 * @throws TableExistsException if table to be cloned already exists 2420 * @return the result of the async clone snapshot. You can use Future.get(long, TimeUnit) to wait 2421 * on the operation to complete. 2422 */ 2423 default Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName) 2424 throws IOException, TableExistsException { 2425 return cloneSnapshotAsync(snapshotName, tableName, false); 2426 } 2427 2428 /** 2429 * Create a new table by cloning the snapshot content. 2430 * @param snapshotName name of the snapshot to be cloned 2431 * @param tableName name of the table where the snapshot will be restored 2432 * @param restoreAcl <code>true</code> to clone acl into newly created table 2433 * @throws IOException if a remote or network exception occurs 2434 * @throws TableExistsException if table to be created already exists 2435 * @throws RestoreSnapshotException if snapshot failed to be cloned 2436 * @throws IllegalArgumentException if the specified table has not a valid name 2437 */ 2438 Future<Void> cloneSnapshotAsync(String snapshotName, TableName tableName, boolean restoreAcl) 2439 throws IOException, TableExistsException, RestoreSnapshotException; 2440 2441 /** 2442 * Execute a distributed procedure on a cluster. 2443 * 2444 * @param signature A distributed procedure is uniquely identified by its signature (default the 2445 * root ZK node name of the procedure). 2446 * @param instance The instance name of the procedure. For some procedures, this parameter is 2447 * optional. 2448 * @param props Property/Value pairs of properties passing to the procedure 2449 * @throws IOException if a remote or network exception occurs 2450 */ 2451 void execProcedure(String signature, String instance, Map<String, String> props) 2452 throws IOException; 2453 2454 /** 2455 * Execute a distributed procedure on a cluster. 2456 * 2457 * @param signature A distributed procedure is uniquely identified by its signature (default the 2458 * root ZK node name of the procedure). 2459 * @param instance The instance name of the procedure. For some procedures, this parameter is 2460 * optional. 2461 * @param props Property/Value pairs of properties passing to the procedure 2462 * @return data returned after procedure execution. null if no return data. 2463 * @throws IOException if a remote or network exception occurs 2464 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 2465 * {@link #execProcedureWithReturn(String, String, Map)} } instead. 2466 */ 2467 @Deprecated 2468 default byte[] execProcedureWithRet(String signature, String instance, Map<String, String> props) 2469 throws IOException { 2470 return execProcedureWithReturn(signature, instance, props); 2471 } 2472 2473 /** 2474 * Execute a distributed procedure on a cluster. 2475 * @param signature A distributed procedure is uniquely identified by its signature (default the 2476 * root ZK node name of the procedure). 2477 * @param instance The instance name of the procedure. For some procedures, this parameter is 2478 * optional. 2479 * @param props Property/Value pairs of properties passing to the procedure 2480 * @return data returned after procedure execution. null if no return data. 2481 * @throws IOException if a remote or network exception occurs 2482 */ 2483 byte[] execProcedureWithReturn(String signature, String instance, Map<String, String> props) 2484 throws IOException; 2485 2486 /** 2487 * Check the current state of the specified procedure. There are three possible states: <ol> 2488 * <li>running - returns <tt>false</tt></li> <li>finished - returns <tt>true</tt></li> 2489 * <li>finished with error - throws the exception that caused the procedure to fail</li> </ol> 2490 * 2491 * @param signature The signature that uniquely identifies a procedure 2492 * @param instance The instance name of the procedure 2493 * @param props Property/Value pairs of properties passing to the procedure 2494 * @return <code>true</code> if the specified procedure is finished successfully, <code>false</code> if it is still running 2495 * @throws IOException if the specified procedure finished with error 2496 */ 2497 boolean isProcedureFinished(String signature, String instance, Map<String, String> props) 2498 throws IOException; 2499 2500 /** 2501 * List completed snapshots. 2502 * 2503 * @return a list of snapshot descriptors for completed snapshots 2504 * @throws IOException if a network error occurs 2505 */ 2506 List<SnapshotDescription> listSnapshots() throws IOException; 2507 2508 /** 2509 * List all the completed snapshots matching the given regular expression. 2510 * 2511 * @param regex The regular expression to match against 2512 * @return list of SnapshotDescription 2513 * @throws IOException if a remote or network exception occurs 2514 * @deprecated since 2.0 version and will be removed in 3.0 version. 2515 * Use {@link #listSnapshots(Pattern)} instead. 2516 */ 2517 @Deprecated 2518 List<SnapshotDescription> listSnapshots(String regex) throws IOException; 2519 2520 /** 2521 * List all the completed snapshots matching the given pattern. 2522 * 2523 * @param pattern The compiled regular expression to match against 2524 * @return list of SnapshotDescription 2525 * @throws IOException if a remote or network exception occurs 2526 */ 2527 List<SnapshotDescription> listSnapshots(Pattern pattern) throws IOException; 2528 2529 /** 2530 * List all the completed snapshots matching the given table name regular expression and snapshot 2531 * name regular expression. 2532 * @param tableNameRegex The table name regular expression to match against 2533 * @param snapshotNameRegex The snapshot name regular expression to match against 2534 * @return list of completed SnapshotDescription 2535 * @throws IOException if a remote or network exception occurs 2536 * @deprecated since 2.0 version and will be removed in 3.0 version. 2537 * Use {@link #listTableSnapshots(Pattern, Pattern)} instead. 2538 */ 2539 @Deprecated 2540 List<SnapshotDescription> listTableSnapshots(String tableNameRegex, 2541 String snapshotNameRegex) throws IOException; 2542 2543 /** 2544 * List all the completed snapshots matching the given table name regular expression and snapshot 2545 * name regular expression. 2546 * @param tableNamePattern The compiled table name regular expression to match against 2547 * @param snapshotNamePattern The compiled snapshot name regular expression to match against 2548 * @return list of completed SnapshotDescription 2549 * @throws IOException if a remote or network exception occurs 2550 */ 2551 List<SnapshotDescription> listTableSnapshots(Pattern tableNamePattern, 2552 Pattern snapshotNamePattern) throws IOException; 2553 2554 /** 2555 * Delete an existing snapshot. 2556 * 2557 * @param snapshotName name of the snapshot 2558 * @throws IOException if a remote or network exception occurs 2559 * @deprecated Since 2.2.0. Will be removed in 3.0.0. Use {@link #deleteSnapshot(String)} instead. 2560 */ 2561 @Deprecated 2562 void deleteSnapshot(byte[] snapshotName) throws IOException; 2563 2564 /** 2565 * Delete an existing snapshot. 2566 * 2567 * @param snapshotName name of the snapshot 2568 * @throws IOException if a remote or network exception occurs 2569 */ 2570 void deleteSnapshot(String snapshotName) throws IOException; 2571 2572 /** 2573 * Delete existing snapshots whose names match the pattern passed. 2574 * 2575 * @param regex The regular expression to match against 2576 * @throws IOException if a remote or network exception occurs 2577 * @deprecated since 2.0 version and will be removed in 3.0 version. 2578 * Use {@link #deleteSnapshots(Pattern)} instead. 2579 */ 2580 @Deprecated 2581 void deleteSnapshots(String regex) throws IOException; 2582 2583 /** 2584 * Delete existing snapshots whose names match the pattern passed. 2585 * 2586 * @param pattern pattern for names of the snapshot to match 2587 * @throws IOException if a remote or network exception occurs 2588 */ 2589 void deleteSnapshots(Pattern pattern) throws IOException; 2590 2591 /** 2592 * Delete all existing snapshots matching the given table name regular expression and snapshot 2593 * name regular expression. 2594 * @param tableNameRegex The table name regular expression to match against 2595 * @param snapshotNameRegex The snapshot name regular expression to match against 2596 * @throws IOException if a remote or network exception occurs 2597 * @deprecated since 2.0 version and will be removed in 3.0 version. 2598 * Use {@link #deleteTableSnapshots(Pattern, Pattern)} instead. 2599 */ 2600 @Deprecated 2601 void deleteTableSnapshots(String tableNameRegex, String snapshotNameRegex) throws IOException; 2602 2603 /** 2604 * Delete all existing snapshots matching the given table name regular expression and snapshot 2605 * name regular expression. 2606 * @param tableNamePattern The compiled table name regular expression to match against 2607 * @param snapshotNamePattern The compiled snapshot name regular expression to match against 2608 * @throws IOException if a remote or network exception occurs 2609 */ 2610 void deleteTableSnapshots(Pattern tableNamePattern, Pattern snapshotNamePattern) 2611 throws IOException; 2612 2613 /** 2614 * Apply the new quota settings. 2615 * 2616 * @param quota the quota settings 2617 * @throws IOException if a remote or network exception occurs 2618 */ 2619 void setQuota(QuotaSettings quota) throws IOException; 2620 2621 /** 2622 * Return a QuotaRetriever to list the quotas based on the filter. 2623 * @param filter the quota settings filter 2624 * @return the quota retriever 2625 * @throws IOException if a remote or network exception occurs 2626 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #getQuota(QuotaFilter)}. 2627 */ 2628 @Deprecated 2629 QuotaRetriever getQuotaRetriever(QuotaFilter filter) throws IOException; 2630 2631 /** 2632 * List the quotas based on the filter. 2633 * @param filter the quota settings filter 2634 * @return the QuotaSetting list 2635 * @throws IOException if a remote or network exception occurs 2636 */ 2637 List<QuotaSettings> getQuota(QuotaFilter filter) throws IOException; 2638 2639 /** 2640 * Creates and returns a {@link com.google.protobuf.RpcChannel} instance connected to the active 2641 * master. <p> The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access 2642 * a published coprocessor {@link com.google.protobuf.Service} using standard protobuf service 2643 * invocations: </p> <div style="background-color: #cccccc; padding: 2px"> 2644 * <blockquote><pre> 2645 * CoprocessorRpcChannel channel = myAdmin.coprocessorService(); 2646 * MyService.BlockingInterface service = MyService.newBlockingStub(channel); 2647 * MyCallRequest request = MyCallRequest.newBuilder() 2648 * ... 2649 * .build(); 2650 * MyCallResponse response = service.myCall(null, request); 2651 * </pre></blockquote></div> 2652 * 2653 * @return A MasterCoprocessorRpcChannel instance 2654 */ 2655 CoprocessorRpcChannel coprocessorService(); 2656 2657 2658 /** 2659 * Creates and returns a {@link com.google.protobuf.RpcChannel} instance 2660 * connected to the passed region server. 2661 * 2662 * <p> 2663 * The obtained {@link com.google.protobuf.RpcChannel} instance can be used to access a published 2664 * coprocessor {@link com.google.protobuf.Service} using standard protobuf service invocations: 2665 * </p> 2666 * 2667 * <div style="background-color: #cccccc; padding: 2px"> 2668 * <blockquote><pre> 2669 * CoprocessorRpcChannel channel = myAdmin.coprocessorService(serverName); 2670 * MyService.BlockingInterface service = MyService.newBlockingStub(channel); 2671 * MyCallRequest request = MyCallRequest.newBuilder() 2672 * ... 2673 * .build(); 2674 * MyCallResponse response = service.myCall(null, request); 2675 * </pre></blockquote></div> 2676 * 2677 * @param serverName the server name to which the endpoint call is made 2678 * @return A RegionServerCoprocessorRpcChannel instance 2679 */ 2680 CoprocessorRpcChannel coprocessorService(ServerName serverName); 2681 2682 2683 /** 2684 * Update the configuration and trigger an online config change 2685 * on the regionserver. 2686 * @param server : The server whose config needs to be updated. 2687 * @throws IOException if a remote or network exception occurs 2688 */ 2689 void updateConfiguration(ServerName server) throws IOException; 2690 2691 2692 /** 2693 * Update the configuration and trigger an online config change 2694 * on all the regionservers. 2695 * @throws IOException if a remote or network exception occurs 2696 */ 2697 void updateConfiguration() throws IOException; 2698 2699 /** 2700 * Get the info port of the current master if one is available. 2701 * @return master info port 2702 * @throws IOException if a remote or network exception occurs 2703 */ 2704 default int getMasterInfoPort() throws IOException { 2705 return getClusterMetrics(EnumSet.of(Option.MASTER_INFO_PORT)).getMasterInfoPort(); 2706 } 2707 2708 /** 2709 * Return the set of supported security capabilities. 2710 * @throws IOException if a remote or network exception occurs 2711 * @throws UnsupportedOperationException 2712 */ 2713 List<SecurityCapability> getSecurityCapabilities() throws IOException; 2714 2715 /** 2716 * Turn the Split or Merge switches on or off. 2717 * @param enabled enabled or not 2718 * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding, 2719 * to return. 2720 * @param switchTypes switchType list {@link MasterSwitchType} 2721 * @return Previous switch value array 2722 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use {@link #splitSwitch(boolean, boolean)} 2723 * or {@link #mergeSwitch(boolean, boolean)} instead. 2724 * @throws IOException if a remote or network exception occurs 2725 */ 2726 @Deprecated 2727 default boolean[] setSplitOrMergeEnabled(boolean enabled, boolean synchronous, 2728 MasterSwitchType... switchTypes) throws IOException { 2729 boolean[] preValues = new boolean[switchTypes.length]; 2730 for (int i = 0; i < switchTypes.length; i++) { 2731 switch (switchTypes[i]) { 2732 case SPLIT: 2733 preValues[i] = splitSwitch(enabled, synchronous); 2734 break; 2735 case MERGE: 2736 preValues[i] = mergeSwitch(enabled, synchronous); 2737 break; 2738 default: 2739 throw new UnsupportedOperationException("Unsupported switch type:" + switchTypes[i]); 2740 } 2741 } 2742 return preValues; 2743 } 2744 2745 /** 2746 * Turn the split switch on or off. 2747 * @param enabled enabled or not 2748 * @param synchronous If <code>true</code>, it waits until current split() call, if outstanding, 2749 * to return. 2750 * @return Previous switch value 2751 * @throws IOException if a remote or network exception occurs 2752 */ 2753 boolean splitSwitch(boolean enabled, boolean synchronous) throws IOException; 2754 2755 /** 2756 * Turn the merge switch on or off. 2757 * @param enabled enabled or not 2758 * @param synchronous If <code>true</code>, it waits until current merge() call, if outstanding, 2759 * to return. 2760 * @return Previous switch value 2761 * @throws IOException if a remote or network exception occurs 2762 */ 2763 boolean mergeSwitch(boolean enabled, boolean synchronous) throws IOException; 2764 2765 /** 2766 * Query the current state of the switch. 2767 * 2768 * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise. 2769 * @throws IOException if a remote or network exception occurs 2770 * @deprecated Since 2.0.0. Will be removed in 3.0.0. Use 2771 * {@link #isSplitEnabled()} or {@link #isMergeEnabled()} instead. 2772 */ 2773 @Deprecated 2774 default boolean isSplitOrMergeEnabled(MasterSwitchType switchType) throws IOException { 2775 switch (switchType) { 2776 case SPLIT: 2777 return isSplitEnabled(); 2778 case MERGE: 2779 return isMergeEnabled(); 2780 default: 2781 break; 2782 } 2783 throw new UnsupportedOperationException("Unsupported switch type:" + switchType); 2784 } 2785 2786 /** 2787 * Query the current state of the split switch. 2788 * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise. 2789 * @throws IOException if a remote or network exception occurs 2790 */ 2791 boolean isSplitEnabled() throws IOException; 2792 2793 /** 2794 * Query the current state of the merge switch. 2795 * @return <code>true</code> if the switch is enabled, <code>false</code> otherwise. 2796 * @throws IOException if a remote or network exception occurs 2797 */ 2798 boolean isMergeEnabled() throws IOException; 2799 2800 /** 2801 * Add a new replication peer for replicating data to slave cluster. 2802 * @param peerId a short name that identifies the peer 2803 * @param peerConfig configuration for the replication peer 2804 * @throws IOException if a remote or network exception occurs 2805 */ 2806 default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig) 2807 throws IOException { 2808 addReplicationPeer(peerId, peerConfig, true); 2809 } 2810 2811 /** 2812 * Add a new replication peer for replicating data to slave cluster. 2813 * @param peerId a short name that identifies the peer 2814 * @param peerConfig configuration for the replication peer 2815 * @param enabled peer state, true if ENABLED and false if DISABLED 2816 * @throws IOException if a remote or network exception occurs 2817 */ 2818 default void addReplicationPeer(String peerId, ReplicationPeerConfig peerConfig, boolean enabled) 2819 throws IOException { 2820 get(addReplicationPeerAsync(peerId, peerConfig, enabled), getSyncWaitTimeout(), 2821 TimeUnit.MILLISECONDS); 2822 } 2823 2824 /** 2825 * Add a new replication peer but does not block and wait for it. 2826 * <p/> 2827 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2828 * ExecutionException if there was an error while executing the operation or TimeoutException in 2829 * case the wait timeout was not long enough to allow the operation to complete. 2830 * @param peerId a short name that identifies the peer 2831 * @param peerConfig configuration for the replication peer 2832 * @return the result of the async operation 2833 * @throws IOException IOException if a remote or network exception occurs 2834 */ 2835 default Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig) 2836 throws IOException { 2837 return addReplicationPeerAsync(peerId, peerConfig, true); 2838 } 2839 2840 /** 2841 * Add a new replication peer but does not block and wait for it. 2842 * <p> 2843 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2844 * ExecutionException if there was an error while executing the operation or TimeoutException in 2845 * case the wait timeout was not long enough to allow the operation to complete. 2846 * @param peerId a short name that identifies the peer 2847 * @param peerConfig configuration for the replication peer 2848 * @param enabled peer state, true if ENABLED and false if DISABLED 2849 * @return the result of the async operation 2850 * @throws IOException IOException if a remote or network exception occurs 2851 */ 2852 Future<Void> addReplicationPeerAsync(String peerId, ReplicationPeerConfig peerConfig, 2853 boolean enabled) throws IOException; 2854 2855 /** 2856 * Remove a peer and stop the replication. 2857 * @param peerId a short name that identifies the peer 2858 * @throws IOException if a remote or network exception occurs 2859 */ 2860 default void removeReplicationPeer(String peerId) throws IOException { 2861 get(removeReplicationPeerAsync(peerId), getSyncWaitTimeout(), 2862 TimeUnit.MILLISECONDS); 2863 } 2864 2865 /** 2866 * Remove a replication peer but does not block and wait for it. 2867 * <p> 2868 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2869 * ExecutionException if there was an error while executing the operation or TimeoutException in 2870 * case the wait timeout was not long enough to allow the operation to complete. 2871 * @param peerId a short name that identifies the peer 2872 * @return the result of the async operation 2873 * @throws IOException IOException if a remote or network exception occurs 2874 */ 2875 Future<Void> removeReplicationPeerAsync(String peerId) throws IOException; 2876 2877 /** 2878 * Restart the replication stream to the specified peer. 2879 * @param peerId a short name that identifies the peer 2880 * @throws IOException if a remote or network exception occurs 2881 */ 2882 default void enableReplicationPeer(String peerId) throws IOException { 2883 get(enableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 2884 } 2885 2886 /** 2887 * Enable a replication peer but does not block and wait for it. 2888 * <p> 2889 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2890 * ExecutionException if there was an error while executing the operation or TimeoutException in 2891 * case the wait timeout was not long enough to allow the operation to complete. 2892 * @param peerId a short name that identifies the peer 2893 * @return the result of the async operation 2894 * @throws IOException IOException if a remote or network exception occurs 2895 */ 2896 Future<Void> enableReplicationPeerAsync(String peerId) throws IOException; 2897 2898 /** 2899 * Stop the replication stream to the specified peer. 2900 * @param peerId a short name that identifies the peer 2901 * @throws IOException if a remote or network exception occurs 2902 */ 2903 default void disableReplicationPeer(String peerId) throws IOException { 2904 get(disableReplicationPeerAsync(peerId), getSyncWaitTimeout(), TimeUnit.MILLISECONDS); 2905 } 2906 2907 /** 2908 * Disable a replication peer but does not block and wait for it. 2909 * <p/> 2910 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2911 * ExecutionException if there was an error while executing the operation or TimeoutException in 2912 * case the wait timeout was not long enough to allow the operation to complete. 2913 * @param peerId a short name that identifies the peer 2914 * @return the result of the async operation 2915 * @throws IOException IOException if a remote or network exception occurs 2916 */ 2917 Future<Void> disableReplicationPeerAsync(String peerId) throws IOException; 2918 2919 /** 2920 * Returns the configured ReplicationPeerConfig for the specified peer. 2921 * @param peerId a short name that identifies the peer 2922 * @return ReplicationPeerConfig for the peer 2923 * @throws IOException if a remote or network exception occurs 2924 */ 2925 ReplicationPeerConfig getReplicationPeerConfig(String peerId) throws IOException; 2926 2927 /** 2928 * Update the peerConfig for the specified peer. 2929 * @param peerId a short name that identifies the peer 2930 * @param peerConfig new config for the replication peer 2931 * @throws IOException if a remote or network exception occurs 2932 */ 2933 default void updateReplicationPeerConfig(String peerId, ReplicationPeerConfig peerConfig) 2934 throws IOException { 2935 get(updateReplicationPeerConfigAsync(peerId, peerConfig), getSyncWaitTimeout(), 2936 TimeUnit.MILLISECONDS); 2937 } 2938 2939 /** 2940 * Update the peerConfig for the specified peer but does not block and wait for it. 2941 * <p/> 2942 * You can use Future.get(long, TimeUnit) to wait on the operation to complete. It may throw 2943 * ExecutionException if there was an error while executing the operation or TimeoutException in 2944 * case the wait timeout was not long enough to allow the operation to complete. 2945 * @param peerId a short name that identifies the peer 2946 * @param peerConfig new config for the replication peer 2947 * @return the result of the async operation 2948 * @throws IOException IOException if a remote or network exception occurs 2949 */ 2950 Future<Void> updateReplicationPeerConfigAsync(String peerId, ReplicationPeerConfig peerConfig) 2951 throws IOException; 2952 2953 /** 2954 * Append the replicable table column family config from the specified peer. 2955 * @param id a short that identifies the cluster 2956 * @param tableCfs A map from tableName to column family names 2957 * @throws ReplicationException if tableCfs has conflict with existing config 2958 * @throws IOException if a remote or network exception occurs 2959 */ 2960 default void appendReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) 2961 throws ReplicationException, IOException { 2962 if (tableCfs == null) { 2963 throw new ReplicationException("tableCfs is null"); 2964 } 2965 ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id); 2966 ReplicationPeerConfig newPeerConfig = 2967 ReplicationPeerConfigUtil.appendTableCFsToReplicationPeerConfig(tableCfs, peerConfig); 2968 updateReplicationPeerConfig(id, newPeerConfig); 2969 } 2970 2971 /** 2972 * Remove some table-cfs from config of the specified peer. 2973 * @param id a short name that identifies the cluster 2974 * @param tableCfs A map from tableName to column family names 2975 * @throws ReplicationException if tableCfs has conflict with existing config 2976 * @throws IOException if a remote or network exception occurs 2977 */ 2978 default void removeReplicationPeerTableCFs(String id, Map<TableName, List<String>> tableCfs) 2979 throws ReplicationException, IOException { 2980 if (tableCfs == null) { 2981 throw new ReplicationException("tableCfs is null"); 2982 } 2983 ReplicationPeerConfig peerConfig = getReplicationPeerConfig(id); 2984 ReplicationPeerConfig newPeerConfig = 2985 ReplicationPeerConfigUtil.removeTableCFsFromReplicationPeerConfig(tableCfs, peerConfig, id); 2986 updateReplicationPeerConfig(id, newPeerConfig); 2987 } 2988 2989 /** 2990 * Return a list of replication peers. 2991 * @return a list of replication peers description 2992 * @throws IOException if a remote or network exception occurs 2993 */ 2994 List<ReplicationPeerDescription> listReplicationPeers() throws IOException; 2995 2996 /** 2997 * Return a list of replication peers. 2998 * @param pattern The compiled regular expression to match peer id 2999 * @return a list of replication peers description 3000 * @throws IOException if a remote or network exception occurs 3001 */ 3002 List<ReplicationPeerDescription> listReplicationPeers(Pattern pattern) throws IOException; 3003 3004 /** 3005 * Mark region server(s) as decommissioned to prevent additional regions from getting 3006 * assigned to them. Optionally unload the regions on the servers. If there are multiple servers 3007 * to be decommissioned, decommissioning them at the same time can prevent wasteful region 3008 * movements. Region unloading is asynchronous. 3009 * @param servers The list of servers to decommission. 3010 * @param offload True to offload the regions from the decommissioned servers 3011 * @throws IOException if a remote or network exception occurs 3012 */ 3013 void decommissionRegionServers(List<ServerName> servers, boolean offload) throws IOException; 3014 3015 /** 3016 * List region servers marked as decommissioned, which can not be assigned regions. 3017 * @return List of decommissioned region servers. 3018 * @throws IOException if a remote or network exception occurs 3019 */ 3020 List<ServerName> listDecommissionedRegionServers() throws IOException; 3021 3022 /** 3023 * Remove decommission marker from a region server to allow regions assignments. 3024 * Load regions onto the server if a list of regions is given. Region loading is 3025 * asynchronous. 3026 * @param server The server to recommission. 3027 * @param encodedRegionNames Regions to load onto the server. 3028 * @throws IOException if a remote or network exception occurs 3029 */ 3030 void recommissionRegionServer(ServerName server, List<byte[]> encodedRegionNames) 3031 throws IOException; 3032 3033 /** 3034 * Find all table and column families that are replicated from this cluster 3035 * @return the replicated table-cfs list of this cluster. 3036 * @throws IOException if a remote or network exception occurs 3037 */ 3038 List<TableCFs> listReplicatedTableCFs() throws IOException; 3039 3040 /** 3041 * Enable a table's replication switch. 3042 * @param tableName name of the table 3043 * @throws IOException if a remote or network exception occurs 3044 */ 3045 void enableTableReplication(TableName tableName) throws IOException; 3046 3047 /** 3048 * Disable a table's replication switch. 3049 * @param tableName name of the table 3050 * @throws IOException if a remote or network exception occurs 3051 */ 3052 void disableTableReplication(TableName tableName) throws IOException; 3053 3054 /** 3055 * Clear compacting queues on a regionserver. 3056 * @param serverName the region server name 3057 * @param queues the set of queue name 3058 * @throws IOException if a remote or network exception occurs 3059 * @throws InterruptedException 3060 */ 3061 void clearCompactionQueues(ServerName serverName, Set<String> queues) 3062 throws IOException, InterruptedException; 3063 3064 /** 3065 * List dead region servers. 3066 * @return List of dead region servers. 3067 * @throws IOException if a remote or network exception occurs 3068 */ 3069 default List<ServerName> listDeadServers() throws IOException { 3070 return getClusterMetrics(EnumSet.of(Option.DEAD_SERVERS)).getDeadServerNames(); 3071 } 3072 3073 /** 3074 * Clear dead region servers from master. 3075 * @param servers list of dead region servers. 3076 * @throws IOException if a remote or network exception occurs 3077 * @return List of servers that are not cleared 3078 */ 3079 List<ServerName> clearDeadServers(List<ServerName> servers) throws IOException; 3080 3081 /** 3082 * Create a new table by cloning the existent table schema. 3083 * @param tableName name of the table to be cloned 3084 * @param newTableName name of the new table where the table will be created 3085 * @param preserveSplits True if the splits should be preserved 3086 * @throws IOException if a remote or network exception occurs 3087 */ 3088 void cloneTableSchema(TableName tableName, TableName newTableName, boolean preserveSplits) 3089 throws IOException; 3090 3091 /** 3092 * Switch the rpc throttle enable state. 3093 * @param enable Set to <code>true</code> to enable, <code>false</code> to disable. 3094 * @return Previous rpc throttle enabled value 3095 * @throws IOException if a remote or network exception occurs 3096 */ 3097 boolean switchRpcThrottle(boolean enable) throws IOException; 3098 3099 /** 3100 * Get if the rpc throttle is enabled. 3101 * @return True if rpc throttle is enabled 3102 * @throws IOException if a remote or network exception occurs 3103 */ 3104 boolean isRpcThrottleEnabled() throws IOException; 3105 3106 /** 3107 * Switch the exceed throttle quota. If enabled, user/table/namespace throttle quota 3108 * can be exceeded if region server has availble quota. 3109 * @param enable Set to <code>true</code> to enable, <code>false</code> to disable. 3110 * @return Previous exceed throttle enabled value 3111 * @throws IOException if a remote or network exception occurs 3112 */ 3113 boolean exceedThrottleQuotaSwitch(final boolean enable) throws IOException; 3114 3115 /** 3116 * Fetches the table sizes on the filesystem as tracked by the HBase Master. 3117 * @throws IOException if a remote or network exception occurs 3118 */ 3119 Map<TableName, Long> getSpaceQuotaTableSizes() throws IOException; 3120 3121 /** 3122 * Fetches the observed {@link SpaceQuotaSnapshotView}s observed by a RegionServer. 3123 * @throws IOException if a remote or network exception occurs 3124 */ 3125 Map<TableName, ? extends SpaceQuotaSnapshotView> getRegionServerSpaceQuotaSnapshots( 3126 ServerName serverName) throws IOException; 3127 3128 /** 3129 * Returns the Master's view of a quota on the given {@code namespace} or null if the Master has 3130 * no quota information on that namespace. 3131 * @throws IOException if a remote or network exception occurs 3132 */ 3133 SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(String namespace) throws IOException; 3134 3135 /** 3136 * Returns the Master's view of a quota on the given {@code tableName} or null if the Master has 3137 * no quota information on that table. 3138 * @throws IOException if a remote or network exception occurs 3139 */ 3140 SpaceQuotaSnapshotView getCurrentSpaceQuotaSnapshot(TableName tableName) throws IOException; 3141 3142 /** 3143 * Grants user specific permissions 3144 * @param userPermission user name and the specific permission 3145 * @param mergeExistingPermissions If set to false, later granted permissions will override 3146 * previous granted permissions. otherwise, it'll merge with previous granted 3147 * permissions. 3148 * @throws IOException if a remote or network exception occurs 3149 */ 3150 void grant(UserPermission userPermission, boolean mergeExistingPermissions) throws IOException; 3151 3152 /** 3153 * Revokes user specific permissions 3154 * @param userPermission user name and the specific permission 3155 * @throws IOException if a remote or network exception occurs 3156 */ 3157 void revoke(UserPermission userPermission) throws IOException; 3158 3159 /** 3160 * Get the global/namespace/table permissions for user 3161 * @param getUserPermissionsRequest A request contains which user, global, namespace or table 3162 * permissions needed 3163 * @return The user and permission list 3164 * @throws IOException if a remote or network exception occurs 3165 */ 3166 List<UserPermission> getUserPermissions(GetUserPermissionsRequest getUserPermissionsRequest) 3167 throws IOException; 3168 3169 /** 3170 * Check if the user has specific permissions 3171 * @param userName the user name 3172 * @param permissions the specific permission list 3173 * @return True if user has the specific permissions 3174 * @throws IOException if a remote or network exception occurs 3175 */ 3176 List<Boolean> hasUserPermissions(String userName, List<Permission> permissions) 3177 throws IOException; 3178 3179 /** 3180 * Check if call user has specific permissions 3181 * @param permissions the specific permission list 3182 * @return True if user has the specific permissions 3183 * @throws IOException if a remote or network exception occurs 3184 */ 3185 default List<Boolean> hasUserPermissions(List<Permission> permissions) throws IOException { 3186 return hasUserPermissions(null, permissions); 3187 } 3188 3189 /** 3190 * Turn on or off the auto snapshot cleanup based on TTL. 3191 * 3192 * @param on Set to <code>true</code> to enable, <code>false</code> to disable. 3193 * @param synchronous If <code>true</code>, it waits until current snapshot cleanup is completed, 3194 * if outstanding. 3195 * @return Previous auto snapshot cleanup value 3196 * @throws IOException if a remote or network exception occurs 3197 */ 3198 boolean snapshotCleanupSwitch(final boolean on, final boolean synchronous) 3199 throws IOException; 3200 3201 /** 3202 * Query the current state of the auto snapshot cleanup based on TTL. 3203 * 3204 * @return <code>true</code> if the auto snapshot cleanup is enabled, 3205 * <code>false</code> otherwise. 3206 * @throws IOException if a remote or network exception occurs 3207 */ 3208 boolean isSnapshotCleanupEnabled() throws IOException; 3209 3210 3211 /** 3212 * Retrieves online slow/large RPC logs from the provided list of 3213 * RegionServers 3214 * 3215 * @param serverNames Server names to get slowlog responses from 3216 * @param logQueryFilter filter to be used if provided (determines slow / large RPC logs) 3217 * @return online slowlog response list 3218 * @throws IOException if a remote or network exception occurs 3219 * @deprecated since 2.4.0 and will be removed in 4.0.0. 3220 * Use {@link #getLogEntries(Set, String, ServerType, int, Map)} instead. 3221 */ 3222 @Deprecated 3223 default List<OnlineLogRecord> getSlowLogResponses(final Set<ServerName> serverNames, 3224 final LogQueryFilter logQueryFilter) throws IOException { 3225 String logType; 3226 if (LogQueryFilter.Type.LARGE_LOG.equals(logQueryFilter.getType())) { 3227 logType = "LARGE_LOG"; 3228 } else { 3229 logType = "SLOW_LOG"; 3230 } 3231 Map<String, Object> filterParams = new HashMap<>(); 3232 filterParams.put("regionName", logQueryFilter.getRegionName()); 3233 filterParams.put("clientAddress", logQueryFilter.getClientAddress()); 3234 filterParams.put("tableName", logQueryFilter.getTableName()); 3235 filterParams.put("userName", logQueryFilter.getUserName()); 3236 filterParams.put("filterByOperator", logQueryFilter.getFilterByOperator().toString()); 3237 List<LogEntry> logEntries = 3238 getLogEntries(serverNames, logType, ServerType.REGION_SERVER, logQueryFilter.getLimit(), 3239 filterParams); 3240 return logEntries.stream().map(logEntry -> (OnlineLogRecord) logEntry) 3241 .collect(Collectors.toList()); 3242 } 3243 3244 /** 3245 * Clears online slow/large RPC logs from the provided list of 3246 * RegionServers 3247 * 3248 * @param serverNames Set of Server names to clean slowlog responses from 3249 * @return List of booleans representing if online slowlog response buffer is cleaned 3250 * from each RegionServer 3251 * @throws IOException if a remote or network exception occurs 3252 */ 3253 List<Boolean> clearSlowLogResponses(final Set<ServerName> serverNames) 3254 throws IOException; 3255 3256 3257 /** 3258 * Retrieve recent online records from HMaster / RegionServers. 3259 * Examples include slow/large RPC logs, balancer decisions by master. 3260 * 3261 * @param serverNames servers to retrieve records from, useful in case of records maintained 3262 * by RegionServer as we can select specific server. In case of servertype=MASTER, logs will 3263 * only come from the currently active master. 3264 * @param logType string representing type of log records 3265 * @param serverType enum for server type: HMaster or RegionServer 3266 * @param limit put a limit to list of records that server should send in response 3267 * @param filterParams additional filter params 3268 * @return Log entries representing online records from servers 3269 * @throws IOException if a remote or network exception occurs 3270 */ 3271 List<LogEntry> getLogEntries(Set<ServerName> serverNames, String logType, 3272 ServerType serverType, int limit, Map<String, Object> filterParams) throws IOException; 3273}