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.regionserver; 019 020import java.io.IOException; 021import java.util.Collection; 022import java.util.List; 023import java.util.Map; 024 025import org.apache.hadoop.hbase.Cell; 026import org.apache.hadoop.hbase.CellComparator; 027import org.apache.hadoop.hbase.CompareOperator; 028import org.apache.hadoop.hbase.HBaseInterfaceAudience; 029import org.apache.hadoop.hbase.client.Append; 030import org.apache.hadoop.hbase.client.CompactionState; 031import org.apache.hadoop.hbase.client.Delete; 032import org.apache.hadoop.hbase.client.Get; 033import org.apache.hadoop.hbase.client.Increment; 034import org.apache.hadoop.hbase.client.Mutation; 035import org.apache.hadoop.hbase.client.Put; 036import org.apache.hadoop.hbase.client.RegionInfo; 037import org.apache.hadoop.hbase.client.Result; 038import org.apache.hadoop.hbase.client.RowMutations; 039import org.apache.hadoop.hbase.client.Scan; 040import org.apache.hadoop.hbase.client.TableDescriptor; 041import org.apache.hadoop.hbase.conf.ConfigurationObserver; 042import org.apache.hadoop.hbase.coprocessor.RegionCoprocessor; 043import org.apache.hadoop.hbase.filter.ByteArrayComparable; 044import org.apache.hadoop.hbase.io.TimeRange; 045import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; 046import org.apache.yetus.audience.InterfaceAudience; 047import org.apache.yetus.audience.InterfaceStability; 048 049/** 050 * Region is a subset of HRegion with operations required for the {@link RegionCoprocessor 051 * Coprocessors}. The operations include ability to do mutations, requesting compaction, getting 052 * different counters/sizes, locking rows and getting access to {@linkplain Store}s. 053 */ 054@InterfaceAudience.LimitedPrivate(HBaseInterfaceAudience.COPROC) 055@InterfaceStability.Evolving 056public interface Region extends ConfigurationObserver { 057 058 /////////////////////////////////////////////////////////////////////////// 059 // Region state 060 061 /** @return region information for this region */ 062 RegionInfo getRegionInfo(); 063 064 /** @return table descriptor for this region */ 065 TableDescriptor getTableDescriptor(); 066 067 /** @return true if region is available (not closed and not closing) */ 068 boolean isAvailable(); 069 070 /** @return true if region is closed */ 071 boolean isClosed(); 072 073 /** @return True if closing process has started */ 074 boolean isClosing(); 075 076 /** @return True if region is read only */ 077 boolean isReadOnly(); 078 079 /** @return true if region is splittable */ 080 boolean isSplittable(); 081 082 /** 083 * @return true if region is mergeable 084 */ 085 boolean isMergeable(); 086 087 /** 088 * Return the list of Stores managed by this region 089 * <p>Use with caution. Exposed for use of fixup utilities. 090 * @return a list of the Stores managed by this region 091 */ 092 List<? extends Store> getStores(); 093 094 /** 095 * Return the Store for the given family 096 * <p>Use with caution. Exposed for use of fixup utilities. 097 * @return the Store for the given family 098 */ 099 Store getStore(byte[] family); 100 101 /** @return list of store file names for the given families */ 102 List<String> getStoreFileList(byte[][] columns); 103 104 /** 105 * Check the region's underlying store files, open the files that have not 106 * been opened yet, and remove the store file readers for store files no 107 * longer available. 108 * @throws IOException 109 */ 110 boolean refreshStoreFiles() throws IOException; 111 112 /** @return the max sequence id of flushed data on this region; no edit in memory will have 113 * a sequence id that is less that what is returned here. 114 */ 115 long getMaxFlushedSeqId(); 116 117 /** 118 * This can be used to determine the last time all files of this region were major compacted. 119 * @param majorCompactionOnly Only consider HFile that are the result of major compaction 120 * @return the timestamp of the oldest HFile for all stores of this region 121 */ 122 long getOldestHfileTs(boolean majorCompactionOnly) throws IOException; 123 124 /** 125 * @return map of column family names to max sequence id that was read from storage when this 126 * region was opened 127 */ 128 public Map<byte[], Long> getMaxStoreSeqId(); 129 130 /** 131 * @return The earliest time a store in the region was flushed. All 132 * other stores in the region would have been flushed either at, or 133 * after this time. 134 */ 135 long getEarliestFlushTimeForAllStores(); 136 137 /////////////////////////////////////////////////////////////////////////// 138 // Metrics 139 140 /** @return read requests count for this region */ 141 long getReadRequestsCount(); 142 143 /** @return filtered read requests count for this region */ 144 long getFilteredReadRequestsCount(); 145 146 /** @return write request count for this region */ 147 long getWriteRequestsCount(); 148 149 /** 150 * @return memstore size for this region, in bytes. It just accounts data size of cells added to 151 * the memstores of this Region. Means size in bytes for key, value and tags within Cells. 152 * It wont consider any java heap overhead for the cell objects or any other. 153 */ 154 long getMemStoreDataSize(); 155 156 /** 157 * @return memstore heap size for this region, in bytes. It accounts data size of cells 158 * added to the memstores of this Region, as well as java heap overhead for the cell 159 * objects or any other. 160 */ 161 long getMemStoreHeapSize(); 162 163 /** 164 * @return memstore off-heap size for this region, in bytes. It accounts data size of cells 165 * added to the memstores of this Region, as well as overhead for the cell 166 * objects or any other that is allocated off-heap. 167 */ 168 long getMemStoreOffHeapSize(); 169 170 /** @return the number of mutations processed bypassing the WAL */ 171 long getNumMutationsWithoutWAL(); 172 173 /** @return the size of data processed bypassing the WAL, in bytes */ 174 long getDataInMemoryWithoutWAL(); 175 176 /** @return the number of blocked requests */ 177 long getBlockedRequestsCount(); 178 179 /** @return the number of checkAndMutate guards that passed */ 180 long getCheckAndMutateChecksPassed(); 181 182 /** @return the number of failed checkAndMutate guards */ 183 long getCheckAndMutateChecksFailed(); 184 185 /////////////////////////////////////////////////////////////////////////// 186 // Locking 187 188 // Region read locks 189 190 /** 191 * Operation enum is used in {@link Region#startRegionOperation} and elsewhere to provide 192 * context for various checks. 193 */ 194 enum Operation { 195 ANY, GET, PUT, DELETE, SCAN, APPEND, INCREMENT, SPLIT_REGION, MERGE_REGION, BATCH_MUTATE, 196 REPLAY_BATCH_MUTATE, COMPACT_REGION, REPLAY_EVENT, SNAPSHOT, COMPACT_SWITCH 197 } 198 199 /** 200 * This method needs to be called before any public call that reads or 201 * modifies data. 202 * Acquires a read lock and checks if the region is closing or closed. 203 * <p>{@link #closeRegionOperation} MUST then always be called after 204 * the operation has completed, whether it succeeded or failed. 205 * @throws IOException 206 */ 207 // TODO Exposing this and closeRegionOperation() as we have getRowLock() exposed. 208 // Remove if we get rid of exposing getRowLock(). 209 void startRegionOperation() throws IOException; 210 211 /** 212 * This method needs to be called before any public call that reads or 213 * modifies data. 214 * Acquires a read lock and checks if the region is closing or closed. 215 * <p>{@link #closeRegionOperation} MUST then always be called after 216 * the operation has completed, whether it succeeded or failed. 217 * @param op The operation is about to be taken on the region 218 * @throws IOException 219 */ 220 void startRegionOperation(Operation op) throws IOException; 221 222 /** 223 * Closes the region operation lock. 224 * @throws IOException 225 */ 226 void closeRegionOperation() throws IOException; 227 228 /** 229 * Closes the region operation lock. This needs to be called in the finally block corresponding 230 * to the try block of {@link #startRegionOperation(Operation)} 231 * @throws IOException 232 */ 233 void closeRegionOperation(Operation op) throws IOException; 234 235 // Row write locks 236 237 /** 238 * Row lock held by a given thread. 239 * One thread may acquire multiple locks on the same row simultaneously. 240 * The locks must be released by calling release() from the same thread. 241 */ 242 public interface RowLock { 243 /** 244 * Release the given lock. If there are no remaining locks held by the current thread 245 * then unlock the row and allow other threads to acquire the lock. 246 * @throws IllegalArgumentException if called by a different thread than the lock owning 247 * thread 248 */ 249 void release(); 250 } 251 252 /** 253 * 254 * Get a row lock for the specified row. All locks are reentrant. 255 * 256 * Before calling this function make sure that a region operation has already been 257 * started (the calling thread has already acquired the region-close-guard lock). 258 * <p> 259 * The obtained locks should be released after use by {@link RowLock#release()} 260 * <p> 261 * NOTE: the boolean passed here has changed. It used to be a boolean that 262 * stated whether or not to wait on the lock. Now it is whether it an exclusive 263 * lock is requested. 264 * 265 * @param row The row actions will be performed against 266 * @param readLock is the lock reader or writer. True indicates that a non-exclusive 267 * lock is requested 268 * @see #startRegionOperation() 269 * @see #startRegionOperation(Operation) 270 */ 271 // TODO this needs to be exposed as we have RowProcessor now. If RowProcessor is removed, we can 272 // remove this too.. 273 RowLock getRowLock(byte[] row, boolean readLock) throws IOException; 274 275 /////////////////////////////////////////////////////////////////////////// 276 // Region operations 277 278 /** 279 * Perform one or more append operations on a row. 280 * @param append 281 * @return result of the operation 282 * @throws IOException 283 */ 284 Result append(Append append) throws IOException; 285 286 /** 287 * Perform a batch of mutations. 288 * <p> 289 * Note this supports only Put and Delete mutations and will ignore other types passed. 290 * @param mutations the list of mutations 291 * @return an array of OperationStatus which internally contains the 292 * OperationStatusCode and the exceptionMessage if any. 293 * @throws IOException 294 */ 295 OperationStatus[] batchMutate(Mutation[] mutations) 296 throws IOException; 297 298 /** 299 * Atomically checks if a row/family/qualifier value matches the expected value and if it does, 300 * it performs the mutation. If the passed value is null, the lack of column value 301 * (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a 302 * single row. 303 * @param row to check 304 * @param family column family to check 305 * @param qualifier column qualifier to check 306 * @param op the comparison operator 307 * @param comparator the expected value 308 * @param mutation data to put if check succeeds 309 * @return true if mutation was applied, false otherwise 310 */ 311 default boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op, 312 ByteArrayComparable comparator, Mutation mutation) throws IOException { 313 return checkAndMutate(row, family, qualifier, op, comparator, TimeRange.allTime(), mutation); 314 } 315 316 /** 317 * Atomically checks if a row/family/qualifier value matches the expected value and if it does, 318 * it performs the mutation. If the passed value is null, the lack of column value 319 * (ie: non-existence) is used. See checkAndRowMutate to do many checkAndPuts at a time on a 320 * single row. 321 * @param row to check 322 * @param family column family to check 323 * @param qualifier column qualifier to check 324 * @param op the comparison operator 325 * @param comparator the expected value 326 * @param mutation data to put if check succeeds 327 * @param timeRange time range to check 328 * @return true if mutation was applied, false otherwise 329 */ 330 boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op, 331 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException; 332 333 /** 334 * Atomically checks if a row/family/qualifier value matches the expected values and if it does, 335 * it performs the row mutations. If the passed value is null, the lack of column value 336 * (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate 337 * to do one checkAndMutate at a time. 338 * @param row to check 339 * @param family column family to check 340 * @param qualifier column qualifier to check 341 * @param op the comparison operator 342 * @param comparator the expected value 343 * @param mutations data to put if check succeeds 344 * @return true if mutations were applied, false otherwise 345 */ 346 default boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, 347 ByteArrayComparable comparator, RowMutations mutations) throws IOException { 348 return checkAndRowMutate(row, family, qualifier, op, comparator, TimeRange.allTime(), 349 mutations); 350 } 351 352 /** 353 * Atomically checks if a row/family/qualifier value matches the expected values and if it does, 354 * it performs the row mutations. If the passed value is null, the lack of column value 355 * (ie: non-existence) is used. Use to do many mutations on a single row. Use checkAndMutate 356 * to do one checkAndMutate at a time. 357 * @param row to check 358 * @param family column family to check 359 * @param qualifier column qualifier to check 360 * @param op the comparison operator 361 * @param comparator the expected value 362 * @param mutations data to put if check succeeds 363 * @param timeRange time range to check 364 * @return true if mutations were applied, false otherwise 365 */ 366 boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier, CompareOperator op, 367 ByteArrayComparable comparator, TimeRange timeRange, RowMutations mutations) 368 throws IOException; 369 370 /** 371 * Deletes the specified cells/row. 372 * @param delete 373 * @throws IOException 374 */ 375 void delete(Delete delete) throws IOException; 376 377 /** 378 * Do a get based on the get parameter. 379 * @param get query parameters 380 * @return result of the operation 381 */ 382 Result get(Get get) throws IOException; 383 384 /** 385 * Do a get based on the get parameter. 386 * @param get query parameters 387 * @param withCoprocessor invoke coprocessor or not. We don't want to 388 * always invoke cp. 389 * @return list of cells resulting from the operation 390 */ 391 List<Cell> get(Get get, boolean withCoprocessor) throws IOException; 392 393 /** 394 * Return an iterator that scans over the HRegion, returning the indicated 395 * columns and rows specified by the {@link Scan}. 396 * <p> 397 * This Iterator must be closed by the caller. 398 * 399 * @param scan configured {@link Scan} 400 * @return RegionScanner 401 * @throws IOException read exceptions 402 */ 403 RegionScanner getScanner(Scan scan) throws IOException; 404 405 /** 406 * Return an iterator that scans over the HRegion, returning the indicated columns and rows 407 * specified by the {@link Scan}. The scanner will also include the additional scanners passed 408 * along with the scanners for the specified Scan instance. Should be careful with the usage to 409 * pass additional scanners only within this Region 410 * <p> 411 * This Iterator must be closed by the caller. 412 * 413 * @param scan configured {@link Scan} 414 * @param additionalScanners Any additional scanners to be used 415 * @return RegionScanner 416 * @throws IOException read exceptions 417 */ 418 RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException; 419 420 /** The comparator to be used with the region */ 421 CellComparator getCellComparator(); 422 423 /** 424 * Perform one or more increment operations on a row. 425 * @param increment 426 * @return result of the operation 427 * @throws IOException 428 */ 429 Result increment(Increment increment) throws IOException; 430 431 /** 432 * Performs multiple mutations atomically on a single row. Currently 433 * {@link Put} and {@link Delete} are supported. 434 * 435 * @param mutations object that specifies the set of mutations to perform atomically 436 * @throws IOException 437 */ 438 void mutateRow(RowMutations mutations) throws IOException; 439 440 /** 441 * Perform atomic mutations within the region. 442 * 443 * @param mutations The list of mutations to perform. 444 * <code>mutations</code> can contain operations for multiple rows. 445 * Caller has to ensure that all rows are contained in this region. 446 * @param rowsToLock Rows to lock 447 * @param nonceGroup Optional nonce group of the operation (client Id) 448 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") 449 * If multiple rows are locked care should be taken that 450 * <code>rowsToLock</code> is sorted in order to avoid deadlocks. 451 * @throws IOException 452 */ 453 // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for 454 // Changing processRowsWithLocks and RowProcessor 455 void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, 456 long nonceGroup, long nonce) throws IOException; 457 458 /** 459 * Performs atomic multiple reads and writes on a given row. 460 * 461 * @param processor The object defines the reads and writes to a row. 462 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use 463 * Coprocessors instead. 464 */ 465 @Deprecated 466 void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException; 467 468 /** 469 * Performs atomic multiple reads and writes on a given row. 470 * 471 * @param processor The object defines the reads and writes to a row. 472 * @param nonceGroup Optional nonce group of the operation (client Id) 473 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") 474 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use 475 * Coprocessors instead. 476 */ 477 // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for 478 // Changing processRowsWithLocks and RowProcessor 479 @Deprecated 480 void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce) 481 throws IOException; 482 483 /** 484 * Performs atomic multiple reads and writes on a given row. 485 * 486 * @param processor The object defines the reads and writes to a row. 487 * @param timeout The timeout of the processor.process() execution 488 * Use a negative number to switch off the time bound 489 * @param nonceGroup Optional nonce group of the operation (client Id) 490 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") 491 * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0. For customization, use 492 * Coprocessors instead. 493 */ 494 // TODO Should not be exposing with params nonceGroup, nonce. Change when doing the jira for 495 // Changing processRowsWithLocks and RowProcessor 496 @Deprecated 497 void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, long nonceGroup, long nonce) 498 throws IOException; 499 500 /** 501 * Puts some data in the table. 502 * @param put 503 * @throws IOException 504 */ 505 void put(Put put) throws IOException; 506 507 /////////////////////////////////////////////////////////////////////////// 508 // Flushes, compactions, splits, etc. 509 // Wizards only, please 510 511 /** 512 * @return if a given region is in compaction now. 513 */ 514 CompactionState getCompactionState(); 515 516 /** 517 * Request compaction on this region. 518 */ 519 void requestCompaction(String why, int priority, boolean major, 520 CompactionLifeCycleTracker tracker) throws IOException; 521 522 /** 523 * Request compaction for the given family 524 */ 525 void requestCompaction(byte[] family, String why, int priority, boolean major, 526 CompactionLifeCycleTracker tracker) throws IOException; 527 528 /** 529 * Request flush on this region. 530 */ 531 void requestFlush(FlushLifeCycleTracker tracker) throws IOException; 532 533 /** 534 * Wait for all current flushes of the region to complete 535 * 536 * @param timeout The maximum time to wait in milliseconds. 537 * @return False when timeout elapsed but flushes are not over. True when flushes are over within 538 * max wait time period. 539 */ 540 boolean waitForFlushes(long timeout); 541}