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