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 static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL; 021import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY; 022import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent; 023 024import edu.umd.cs.findbugs.annotations.Nullable; 025import java.io.EOFException; 026import java.io.FileNotFoundException; 027import java.io.IOException; 028import java.io.InterruptedIOException; 029import java.lang.reflect.Constructor; 030import java.nio.ByteBuffer; 031import java.nio.charset.StandardCharsets; 032import java.text.ParseException; 033import java.util.AbstractList; 034import java.util.ArrayList; 035import java.util.Arrays; 036import java.util.Collection; 037import java.util.Collections; 038import java.util.HashMap; 039import java.util.HashSet; 040import java.util.Iterator; 041import java.util.List; 042import java.util.Map; 043import java.util.Map.Entry; 044import java.util.NavigableMap; 045import java.util.NavigableSet; 046import java.util.Objects; 047import java.util.Optional; 048import java.util.RandomAccess; 049import java.util.Set; 050import java.util.TreeMap; 051import java.util.UUID; 052import java.util.concurrent.Callable; 053import java.util.concurrent.CompletionService; 054import java.util.concurrent.ConcurrentHashMap; 055import java.util.concurrent.ConcurrentMap; 056import java.util.concurrent.ConcurrentSkipListMap; 057import java.util.concurrent.ExecutionException; 058import java.util.concurrent.ExecutorCompletionService; 059import java.util.concurrent.ExecutorService; 060import java.util.concurrent.Executors; 061import java.util.concurrent.Future; 062import java.util.concurrent.FutureTask; 063import java.util.concurrent.ThreadFactory; 064import java.util.concurrent.ThreadPoolExecutor; 065import java.util.concurrent.TimeUnit; 066import java.util.concurrent.TimeoutException; 067import java.util.concurrent.atomic.AtomicBoolean; 068import java.util.concurrent.atomic.AtomicInteger; 069import java.util.concurrent.atomic.LongAdder; 070import java.util.concurrent.locks.Lock; 071import java.util.concurrent.locks.ReadWriteLock; 072import java.util.concurrent.locks.ReentrantReadWriteLock; 073import java.util.function.Function; 074import java.util.stream.Collectors; 075import java.util.stream.Stream; 076import org.apache.hadoop.conf.Configuration; 077import org.apache.hadoop.fs.FileStatus; 078import org.apache.hadoop.fs.FileSystem; 079import org.apache.hadoop.fs.LocatedFileStatus; 080import org.apache.hadoop.fs.Path; 081import org.apache.hadoop.hbase.Cell; 082import org.apache.hadoop.hbase.CellBuilderType; 083import org.apache.hadoop.hbase.CellComparator; 084import org.apache.hadoop.hbase.CellComparatorImpl; 085import org.apache.hadoop.hbase.CellScanner; 086import org.apache.hadoop.hbase.CellUtil; 087import org.apache.hadoop.hbase.CompareOperator; 088import org.apache.hadoop.hbase.CompoundConfiguration; 089import org.apache.hadoop.hbase.DoNotRetryIOException; 090import org.apache.hadoop.hbase.DroppedSnapshotException; 091import org.apache.hadoop.hbase.ExtendedCellBuilderFactory; 092import org.apache.hadoop.hbase.HConstants; 093import org.apache.hadoop.hbase.HConstants.OperationStatusCode; 094import org.apache.hadoop.hbase.HDFSBlocksDistribution; 095import org.apache.hadoop.hbase.KeyValue; 096import org.apache.hadoop.hbase.MetaCellComparator; 097import org.apache.hadoop.hbase.NamespaceDescriptor; 098import org.apache.hadoop.hbase.NotServingRegionException; 099import org.apache.hadoop.hbase.PrivateCellUtil; 100import org.apache.hadoop.hbase.RegionTooBusyException; 101import org.apache.hadoop.hbase.TableName; 102import org.apache.hadoop.hbase.Tag; 103import org.apache.hadoop.hbase.TagUtil; 104import org.apache.hadoop.hbase.UnknownScannerException; 105import org.apache.hadoop.hbase.client.Append; 106import org.apache.hadoop.hbase.client.CheckAndMutate; 107import org.apache.hadoop.hbase.client.CheckAndMutateResult; 108import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor; 109import org.apache.hadoop.hbase.client.CompactionState; 110import org.apache.hadoop.hbase.client.Delete; 111import org.apache.hadoop.hbase.client.Durability; 112import org.apache.hadoop.hbase.client.Get; 113import org.apache.hadoop.hbase.client.Increment; 114import org.apache.hadoop.hbase.client.IsolationLevel; 115import org.apache.hadoop.hbase.client.Mutation; 116import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor; 117import org.apache.hadoop.hbase.client.Put; 118import org.apache.hadoop.hbase.client.RegionInfo; 119import org.apache.hadoop.hbase.client.RegionInfoBuilder; 120import org.apache.hadoop.hbase.client.RegionReplicaUtil; 121import org.apache.hadoop.hbase.client.Result; 122import org.apache.hadoop.hbase.client.Row; 123import org.apache.hadoop.hbase.client.RowMutations; 124import org.apache.hadoop.hbase.client.Scan; 125import org.apache.hadoop.hbase.client.TableDescriptor; 126import org.apache.hadoop.hbase.client.TableDescriptorBuilder; 127import org.apache.hadoop.hbase.conf.ConfigurationManager; 128import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver; 129import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; 130import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare; 131import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException; 132import org.apache.hadoop.hbase.exceptions.TimeoutIOException; 133import org.apache.hadoop.hbase.exceptions.UnknownProtocolException; 134import org.apache.hadoop.hbase.filter.BinaryComparator; 135import org.apache.hadoop.hbase.filter.ByteArrayComparable; 136import org.apache.hadoop.hbase.filter.Filter; 137import org.apache.hadoop.hbase.filter.FilterWrapper; 138import org.apache.hadoop.hbase.filter.IncompatibleFilterException; 139import org.apache.hadoop.hbase.io.HFileLink; 140import org.apache.hadoop.hbase.io.HeapSize; 141import org.apache.hadoop.hbase.io.TimeRange; 142import org.apache.hadoop.hbase.io.hfile.BlockCache; 143import org.apache.hadoop.hbase.io.hfile.HFile; 144import org.apache.hadoop.hbase.ipc.CallerDisconnectedException; 145import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils; 146import org.apache.hadoop.hbase.ipc.RpcCall; 147import org.apache.hadoop.hbase.ipc.RpcServer; 148import org.apache.hadoop.hbase.mob.MobFileCache; 149import org.apache.hadoop.hbase.monitoring.MonitoredTask; 150import org.apache.hadoop.hbase.monitoring.TaskMonitor; 151import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager; 152import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry; 153import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope; 154import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState; 155import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext; 156import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker; 157import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory; 158import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController; 159import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector; 160import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; 161import org.apache.hadoop.hbase.regionserver.wal.WALUtil; 162import org.apache.hadoop.hbase.replication.ReplicationUtils; 163import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver; 164import org.apache.hadoop.hbase.security.User; 165import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil; 166import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos; 167import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall; 168import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad; 169import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId; 170import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription; 171import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos; 172import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor; 173import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor; 174import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction; 175import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor; 176import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor; 177import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType; 178import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor; 179import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils; 180import org.apache.hadoop.hbase.snapshot.SnapshotManifest; 181import org.apache.hadoop.hbase.trace.TraceUtil; 182import org.apache.hadoop.hbase.util.Bytes; 183import org.apache.hadoop.hbase.util.CancelableProgressable; 184import org.apache.hadoop.hbase.util.ClassSize; 185import org.apache.hadoop.hbase.util.CommonFSUtils; 186import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; 187import org.apache.hadoop.hbase.util.FSUtils; 188import org.apache.hadoop.hbase.util.HashedBytes; 189import org.apache.hadoop.hbase.util.NonceKey; 190import org.apache.hadoop.hbase.util.Pair; 191import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil; 192import org.apache.hadoop.hbase.util.TableDescriptorChecker; 193import org.apache.hadoop.hbase.util.Threads; 194import org.apache.hadoop.hbase.wal.WAL; 195import org.apache.hadoop.hbase.wal.WALEdit; 196import org.apache.hadoop.hbase.wal.WALFactory; 197import org.apache.hadoop.hbase.wal.WALKey; 198import org.apache.hadoop.hbase.wal.WALKeyImpl; 199import org.apache.hadoop.hbase.wal.WALSplitUtil; 200import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay; 201import org.apache.hadoop.util.StringUtils; 202import org.apache.htrace.core.TraceScope; 203import org.apache.hbase.thirdparty.com.google.common.base.Preconditions; 204import org.apache.hbase.thirdparty.com.google.common.collect.Iterables; 205import org.apache.hbase.thirdparty.com.google.common.collect.Lists; 206import org.apache.hbase.thirdparty.com.google.common.collect.Maps; 207import org.apache.hbase.thirdparty.com.google.common.io.Closeables; 208import org.apache.hbase.thirdparty.com.google.protobuf.Service; 209import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat; 210import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations; 211import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils; 212import org.apache.yetus.audience.InterfaceAudience; 213import org.slf4j.Logger; 214import org.slf4j.LoggerFactory; 215 216/** 217 * Regions store data for a certain region of a table. It stores all columns 218 * for each row. A given table consists of one or more Regions. 219 * 220 * <p>An Region is defined by its table and its key extent. 221 * 222 * <p>Locking at the Region level serves only one purpose: preventing the 223 * region from being closed (and consequently split) while other operations 224 * are ongoing. Each row level operation obtains both a row lock and a region 225 * read lock for the duration of the operation. While a scanner is being 226 * constructed, getScanner holds a read lock. If the scanner is successfully 227 * constructed, it holds a read lock until it is closed. A close takes out a 228 * write lock and consequently will block for ongoing operations and will block 229 * new operations from starting while the close is in progress. 230 */ 231@SuppressWarnings("deprecation") 232@InterfaceAudience.Private 233public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region { 234 private static final Logger LOG = LoggerFactory.getLogger(HRegion.class); 235 236 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = 237 "hbase.hregion.scan.loadColumnFamiliesOnDemand"; 238 239 public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize"; 240 public static final int DEFAULT_MAX_CELL_SIZE = 10485760; 241 242 public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE = 243 "hbase.regionserver.minibatch.size"; 244 public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000; 245 246 public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync"; 247 public static final boolean DEFAULT_WAL_HSYNC = false; 248 249 /** 250 * This is for for using HRegion as a local storage, where we may put the recovered edits in a 251 * special place. Once this is set, we will only replay the recovered edits under this directory 252 * and ignore the original replay directory configs. 253 */ 254 public static final String SPECIAL_RECOVERED_EDITS_DIR = 255 "hbase.hregion.special.recovered.edits.dir"; 256 257 /** 258 * Whether to use {@link MetaCellComparator} even if we are not meta region. Used when creating 259 * master local region. 260 */ 261 public static final String USE_META_CELL_COMPARATOR = "hbase.region.use.meta.cell.comparator"; 262 263 public static final boolean DEFAULT_USE_META_CELL_COMPARATOR = false; 264 265 final AtomicBoolean closed = new AtomicBoolean(false); 266 267 /* Closing can take some time; use the closing flag if there is stuff we don't 268 * want to do while in closing state; e.g. like offer this region up to the 269 * master as a region to close if the carrying regionserver is overloaded. 270 * Once set, it is never cleared. 271 */ 272 final AtomicBoolean closing = new AtomicBoolean(false); 273 274 /** 275 * The max sequence id of flushed data on this region. There is no edit in memory that is 276 * less that this sequence id. 277 */ 278 private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM; 279 280 /** 281 * Record the sequence id of last flush operation. Can be in advance of 282 * {@link #maxFlushedSeqId} when flushing a single column family. In this case, 283 * {@link #maxFlushedSeqId} will be older than the oldest edit in memory. 284 */ 285 private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM; 286 287 /** 288 * The sequence id of the last replayed open region event from the primary region. This is used 289 * to skip entries before this due to the possibility of replay edits coming out of order from 290 * replication. 291 */ 292 protected volatile long lastReplayedOpenRegionSeqId = -1L; 293 protected volatile long lastReplayedCompactionSeqId = -1L; 294 295 ////////////////////////////////////////////////////////////////////////////// 296 // Members 297 ////////////////////////////////////////////////////////////////////////////// 298 299 // map from a locked row to the context for that lock including: 300 // - CountDownLatch for threads waiting on that row 301 // - the thread that owns the lock (allow reentrancy) 302 // - reference count of (reentrant) locks held by the thread 303 // - the row itself 304 private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows = 305 new ConcurrentHashMap<>(); 306 307 protected final Map<byte[], HStore> stores = 308 new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR); 309 310 // TODO: account for each registered handler in HeapSize computation 311 private Map<String, com.google.protobuf.Service> coprocessorServiceHandlers = Maps.newHashMap(); 312 313 // Track data size in all memstores 314 private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing(); 315 RegionServicesForStores regionServicesForStores; 316 317 // Debug possible data loss due to WAL off 318 final LongAdder numMutationsWithoutWAL = new LongAdder(); 319 final LongAdder dataInMemoryWithoutWAL = new LongAdder(); 320 321 // Debug why CAS operations are taking a while. 322 final LongAdder checkAndMutateChecksPassed = new LongAdder(); 323 final LongAdder checkAndMutateChecksFailed = new LongAdder(); 324 325 // Number of requests 326 // Count rows for scan 327 final LongAdder readRequestsCount = new LongAdder(); 328 final LongAdder filteredReadRequestsCount = new LongAdder(); 329 // Count rows for multi row mutations 330 final LongAdder writeRequestsCount = new LongAdder(); 331 332 // Number of requests blocked by memstore size. 333 private final LongAdder blockedRequestsCount = new LongAdder(); 334 335 // Compaction LongAdders 336 final LongAdder compactionsFinished = new LongAdder(); 337 final LongAdder compactionsFailed = new LongAdder(); 338 final LongAdder compactionNumFilesCompacted = new LongAdder(); 339 final LongAdder compactionNumBytesCompacted = new LongAdder(); 340 final LongAdder compactionsQueued = new LongAdder(); 341 final LongAdder flushesQueued = new LongAdder(); 342 343 private BlockCache blockCache; 344 private MobFileCache mobFileCache; 345 private final WAL wal; 346 private final HRegionFileSystem fs; 347 protected final Configuration conf; 348 private final Configuration baseConf; 349 private final int rowLockWaitDuration; 350 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000; 351 352 private Path regionDir; 353 private FileSystem walFS; 354 355 // set to true if the region is restored from snapshot 356 private boolean isRestoredRegion = false; 357 358 public void setRestoredRegion(boolean restoredRegion) { 359 isRestoredRegion = restoredRegion; 360 } 361 362 // The internal wait duration to acquire a lock before read/update 363 // from the region. It is not per row. The purpose of this wait time 364 // is to avoid waiting a long time while the region is busy, so that 365 // we can release the IPC handler soon enough to improve the 366 // availability of the region server. It can be adjusted by 367 // tuning configuration "hbase.busy.wait.duration". 368 final long busyWaitDuration; 369 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT; 370 371 // If updating multiple rows in one call, wait longer, 372 // i.e. waiting for busyWaitDuration * # of rows. However, 373 // we can limit the max multiplier. 374 final int maxBusyWaitMultiplier; 375 376 // Max busy wait duration. There is no point to wait longer than the RPC 377 // purge timeout, when a RPC call will be terminated by the RPC engine. 378 final long maxBusyWaitDuration; 379 380 // Max cell size. If nonzero, the maximum allowed size for any given cell 381 // in bytes 382 final long maxCellSize; 383 384 // Number of mutations for minibatch processing. 385 private final int miniBatchSize; 386 387 // negative number indicates infinite timeout 388 static final long DEFAULT_ROW_PROCESSOR_TIMEOUT = 60 * 1000L; 389 final ExecutorService rowProcessorExecutor = Executors.newCachedThreadPool(); 390 391 private final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints; 392 393 /** 394 * The sequence ID that was enLongAddered when this region was opened. 395 */ 396 private long openSeqNum = HConstants.NO_SEQNUM; 397 398 /** 399 * The default setting for whether to enable on-demand CF loading for 400 * scan requests to this region. Requests can override it. 401 */ 402 private boolean isLoadingCfsOnDemandDefault = false; 403 404 private final AtomicInteger majorInProgress = new AtomicInteger(0); 405 private final AtomicInteger minorInProgress = new AtomicInteger(0); 406 407 // 408 // Context: During replay we want to ensure that we do not lose any data. So, we 409 // have to be conservative in how we replay wals. For each store, we calculate 410 // the maxSeqId up to which the store was flushed. And, skip the edits which 411 // are equal to or lower than maxSeqId for each store. 412 // The following map is populated when opening the region 413 Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR); 414 415 /** Saved state from replaying prepare flush cache */ 416 private PrepareFlushResult prepareFlushResult = null; 417 418 private volatile ConfigurationManager configurationManager; 419 420 // Used for testing. 421 private volatile Long timeoutForWriteLock = null; 422 423 private final CellComparator cellComparator; 424 425 /** 426 * @return The smallest mvcc readPoint across all the scanners in this 427 * region. Writes older than this readPoint, are included in every 428 * read operation. 429 */ 430 public long getSmallestReadPoint() { 431 long minimumReadPoint; 432 // We need to ensure that while we are calculating the smallestReadPoint 433 // no new RegionScanners can grab a readPoint that we are unaware of. 434 // We achieve this by synchronizing on the scannerReadPoints object. 435 synchronized (scannerReadPoints) { 436 minimumReadPoint = mvcc.getReadPoint(); 437 for (Long readPoint : this.scannerReadPoints.values()) { 438 if (readPoint < minimumReadPoint) { 439 minimumReadPoint = readPoint; 440 } 441 } 442 } 443 return minimumReadPoint; 444 } 445 446 /* 447 * Data structure of write state flags used coordinating flushes, 448 * compactions and closes. 449 */ 450 static class WriteState { 451 // Set while a memstore flush is happening. 452 volatile boolean flushing = false; 453 // Set when a flush has been requested. 454 volatile boolean flushRequested = false; 455 // Number of compactions running. 456 AtomicInteger compacting = new AtomicInteger(0); 457 // Gets set in close. If set, cannot compact or flush again. 458 volatile boolean writesEnabled = true; 459 // Set if region is read-only 460 volatile boolean readOnly = false; 461 // whether the reads are enabled. This is different than readOnly, because readOnly is 462 // static in the lifetime of the region, while readsEnabled is dynamic 463 volatile boolean readsEnabled = true; 464 465 /** 466 * Set flags that make this region read-only. 467 * 468 * @param onOff flip value for region r/o setting 469 */ 470 synchronized void setReadOnly(final boolean onOff) { 471 this.writesEnabled = !onOff; 472 this.readOnly = onOff; 473 } 474 475 boolean isReadOnly() { 476 return this.readOnly; 477 } 478 479 boolean isFlushRequested() { 480 return this.flushRequested; 481 } 482 483 void setReadsEnabled(boolean readsEnabled) { 484 this.readsEnabled = readsEnabled; 485 } 486 487 static final long HEAP_SIZE = ClassSize.align( 488 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN); 489 } 490 491 /** 492 * Objects from this class are created when flushing to describe all the different states that 493 * that method ends up in. The Result enum describes those states. The sequence id should only 494 * be specified if the flush was successful, and the failure message should only be specified 495 * if it didn't flush. 496 */ 497 public static class FlushResultImpl implements FlushResult { 498 final Result result; 499 final String failureReason; 500 final long flushSequenceId; 501 final boolean wroteFlushWalMarker; 502 503 /** 504 * Convenience constructor to use when the flush is successful, the failure message is set to 505 * null. 506 * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED. 507 * @param flushSequenceId Generated sequence id that comes right after the edits in the 508 * memstores. 509 */ 510 FlushResultImpl(Result result, long flushSequenceId) { 511 this(result, flushSequenceId, null, false); 512 assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result 513 .FLUSHED_COMPACTION_NEEDED; 514 } 515 516 /** 517 * Convenience constructor to use when we cannot flush. 518 * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH. 519 * @param failureReason Reason why we couldn't flush. 520 */ 521 FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) { 522 this(result, -1, failureReason, wroteFlushMarker); 523 assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH; 524 } 525 526 /** 527 * Constructor with all the parameters. 528 * @param result Any of the Result. 529 * @param flushSequenceId Generated sequence id if the memstores were flushed else -1. 530 * @param failureReason Reason why we couldn't flush, or null. 531 */ 532 FlushResultImpl(Result result, long flushSequenceId, String failureReason, 533 boolean wroteFlushMarker) { 534 this.result = result; 535 this.flushSequenceId = flushSequenceId; 536 this.failureReason = failureReason; 537 this.wroteFlushWalMarker = wroteFlushMarker; 538 } 539 540 /** 541 * Convenience method, the equivalent of checking if result is 542 * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED. 543 * @return true if the memstores were flushed, else false. 544 */ 545 @Override 546 public boolean isFlushSucceeded() { 547 return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result 548 .FLUSHED_COMPACTION_NEEDED; 549 } 550 551 /** 552 * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED. 553 * @return True if the flush requested a compaction, else false (doesn't even mean it flushed). 554 */ 555 @Override 556 public boolean isCompactionNeeded() { 557 return result == Result.FLUSHED_COMPACTION_NEEDED; 558 } 559 560 @Override 561 public String toString() { 562 return new StringBuilder() 563 .append("flush result:").append(result).append(", ") 564 .append("failureReason:").append(failureReason).append(",") 565 .append("flush seq id").append(flushSequenceId).toString(); 566 } 567 568 @Override 569 public Result getResult() { 570 return result; 571 } 572 } 573 574 /** A result object from prepare flush cache stage */ 575 static class PrepareFlushResult { 576 final FlushResultImpl result; // indicating a failure result from prepare 577 final TreeMap<byte[], StoreFlushContext> storeFlushCtxs; 578 final TreeMap<byte[], List<Path>> committedFiles; 579 final TreeMap<byte[], MemStoreSize> storeFlushableSize; 580 final long startTime; 581 final long flushOpSeqId; 582 final long flushedSeqId; 583 final MemStoreSizing totalFlushableSize; 584 585 /** Constructs an early exit case */ 586 PrepareFlushResult(FlushResultImpl result, long flushSeqId) { 587 this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, MemStoreSizing.DUD); 588 } 589 590 /** Constructs a successful prepare flush result */ 591 PrepareFlushResult( 592 TreeMap<byte[], StoreFlushContext> storeFlushCtxs, 593 TreeMap<byte[], List<Path>> committedFiles, 594 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId, 595 long flushedSeqId, MemStoreSizing totalFlushableSize) { 596 this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime, 597 flushSeqId, flushedSeqId, totalFlushableSize); 598 } 599 600 private PrepareFlushResult( 601 FlushResultImpl result, 602 TreeMap<byte[], StoreFlushContext> storeFlushCtxs, 603 TreeMap<byte[], List<Path>> committedFiles, 604 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId, 605 long flushedSeqId, MemStoreSizing totalFlushableSize) { 606 this.result = result; 607 this.storeFlushCtxs = storeFlushCtxs; 608 this.committedFiles = committedFiles; 609 this.storeFlushableSize = storeFlushableSize; 610 this.startTime = startTime; 611 this.flushOpSeqId = flushSeqId; 612 this.flushedSeqId = flushedSeqId; 613 this.totalFlushableSize = totalFlushableSize; 614 } 615 616 public FlushResult getResult() { 617 return this.result; 618 } 619 } 620 621 /** 622 * A class that tracks exceptions that have been observed in one batch. Not thread safe. 623 */ 624 static class ObservedExceptionsInBatch { 625 private boolean wrongRegion = false; 626 private boolean failedSanityCheck = false; 627 private boolean wrongFamily = false; 628 629 /** 630 * @return If a {@link WrongRegionException} has been observed. 631 */ 632 boolean hasSeenWrongRegion() { 633 return wrongRegion; 634 } 635 636 /** 637 * Records that a {@link WrongRegionException} has been observed. 638 */ 639 void sawWrongRegion() { 640 wrongRegion = true; 641 } 642 643 /** 644 * @return If a {@link FailedSanityCheckException} has been observed. 645 */ 646 boolean hasSeenFailedSanityCheck() { 647 return failedSanityCheck; 648 } 649 650 /** 651 * Records that a {@link FailedSanityCheckException} has been observed. 652 */ 653 void sawFailedSanityCheck() { 654 failedSanityCheck = true; 655 } 656 657 /** 658 * @return If a {@link NoSuchColumnFamilyException} has been observed. 659 */ 660 boolean hasSeenNoSuchFamily() { 661 return wrongFamily; 662 } 663 664 /** 665 * Records that a {@link NoSuchColumnFamilyException} has been observed. 666 */ 667 void sawNoSuchFamily() { 668 wrongFamily = true; 669 } 670 } 671 672 final WriteState writestate = new WriteState(); 673 674 long memstoreFlushSize; 675 final long timestampSlop; 676 final long rowProcessorTimeout; 677 678 // Last flush time for each Store. Useful when we are flushing for each column 679 private final ConcurrentMap<HStore, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>(); 680 681 protected RegionServerServices rsServices; 682 private RegionServerAccounting rsAccounting; 683 private long flushCheckInterval; 684 // flushPerChanges is to prevent too many changes in memstore 685 private long flushPerChanges; 686 private long blockingMemStoreSize; 687 // Used to guard closes 688 final ReentrantReadWriteLock lock; 689 // Used to track interruptible holders of the region lock. Currently that is only RPC handler 690 // threads. Boolean value in map determines if lock holder can be interrupted, normally true, 691 // but may be false when thread is transiting a critical section. 692 final ConcurrentHashMap<Thread, Boolean> regionLockHolders; 693 694 // Stop updates lock 695 private final ReentrantReadWriteLock updatesLock = new ReentrantReadWriteLock(); 696 697 private final MultiVersionConcurrencyControl mvcc; 698 699 // Coprocessor host 700 private RegionCoprocessorHost coprocessorHost; 701 702 private TableDescriptor htableDescriptor = null; 703 private RegionSplitPolicy splitPolicy; 704 private FlushPolicy flushPolicy; 705 706 private final MetricsRegion metricsRegion; 707 private final MetricsRegionWrapperImpl metricsRegionWrapper; 708 private final Durability regionDurability; 709 private final boolean regionStatsEnabled; 710 // Stores the replication scope of the various column families of the table 711 // that has non-default scope 712 private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<>( 713 Bytes.BYTES_COMPARATOR); 714 715 private final StoreHotnessProtector storeHotnessProtector; 716 717 /** 718 * HRegion constructor. This constructor should only be used for testing and 719 * extensions. Instances of HRegion should be instantiated with the 720 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method. 721 * 722 * @param tableDir qualified path of directory where region should be located, 723 * usually the table directory. 724 * @param wal The WAL is the outbound log for any updates to the HRegion 725 * The wal file is a logfile from the previous execution that's 726 * custom-computed for this HRegion. The HRegionServer computes and sorts the 727 * appropriate wal info for this HRegion. If there is a previous wal file 728 * (implying that the HRegion has been written-to before), then read it from 729 * the supplied path. 730 * @param fs is the filesystem. 731 * @param confParam is global configuration settings. 732 * @param regionInfo - RegionInfo that describes the region 733 * is new), then read them from the supplied path. 734 * @param htd the table descriptor 735 * @param rsServices reference to {@link RegionServerServices} or null 736 * @deprecated Use other constructors. 737 */ 738 @Deprecated 739 public HRegion(final Path tableDir, final WAL wal, final FileSystem fs, 740 final Configuration confParam, final RegionInfo regionInfo, 741 final TableDescriptor htd, final RegionServerServices rsServices) { 742 this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo), 743 wal, confParam, htd, rsServices); 744 } 745 746 /** 747 * HRegion constructor. This constructor should only be used for testing and 748 * extensions. Instances of HRegion should be instantiated with the 749 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method. 750 * 751 * @param fs is the filesystem. 752 * @param wal The WAL is the outbound log for any updates to the HRegion 753 * The wal file is a logfile from the previous execution that's 754 * custom-computed for this HRegion. The HRegionServer computes and sorts the 755 * appropriate wal info for this HRegion. If there is a previous wal file 756 * (implying that the HRegion has been written-to before), then read it from 757 * the supplied path. 758 * @param confParam is global configuration settings. 759 * @param htd the table descriptor 760 * @param rsServices reference to {@link RegionServerServices} or null 761 */ 762 public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam, 763 final TableDescriptor htd, final RegionServerServices rsServices) { 764 if (htd == null) { 765 throw new IllegalArgumentException("Need table descriptor"); 766 } 767 768 if (confParam instanceof CompoundConfiguration) { 769 throw new IllegalArgumentException("Need original base configuration"); 770 } 771 772 this.wal = wal; 773 this.fs = fs; 774 this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog()); 775 776 // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor 777 this.baseConf = confParam; 778 this.conf = new CompoundConfiguration().add(confParam).addBytesMap(htd.getValues()); 779 this.cellComparator = htd.isMetaTable() || 780 conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR) ? 781 MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR; 782 this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK, 783 DEFAULT_FAIR_REENTRANT_CLOSE_LOCK)); 784 this.regionLockHolders = new ConcurrentHashMap<>(); 785 this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL, 786 DEFAULT_CACHE_FLUSH_INTERVAL); 787 this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES); 788 if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) { 789 throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed " 790 + MAX_FLUSH_PER_CHANGES); 791 } 792 int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration", 793 DEFAULT_ROWLOCK_WAIT_DURATION); 794 if (tmpRowLockDuration <= 0) { 795 LOG.info("Found hbase.rowlock.wait.duration set to {}. values <= 0 will cause all row " + 796 "locking to fail. Treating it as 1ms to avoid region failure.", tmpRowLockDuration); 797 tmpRowLockDuration = 1; 798 } 799 this.rowLockWaitDuration = tmpRowLockDuration; 800 801 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true); 802 this.htableDescriptor = htd; 803 Set<byte[]> families = this.htableDescriptor.getColumnFamilyNames(); 804 for (byte[] family : families) { 805 if (!replicationScope.containsKey(family)) { 806 int scope = htd.getColumnFamily(family).getScope(); 807 // Only store those families that has NON-DEFAULT scope 808 if (scope != REPLICATION_SCOPE_LOCAL) { 809 // Do a copy before storing it here. 810 replicationScope.put(Bytes.copy(family), scope); 811 } 812 } 813 } 814 815 this.rsServices = rsServices; 816 if (rsServices != null) { 817 this.blockCache = rsServices.getBlockCache().orElse(null); 818 this.mobFileCache = rsServices.getMobFileCache().orElse(null); 819 } 820 this.regionServicesForStores = new RegionServicesForStores(this, rsServices); 821 822 setHTableSpecificConf(); 823 this.scannerReadPoints = new ConcurrentHashMap<>(); 824 825 this.busyWaitDuration = conf.getLong( 826 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION); 827 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2); 828 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) { 829 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration (" 830 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max (" 831 + maxBusyWaitMultiplier + "). Their product should be positive"); 832 } 833 this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout", 834 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT); 835 836 /* 837 * timestamp.slop provides a server-side constraint on the timestamp. This 838 * assumes that you base your TS around currentTimeMillis(). In this case, 839 * throw an error to the user if the user-specified TS is newer than now + 840 * slop. LATEST_TIMESTAMP == don't use this functionality 841 */ 842 this.timestampSlop = conf.getLong( 843 "hbase.hregion.keyvalue.timestamp.slop.millisecs", 844 HConstants.LATEST_TIMESTAMP); 845 846 /** 847 * Timeout for the process time in processRowsWithLocks(). 848 * Use -1 to switch off time bound. 849 */ 850 this.rowProcessorTimeout = conf.getLong( 851 "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT); 852 853 this.storeHotnessProtector = new StoreHotnessProtector(this, conf); 854 855 boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC); 856 /** 857 * This is the global default value for durability. All tables/mutations not defining a 858 * durability or using USE_DEFAULT will default to this value. 859 */ 860 Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL; 861 this.regionDurability = 862 this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability : 863 this.htableDescriptor.getDurability(); 864 865 decorateRegionConfiguration(conf); 866 if (rsServices != null) { 867 this.rsAccounting = this.rsServices.getRegionServerAccounting(); 868 // don't initialize coprocessors if not running within a regionserver 869 // TODO: revisit if coprocessors should load in other cases 870 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf); 871 this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this); 872 this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper, conf); 873 } else { 874 this.metricsRegionWrapper = null; 875 this.metricsRegion = null; 876 } 877 if (LOG.isDebugEnabled()) { 878 // Write out region name, its encoded name and storeHotnessProtector as string. 879 LOG.debug("Instantiated " + this +"; "+ storeHotnessProtector.toString()); 880 } 881 882 configurationManager = null; 883 884 // disable stats tracking system tables, but check the config for everything else 885 this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals( 886 NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ? 887 false : 888 conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE, 889 HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE); 890 891 this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE); 892 this.miniBatchSize = conf.getInt(HBASE_REGIONSERVER_MINIBATCH_SIZE, 893 DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE); 894 895 // recover the metrics of read and write requests count if they were retained 896 if (rsServices != null && rsServices.getRegionServerAccounting() != null) { 897 Pair<Long, Long> retainedRWRequestsCnt = rsServices.getRegionServerAccounting() 898 .getRetainedRegionRWRequestsCnt().get(getRegionInfo().getEncodedName()); 899 if (retainedRWRequestsCnt != null) { 900 this.setReadRequestsCount(retainedRWRequestsCnt.getFirst()); 901 this.setWriteRequestsCount(retainedRWRequestsCnt.getSecond()); 902 // remove them since won't use again 903 rsServices.getRegionServerAccounting().getRetainedRegionRWRequestsCnt() 904 .remove(getRegionInfo().getEncodedName()); 905 } 906 } 907 } 908 909 void setHTableSpecificConf() { 910 if (this.htableDescriptor == null) return; 911 long flushSize = this.htableDescriptor.getMemStoreFlushSize(); 912 913 if (flushSize <= 0) { 914 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE, 915 TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE); 916 } 917 this.memstoreFlushSize = flushSize; 918 long mult = conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER, 919 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER); 920 this.blockingMemStoreSize = this.memstoreFlushSize * mult; 921 } 922 923 /** 924 * Initialize this region. 925 * Used only by tests and SplitTransaction to reopen the region. 926 * You should use createHRegion() or openHRegion() 927 * @return What the next sequence (edit) id should be. 928 * @throws IOException e 929 * @deprecated use HRegion.createHRegion() or HRegion.openHRegion() 930 */ 931 @Deprecated 932 public long initialize() throws IOException { 933 return initialize(null); 934 } 935 936 /** 937 * Initialize this region. 938 * 939 * @param reporter Tickle every so often if initialize is taking a while. 940 * @return What the next sequence (edit) id should be. 941 * @throws IOException e 942 */ 943 long initialize(final CancelableProgressable reporter) throws IOException { 944 945 //Refuse to open the region if there is no column family in the table 946 if (htableDescriptor.getColumnFamilyCount() == 0) { 947 throw new DoNotRetryIOException("Table " + htableDescriptor.getTableName().getNameAsString()+ 948 " should have at least one column family."); 949 } 950 951 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this); 952 status.enableStatusJournal(true); 953 long nextSeqId = -1; 954 try { 955 nextSeqId = initializeRegionInternals(reporter, status); 956 return nextSeqId; 957 } catch (IOException e) { 958 LOG.warn("Failed initialize of region= {}, starting to roll back memstore", 959 getRegionInfo().getRegionNameAsString(), e); 960 // global memstore size will be decreased when dropping memstore 961 try { 962 //drop the memory used by memstore if open region fails 963 dropMemStoreContents(); 964 } catch (IOException ioE) { 965 if (conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) { 966 LOG.warn("Failed drop memstore of region= {}, " 967 + "some chunks may not released forever since MSLAB is enabled", 968 getRegionInfo().getRegionNameAsString()); 969 } 970 971 } 972 throw e; 973 } finally { 974 // nextSeqid will be -1 if the initialization fails. 975 // At least it will be 0 otherwise. 976 if (nextSeqId == -1) { 977 status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() + 978 " initialization."); 979 } 980 if (LOG.isDebugEnabled()) { 981 LOG.debug("Region open journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 982 status.prettyPrintJournal()); 983 } 984 status.cleanup(); 985 } 986 } 987 988 private long initializeRegionInternals(final CancelableProgressable reporter, 989 final MonitoredTask status) throws IOException { 990 if (coprocessorHost != null) { 991 status.setStatus("Running coprocessor pre-open hook"); 992 coprocessorHost.preOpen(); 993 } 994 995 // Write HRI to a file in case we need to recover hbase:meta 996 // Only the primary replica should write .regioninfo 997 if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 998 status.setStatus("Writing region info on filesystem"); 999 fs.checkRegionInfoOnFilesystem(); 1000 } 1001 1002 // Initialize all the HStores 1003 status.setStatus("Initializing all the Stores"); 1004 long maxSeqId = initializeStores(reporter, status); 1005 this.mvcc.advanceTo(maxSeqId); 1006 if (!isRestoredRegion && ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) { 1007 Collection<HStore> stores = this.stores.values(); 1008 try { 1009 // update the stores that we are replaying 1010 stores.forEach(HStore::startReplayingFromWAL); 1011 // Recover any edits if available. 1012 maxSeqId = Math.max(maxSeqId, 1013 replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status)); 1014 // Recover any hfiles if available 1015 maxSeqId = Math.max(maxSeqId, loadRecoveredHFilesIfAny(stores)); 1016 // Make sure mvcc is up to max. 1017 this.mvcc.advanceTo(maxSeqId); 1018 } finally { 1019 // update the stores that we are done replaying 1020 stores.forEach(HStore::stopReplayingFromWAL); 1021 } 1022 } 1023 this.lastReplayedOpenRegionSeqId = maxSeqId; 1024 1025 this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this)); 1026 this.writestate.flushRequested = false; 1027 this.writestate.compacting.set(0); 1028 1029 if (this.writestate.writesEnabled) { 1030 // Remove temporary data left over from old regions 1031 status.setStatus("Cleaning up temporary data from old regions"); 1032 fs.cleanupTempDir(); 1033 } 1034 1035 if (this.writestate.writesEnabled) { 1036 status.setStatus("Cleaning up detritus from prior splits"); 1037 // Get rid of any splits or merges that were lost in-progress. Clean out 1038 // these directories here on open. We may be opening a region that was 1039 // being split but we crashed in the middle of it all. 1040 fs.cleanupAnySplitDetritus(); 1041 fs.cleanupMergesDir(); 1042 } 1043 1044 // Initialize split policy 1045 this.splitPolicy = RegionSplitPolicy.create(this, conf); 1046 1047 // Initialize flush policy 1048 this.flushPolicy = FlushPolicyFactory.create(this, conf); 1049 1050 long lastFlushTime = EnvironmentEdgeManager.currentTime(); 1051 for (HStore store: stores.values()) { 1052 this.lastStoreFlushTimeMap.put(store, lastFlushTime); 1053 } 1054 1055 // Use maximum of log sequenceid or that which was found in stores 1056 // (particularly if no recovered edits, seqid will be -1). 1057 long nextSeqId = maxSeqId + 1; 1058 if (!isRestoredRegion) { 1059 // always get openSeqNum from the default replica, even if we are secondary replicas 1060 long maxSeqIdFromFile = WALSplitUtil.getMaxRegionSequenceId(conf, 1061 RegionReplicaUtil.getRegionInfoForDefaultReplica(getRegionInfo()), this::getFilesystem, 1062 this::getWalFileSystem); 1063 nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1; 1064 // The openSeqNum will always be increase even for read only region, as we rely on it to 1065 // determine whether a region has been successfully reopened, so here we always need to update 1066 // the max sequence id file. 1067 if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) { 1068 LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName()); 1069 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(), 1070 nextSeqId - 1); 1071 // This means we have replayed all the recovered edits and also written out the max sequence 1072 // id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617 1073 // for more details. 1074 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, 1075 getRegionInfo().getTable(), getRegionInfo().getEncodedName()); 1076 FileSystem walFs = getWalFileSystem(); 1077 if (walFs.exists(wrongRegionWALDir)) { 1078 if (!walFs.delete(wrongRegionWALDir, true)) { 1079 LOG.debug("Failed to clean up wrong region WAL directory {}", wrongRegionWALDir); 1080 } 1081 } 1082 } 1083 } 1084 1085 LOG.info("Opened {}; next sequenceid={}", this.getRegionInfo().getShortNameToLog(), nextSeqId); 1086 1087 // A region can be reopened if failed a split; reset flags 1088 this.closing.set(false); 1089 this.closed.set(false); 1090 1091 if (coprocessorHost != null) { 1092 status.setStatus("Running coprocessor post-open hooks"); 1093 coprocessorHost.postOpen(); 1094 } 1095 1096 status.markComplete("Region opened successfully"); 1097 return nextSeqId; 1098 } 1099 1100 /** 1101 * Open all Stores. 1102 * @param reporter 1103 * @param status 1104 * @return Highest sequenceId found out in a Store. 1105 * @throws IOException 1106 */ 1107 private long initializeStores(CancelableProgressable reporter, MonitoredTask status) 1108 throws IOException { 1109 return initializeStores(reporter, status, false); 1110 } 1111 1112 private long initializeStores(CancelableProgressable reporter, MonitoredTask status, 1113 boolean warmup) throws IOException { 1114 // Load in all the HStores. 1115 long maxSeqId = -1; 1116 // initialized to -1 so that we pick up MemstoreTS from column families 1117 long maxMemstoreTS = -1; 1118 1119 if (htableDescriptor.getColumnFamilyCount() != 0) { 1120 // initialize the thread pool for opening stores in parallel. 1121 ThreadPoolExecutor storeOpenerThreadPool = 1122 getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog()); 1123 CompletionService<HStore> completionService = new ExecutorCompletionService<>(storeOpenerThreadPool); 1124 1125 // initialize each store in parallel 1126 for (final ColumnFamilyDescriptor family : htableDescriptor.getColumnFamilies()) { 1127 status.setStatus("Instantiating store for column family " + family); 1128 completionService.submit(new Callable<HStore>() { 1129 @Override 1130 public HStore call() throws IOException { 1131 return instantiateHStore(family, warmup); 1132 } 1133 }); 1134 } 1135 boolean allStoresOpened = false; 1136 boolean hasSloppyStores = false; 1137 try { 1138 for (int i = 0; i < htableDescriptor.getColumnFamilyCount(); i++) { 1139 Future<HStore> future = completionService.take(); 1140 HStore store = future.get(); 1141 this.stores.put(store.getColumnFamilyDescriptor().getName(), store); 1142 if (store.isSloppyMemStore()) { 1143 hasSloppyStores = true; 1144 } 1145 1146 long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L); 1147 maxSeqIdInStores.put(Bytes.toBytes(store.getColumnFamilyName()), 1148 storeMaxSequenceId); 1149 if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) { 1150 maxSeqId = storeMaxSequenceId; 1151 } 1152 long maxStoreMemstoreTS = store.getMaxMemStoreTS().orElse(0L); 1153 if (maxStoreMemstoreTS > maxMemstoreTS) { 1154 maxMemstoreTS = maxStoreMemstoreTS; 1155 } 1156 } 1157 allStoresOpened = true; 1158 if(hasSloppyStores) { 1159 htableDescriptor = TableDescriptorBuilder.newBuilder(htableDescriptor) 1160 .setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class.getName()) 1161 .build(); 1162 LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this); 1163 } 1164 } catch (InterruptedException e) { 1165 throw throwOnInterrupt(e); 1166 } catch (ExecutionException e) { 1167 throw new IOException(e.getCause()); 1168 } finally { 1169 storeOpenerThreadPool.shutdownNow(); 1170 if (!allStoresOpened) { 1171 // something went wrong, close all opened stores 1172 LOG.error("Could not initialize all stores for the region=" + this); 1173 for (HStore store : this.stores.values()) { 1174 try { 1175 store.close(); 1176 } catch (IOException e) { 1177 LOG.warn("close store {} failed in region {}", store.toString(), this, e); 1178 } 1179 } 1180 } 1181 } 1182 } 1183 return Math.max(maxSeqId, maxMemstoreTS + 1); 1184 } 1185 1186 private void initializeWarmup(final CancelableProgressable reporter) throws IOException { 1187 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this); 1188 // Initialize all the HStores 1189 status.setStatus("Warmup all stores of " + this.getRegionInfo().getRegionNameAsString()); 1190 try { 1191 initializeStores(reporter, status, true); 1192 } finally { 1193 status.markComplete("Warmed up " + this.getRegionInfo().getRegionNameAsString()); 1194 } 1195 } 1196 1197 /** 1198 * @return Map of StoreFiles by column family 1199 */ 1200 private NavigableMap<byte[], List<Path>> getStoreFiles() { 1201 NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); 1202 for (HStore store : stores.values()) { 1203 Collection<HStoreFile> storeFiles = store.getStorefiles(); 1204 if (storeFiles == null) { 1205 continue; 1206 } 1207 List<Path> storeFileNames = new ArrayList<>(); 1208 for (HStoreFile storeFile : storeFiles) { 1209 storeFileNames.add(storeFile.getPath()); 1210 } 1211 allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames); 1212 } 1213 return allStoreFiles; 1214 } 1215 1216 protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException { 1217 Map<byte[], List<Path>> storeFiles = getStoreFiles(); 1218 RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor( 1219 RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId, 1220 getRegionServerServices().getServerName(), storeFiles); 1221 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionOpenDesc, 1222 mvcc); 1223 } 1224 1225 private void writeRegionCloseMarker(WAL wal) throws IOException { 1226 Map<byte[], List<Path>> storeFiles = getStoreFiles(); 1227 RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor( 1228 RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(), 1229 getRegionServerServices().getServerName(), storeFiles); 1230 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionEventDesc, 1231 mvcc); 1232 1233 // Store SeqId in WAL FileSystem when a region closes 1234 // checking region folder exists is due to many tests which delete the table folder while a 1235 // table is still online 1236 if (getWalFileSystem().exists(getWALRegionDir())) { 1237 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(), 1238 mvcc.getReadPoint()); 1239 } 1240 } 1241 1242 /** 1243 * @return True if this region has references. 1244 */ 1245 public boolean hasReferences() { 1246 return stores.values().stream().anyMatch(HStore::hasReferences); 1247 } 1248 1249 public void blockUpdates() { 1250 this.updatesLock.writeLock().lock(); 1251 } 1252 1253 public void unblockUpdates() { 1254 this.updatesLock.writeLock().unlock(); 1255 } 1256 1257 public HDFSBlocksDistribution getHDFSBlocksDistribution() { 1258 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); 1259 stores.values().stream().filter(s -> s.getStorefiles() != null) 1260 .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution) 1261 .forEachOrdered(hdfsBlocksDistribution::add); 1262 return hdfsBlocksDistribution; 1263 } 1264 1265 /** 1266 * This is a helper function to compute HDFS block distribution on demand 1267 * @param conf configuration 1268 * @param tableDescriptor TableDescriptor of the table 1269 * @param regionInfo encoded name of the region 1270 * @return The HDFS blocks distribution for the given region. 1271 */ 1272 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf, 1273 TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException { 1274 Path tablePath = 1275 CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName()); 1276 return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath); 1277 } 1278 1279 /** 1280 * This is a helper function to compute HDFS block distribution on demand 1281 * @param conf configuration 1282 * @param tableDescriptor TableDescriptor of the table 1283 * @param regionInfo encoded name of the region 1284 * @param tablePath the table directory 1285 * @return The HDFS blocks distribution for the given region. 1286 * @throws IOException 1287 */ 1288 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf, 1289 TableDescriptor tableDescriptor, RegionInfo regionInfo, Path tablePath) throws IOException { 1290 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution(); 1291 FileSystem fs = tablePath.getFileSystem(conf); 1292 1293 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo); 1294 for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()) { 1295 List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem 1296 .getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true); 1297 if (locatedFileStatusList == null) { 1298 continue; 1299 } 1300 1301 for (LocatedFileStatus status : locatedFileStatusList) { 1302 Path p = status.getPath(); 1303 if (StoreFileInfo.isReference(p) || HFileLink.isHFileLink(p)) { 1304 // Only construct StoreFileInfo object if its not a hfile, save obj 1305 // creation 1306 StoreFileInfo storeFileInfo = new StoreFileInfo(conf, fs, status); 1307 hdfsBlocksDistribution.add(storeFileInfo 1308 .computeHDFSBlocksDistribution(fs)); 1309 } else if (StoreFileInfo.isHFile(p)) { 1310 // If its a HFile, then lets just add to the block distribution 1311 // lets not create more objects here, not even another HDFSBlocksDistribution 1312 FSUtils.addToHDFSBlocksDistribution(hdfsBlocksDistribution, 1313 status.getBlockLocations()); 1314 } else { 1315 throw new IOException("path=" + p 1316 + " doesn't look like a valid StoreFile"); 1317 } 1318 } 1319 } 1320 return hdfsBlocksDistribution; 1321 } 1322 1323 /** 1324 * Increase the size of mem store in this region and the size of global mem 1325 * store 1326 */ 1327 void incMemStoreSize(MemStoreSize mss) { 1328 incMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), 1329 mss.getCellsCount()); 1330 } 1331 1332 void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta, 1333 int cellsCountDelta) { 1334 if (this.rsAccounting != null) { 1335 rsAccounting.incGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta); 1336 } 1337 long dataSize = this.memStoreSizing.incMemStoreSize(dataSizeDelta, heapSizeDelta, 1338 offHeapSizeDelta, cellsCountDelta); 1339 checkNegativeMemStoreDataSize(dataSize, dataSizeDelta); 1340 } 1341 1342 void decrMemStoreSize(MemStoreSize mss) { 1343 decrMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(), 1344 mss.getCellsCount()); 1345 } 1346 1347 void decrMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta, 1348 int cellsCountDelta) { 1349 if (this.rsAccounting != null) { 1350 rsAccounting.decGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta); 1351 } 1352 long dataSize = this.memStoreSizing.decMemStoreSize(dataSizeDelta, heapSizeDelta, 1353 offHeapSizeDelta, cellsCountDelta); 1354 checkNegativeMemStoreDataSize(dataSize, -dataSizeDelta); 1355 } 1356 1357 private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta) { 1358 // This is extremely bad if we make memStoreSizing negative. Log as much info on the offending 1359 // caller as possible. (memStoreSizing might be a negative value already -- freeing memory) 1360 if (memStoreDataSize < 0) { 1361 LOG.error("Asked to modify this region's (" + this.toString() 1362 + ") memStoreSizing to a negative value which is incorrect. Current memStoreSizing=" 1363 + (memStoreDataSize - delta) + ", delta=" + delta, new Exception()); 1364 } 1365 } 1366 1367 @Override 1368 public RegionInfo getRegionInfo() { 1369 return this.fs.getRegionInfo(); 1370 } 1371 1372 /** 1373 * @return Instance of {@link RegionServerServices} used by this HRegion. 1374 * Can be null. 1375 */ 1376 RegionServerServices getRegionServerServices() { 1377 return this.rsServices; 1378 } 1379 1380 @Override 1381 public long getReadRequestsCount() { 1382 return readRequestsCount.sum(); 1383 } 1384 1385 @Override 1386 public long getFilteredReadRequestsCount() { 1387 return filteredReadRequestsCount.sum(); 1388 } 1389 1390 @Override 1391 public long getWriteRequestsCount() { 1392 return writeRequestsCount.sum(); 1393 } 1394 1395 @Override 1396 public long getMemStoreDataSize() { 1397 return memStoreSizing.getDataSize(); 1398 } 1399 1400 @Override 1401 public long getMemStoreHeapSize() { 1402 return memStoreSizing.getHeapSize(); 1403 } 1404 1405 @Override 1406 public long getMemStoreOffHeapSize() { 1407 return memStoreSizing.getOffHeapSize(); 1408 } 1409 1410 /** @return store services for this region, to access services required by store level needs */ 1411 public RegionServicesForStores getRegionServicesForStores() { 1412 return regionServicesForStores; 1413 } 1414 1415 @Override 1416 public long getNumMutationsWithoutWAL() { 1417 return numMutationsWithoutWAL.sum(); 1418 } 1419 1420 @Override 1421 public long getDataInMemoryWithoutWAL() { 1422 return dataInMemoryWithoutWAL.sum(); 1423 } 1424 1425 @Override 1426 public long getBlockedRequestsCount() { 1427 return blockedRequestsCount.sum(); 1428 } 1429 1430 @Override 1431 public long getCheckAndMutateChecksPassed() { 1432 return checkAndMutateChecksPassed.sum(); 1433 } 1434 1435 @Override 1436 public long getCheckAndMutateChecksFailed() { 1437 return checkAndMutateChecksFailed.sum(); 1438 } 1439 1440 // TODO Needs to check whether we should expose our metrics system to CPs. If CPs themselves doing 1441 // the op and bypassing the core, this might be needed? Should be stop supporting the bypass 1442 // feature? 1443 public MetricsRegion getMetrics() { 1444 return metricsRegion; 1445 } 1446 1447 @Override 1448 public boolean isClosed() { 1449 return this.closed.get(); 1450 } 1451 1452 @Override 1453 public boolean isClosing() { 1454 return this.closing.get(); 1455 } 1456 1457 @Override 1458 public boolean isReadOnly() { 1459 return this.writestate.isReadOnly(); 1460 } 1461 1462 @Override 1463 public boolean isAvailable() { 1464 return !isClosed() && !isClosing(); 1465 } 1466 1467 @Override 1468 public boolean isSplittable() { 1469 return splitPolicy.canSplit(); 1470 } 1471 1472 @Override 1473 public boolean isMergeable() { 1474 if (!isAvailable()) { 1475 LOG.debug("Region " + this 1476 + " is not mergeable because it is closing or closed"); 1477 return false; 1478 } 1479 if (hasReferences()) { 1480 LOG.debug("Region " + this 1481 + " is not mergeable because it has references"); 1482 return false; 1483 } 1484 1485 return true; 1486 } 1487 1488 public boolean areWritesEnabled() { 1489 synchronized(this.writestate) { 1490 return this.writestate.writesEnabled; 1491 } 1492 } 1493 1494 public MultiVersionConcurrencyControl getMVCC() { 1495 return mvcc; 1496 } 1497 1498 @Override 1499 public long getMaxFlushedSeqId() { 1500 return maxFlushedSeqId; 1501 } 1502 1503 /** 1504 * @return readpoint considering given IsolationLevel. Pass {@code null} for default 1505 */ 1506 public long getReadPoint(IsolationLevel isolationLevel) { 1507 if (isolationLevel != null && isolationLevel == IsolationLevel.READ_UNCOMMITTED) { 1508 // This scan can read even uncommitted transactions 1509 return Long.MAX_VALUE; 1510 } 1511 return mvcc.getReadPoint(); 1512 } 1513 1514 public boolean isLoadingCfsOnDemandDefault() { 1515 return this.isLoadingCfsOnDemandDefault; 1516 } 1517 1518 /** 1519 * Close down this HRegion. Flush the cache, shut down each HStore, don't 1520 * service any more calls. 1521 * 1522 * <p>This method could take some time to execute, so don't call it from a 1523 * time-sensitive thread. 1524 * 1525 * @return Vector of all the storage files that the HRegion's component 1526 * HStores make use of. It's a list of all StoreFile objects. Returns empty 1527 * vector if already closed and null if judged that it should not close. 1528 * 1529 * @throws IOException e 1530 * @throws DroppedSnapshotException Thrown when replay of wal is required 1531 * because a Snapshot was not properly persisted. The region is put in closing mode, and the 1532 * caller MUST abort after this. 1533 */ 1534 public Map<byte[], List<HStoreFile>> close() throws IOException { 1535 return close(false); 1536 } 1537 1538 private final Object closeLock = new Object(); 1539 1540 /** Conf key for fair locking policy */ 1541 public static final String FAIR_REENTRANT_CLOSE_LOCK = 1542 "hbase.regionserver.fair.region.close.lock"; 1543 public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true; 1544 /** Conf key for the periodic flush interval */ 1545 public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL = 1546 "hbase.regionserver.optionalcacheflushinterval"; 1547 /** Default interval for the memstore flush */ 1548 public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000; 1549 /** Default interval for System tables memstore flush */ 1550 public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes 1551 1552 /** Conf key to force a flush if there are already enough changes for one region in memstore */ 1553 public static final String MEMSTORE_FLUSH_PER_CHANGES = 1554 "hbase.regionserver.flush.per.changes"; 1555 public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000; // 30 millions 1556 /** 1557 * The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes 1558 * overhead. Therefore, even 1G empty KVs occupy at least 20GB memstore size for a single region 1559 */ 1560 public static final long MAX_FLUSH_PER_CHANGES = 1000000000; // 1G 1561 1562 public static final String CLOSE_WAIT_ABORT = "hbase.regionserver.close.wait.abort"; 1563 public static final boolean DEFAULT_CLOSE_WAIT_ABORT = false; 1564 public static final String CLOSE_WAIT_TIME = "hbase.regionserver.close.wait.time.ms"; 1565 public static final long DEFAULT_CLOSE_WAIT_TIME = 60000; // 1 minute 1566 public static final String CLOSE_WAIT_INTERVAL = "hbase.regionserver.close.wait.interval.ms"; 1567 public static final long DEFAULT_CLOSE_WAIT_INTERVAL = 10000; // 10 seconds 1568 1569 /** 1570 * Close down this HRegion. Flush the cache unless abort parameter is true, 1571 * Shut down each HStore, don't service any more calls. 1572 * 1573 * This method could take some time to execute, so don't call it from a 1574 * time-sensitive thread. 1575 * 1576 * @param abort true if server is aborting (only during testing) 1577 * @return Vector of all the storage files that the HRegion's component 1578 * HStores make use of. It's a list of StoreFile objects. Can be null if 1579 * we are not to close at this time or we are already closed. 1580 * 1581 * @throws IOException e 1582 * @throws DroppedSnapshotException Thrown when replay of wal is required 1583 * because a Snapshot was not properly persisted. The region is put in closing mode, and the 1584 * caller MUST abort after this. 1585 */ 1586 public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException { 1587 // Only allow one thread to close at a time. Serialize them so dual 1588 // threads attempting to close will run up against each other. 1589 MonitoredTask status = TaskMonitor.get().createStatus( 1590 "Closing region " + this.getRegionInfo().getEncodedName() + 1591 (abort ? " due to abort" : "")); 1592 status.enableStatusJournal(true); 1593 status.setStatus("Waiting for close lock"); 1594 try { 1595 synchronized (closeLock) { 1596 return doClose(abort, status); 1597 } 1598 } finally { 1599 if (LOG.isDebugEnabled()) { 1600 LOG.debug("Region close journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 1601 status.prettyPrintJournal()); 1602 } 1603 status.cleanup(); 1604 } 1605 } 1606 1607 /** 1608 * Exposed for some very specific unit tests. 1609 */ 1610 public void setClosing(boolean closing) { 1611 this.closing.set(closing); 1612 } 1613 1614 /** 1615 * The {@link HRegion#doClose} will block forever if someone tries proving the dead lock via the unit test. 1616 * Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout. 1617 * @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose} 1618 */ 1619 public void setTimeoutForWriteLock(long timeoutForWriteLock) { 1620 assert timeoutForWriteLock >= 0; 1621 this.timeoutForWriteLock = timeoutForWriteLock; 1622 } 1623 1624 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH", 1625 justification="I think FindBugs is confused") 1626 private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status) 1627 throws IOException { 1628 if (isClosed()) { 1629 LOG.warn("Region " + this + " already closed"); 1630 return null; 1631 } 1632 1633 if (coprocessorHost != null) { 1634 status.setStatus("Running coprocessor pre-close hooks"); 1635 this.coprocessorHost.preClose(abort); 1636 } 1637 status.setStatus("Disabling compacts and flushes for region"); 1638 boolean canFlush = true; 1639 synchronized (writestate) { 1640 // Disable compacting and flushing by background threads for this 1641 // region. 1642 canFlush = !writestate.readOnly; 1643 writestate.writesEnabled = false; 1644 LOG.debug("Closing {}, disabling compactions & flushes", 1645 this.getRegionInfo().getEncodedName()); 1646 waitForFlushesAndCompactions(); 1647 } 1648 // If we were not just flushing, is it worth doing a preflush...one 1649 // that will clear out of the bulk of the memstore before we put up 1650 // the close flag? 1651 if (!abort && worthPreFlushing() && canFlush) { 1652 status.setStatus("Pre-flushing region before close"); 1653 LOG.info("Running close preflush of {}", this.getRegionInfo().getEncodedName()); 1654 try { 1655 internalFlushcache(status); 1656 } catch (IOException ioe) { 1657 // Failed to flush the region. Keep going. 1658 status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage()); 1659 } 1660 } 1661 1662 // Set the closing flag 1663 // From this point new arrivals at the region lock will get NSRE. 1664 1665 this.closing.set(true); 1666 LOG.info("Closing region {}", this); 1667 1668 // Acquire the close lock 1669 1670 // The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both 1671 // the new regionserver abort condition and interrupts for running requests. 1672 // If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior, 1673 // we will not attempt to interrupt threads servicing requests nor crash out 1674 // the regionserver if something remains stubborn. 1675 1676 final boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT); 1677 boolean useTimedWait = false; 1678 if (timeoutForWriteLock != null && timeoutForWriteLock != Long.MAX_VALUE) { 1679 // convert legacy use of timeoutForWriteLock in seconds to new use in millis 1680 timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock); 1681 useTimedWait = true; 1682 } else if (canAbort) { 1683 timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME); 1684 useTimedWait = true; 1685 } 1686 if (LOG.isDebugEnabled()) { 1687 LOG.debug((useTimedWait ? "Time limited wait" : "Waiting without time limit") + 1688 " for close lock on " + this); 1689 } 1690 final long closeWaitInterval = conf.getLong(CLOSE_WAIT_INTERVAL, DEFAULT_CLOSE_WAIT_INTERVAL); 1691 long elapsedWaitTime = 0; 1692 if (useTimedWait) { 1693 // Sanity check configuration 1694 long remainingWaitTime = timeoutForWriteLock; 1695 if (remainingWaitTime < closeWaitInterval) { 1696 LOG.warn("Time limit for close wait of " + timeoutForWriteLock + 1697 " ms is less than the configured lock acquisition wait interval " + 1698 closeWaitInterval + " ms, using wait interval as time limit"); 1699 remainingWaitTime = closeWaitInterval; 1700 } 1701 boolean acquired = false; 1702 do { 1703 long start = EnvironmentEdgeManager.currentTime(); 1704 try { 1705 acquired = lock.writeLock().tryLock(Math.min(remainingWaitTime, closeWaitInterval), 1706 TimeUnit.MILLISECONDS); 1707 } catch (InterruptedException e) { 1708 // Interrupted waiting for close lock. More likely the server is shutting down, not 1709 // normal operation, so aborting upon interrupt while waiting on this lock would not 1710 // provide much value. Throw an IOE (as IIOE) like we would in the case where we 1711 // fail to acquire the lock. 1712 String msg = "Interrupted while waiting for close lock on " + this; 1713 LOG.warn(msg, e); 1714 throw (InterruptedIOException) new InterruptedIOException(msg).initCause(e); 1715 } 1716 long elapsed = EnvironmentEdgeManager.currentTime() - start; 1717 elapsedWaitTime += elapsed; 1718 remainingWaitTime -= elapsed; 1719 if (canAbort && !acquired && remainingWaitTime > 0) { 1720 // Before we loop to wait again, interrupt all region operations that might 1721 // still be in progress, to encourage them to break out of waiting states or 1722 // inner loops, throw an exception to clients, and release the read lock via 1723 // endRegionOperation. 1724 if (LOG.isDebugEnabled()) { 1725 LOG.debug("Interrupting region operations after waiting for close lock for " + 1726 elapsedWaitTime + " ms on " + this + ", " + remainingWaitTime + 1727 " ms remaining"); 1728 } 1729 interruptRegionOperations(); 1730 } 1731 } while (!acquired && remainingWaitTime > 0); 1732 1733 // If we fail to acquire the lock, trigger an abort if we can; otherwise throw an IOE 1734 // to let the caller know we could not proceed with the close. 1735 if (!acquired) { 1736 String msg = "Failed to acquire close lock on " + this + " after waiting " + 1737 elapsedWaitTime + " ms"; 1738 LOG.error(msg); 1739 if (canAbort) { 1740 // If we failed to acquire the write lock, abort the server 1741 rsServices.abort(msg, null); 1742 } 1743 throw new IOException(msg); 1744 } 1745 1746 } else { 1747 1748 long start = EnvironmentEdgeManager.currentTime(); 1749 lock.writeLock().lock(); 1750 elapsedWaitTime = EnvironmentEdgeManager.currentTime() - start; 1751 1752 } 1753 1754 if (LOG.isDebugEnabled()) { 1755 LOG.debug("Acquired close lock on " + this + " after waiting " + 1756 elapsedWaitTime + " ms"); 1757 } 1758 1759 status.setStatus("Disabling writes for close"); 1760 try { 1761 if (this.isClosed()) { 1762 status.abort("Already got closed by another process"); 1763 // SplitTransaction handles the null 1764 return null; 1765 } 1766 LOG.debug("Updates disabled for region " + this); 1767 // Don't flush the cache if we are aborting 1768 if (!abort && canFlush) { 1769 int failedfFlushCount = 0; 1770 int flushCount = 0; 1771 long tmp = 0; 1772 long remainingSize = this.memStoreSizing.getDataSize(); 1773 while (remainingSize > 0) { 1774 try { 1775 internalFlushcache(status); 1776 if(flushCount >0) { 1777 LOG.info("Running extra flush, " + flushCount + 1778 " (carrying snapshot?) " + this); 1779 } 1780 flushCount++; 1781 tmp = this.memStoreSizing.getDataSize(); 1782 if (tmp >= remainingSize) { 1783 failedfFlushCount++; 1784 } 1785 remainingSize = tmp; 1786 if (failedfFlushCount > 5) { 1787 // If we failed 5 times and are unable to clear memory, abort 1788 // so we do not lose data 1789 throw new DroppedSnapshotException("Failed clearing memory after " + 1790 flushCount + " attempts on region: " + 1791 Bytes.toStringBinary(getRegionInfo().getRegionName())); 1792 } 1793 } catch (IOException ioe) { 1794 status.setStatus("Failed flush " + this + ", putting online again"); 1795 synchronized (writestate) { 1796 writestate.writesEnabled = true; 1797 } 1798 // Have to throw to upper layers. I can't abort server from here. 1799 throw ioe; 1800 } 1801 } 1802 } 1803 1804 Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR); 1805 if (!stores.isEmpty()) { 1806 // initialize the thread pool for closing stores in parallel. 1807 ThreadPoolExecutor storeCloserThreadPool = 1808 getStoreOpenAndCloseThreadPool("StoreCloser-" + 1809 getRegionInfo().getRegionNameAsString()); 1810 CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService = 1811 new ExecutorCompletionService<>(storeCloserThreadPool); 1812 1813 // close each store in parallel 1814 for (HStore store : stores.values()) { 1815 MemStoreSize mss = store.getFlushableSize(); 1816 if (!(abort || mss.getDataSize() == 0 || writestate.readOnly)) { 1817 if (getRegionServerServices() != null) { 1818 getRegionServerServices().abort("Assertion failed while closing store " 1819 + getRegionInfo().getRegionNameAsString() + " " + store 1820 + ". flushableSize expected=0, actual={" + mss 1821 + "}. Current memStoreSize=" + this.memStoreSizing.getMemStoreSize() + 1822 ". Maybe a coprocessor " 1823 + "operation failed and left the memstore in a partially updated state.", null); 1824 } 1825 } 1826 completionService 1827 .submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() { 1828 @Override 1829 public Pair<byte[], Collection<HStoreFile>> call() throws IOException { 1830 return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close()); 1831 } 1832 }); 1833 } 1834 try { 1835 for (int i = 0; i < stores.size(); i++) { 1836 Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take(); 1837 Pair<byte[], Collection<HStoreFile>> storeFiles = future.get(); 1838 List<HStoreFile> familyFiles = result.get(storeFiles.getFirst()); 1839 if (familyFiles == null) { 1840 familyFiles = new ArrayList<>(); 1841 result.put(storeFiles.getFirst(), familyFiles); 1842 } 1843 familyFiles.addAll(storeFiles.getSecond()); 1844 } 1845 } catch (InterruptedException e) { 1846 throw throwOnInterrupt(e); 1847 } catch (ExecutionException e) { 1848 Throwable cause = e.getCause(); 1849 if (cause instanceof IOException) { 1850 throw (IOException) cause; 1851 } 1852 throw new IOException(cause); 1853 } finally { 1854 storeCloserThreadPool.shutdownNow(); 1855 } 1856 } 1857 1858 status.setStatus("Writing region close event to WAL"); 1859 // Always write close marker to wal even for read only table. This is not a big problem as we 1860 // do not write any data into the region; it is just a meta edit in the WAL file. 1861 if (!abort && wal != null && getRegionServerServices() != null && 1862 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) { 1863 writeRegionCloseMarker(wal); 1864 } 1865 1866 this.closed.set(true); 1867 if (!canFlush) { 1868 decrMemStoreSize(this.memStoreSizing.getMemStoreSize()); 1869 } else if (this.memStoreSizing.getDataSize() != 0) { 1870 LOG.error("Memstore data size is {} in region {}", this.memStoreSizing.getDataSize(), this); 1871 } 1872 if (coprocessorHost != null) { 1873 status.setStatus("Running coprocessor post-close hooks"); 1874 this.coprocessorHost.postClose(abort); 1875 } 1876 if (this.metricsRegion != null) { 1877 this.metricsRegion.close(); 1878 } 1879 if (this.metricsRegionWrapper != null) { 1880 Closeables.close(this.metricsRegionWrapper, true); 1881 } 1882 status.markComplete("Closed"); 1883 LOG.info("Closed {}", this); 1884 return result; 1885 } finally { 1886 lock.writeLock().unlock(); 1887 } 1888 } 1889 1890 /** Wait for all current flushes and compactions of the region to complete */ 1891 // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for 1892 // Phoenix needs. 1893 public void waitForFlushesAndCompactions() { 1894 synchronized (writestate) { 1895 if (this.writestate.readOnly) { 1896 // we should not wait for replayed flushed if we are read only (for example in case the 1897 // region is a secondary replica). 1898 return; 1899 } 1900 boolean interrupted = false; 1901 try { 1902 while (writestate.compacting.get() > 0 || writestate.flushing) { 1903 LOG.debug("waiting for " + writestate.compacting + " compactions" 1904 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this); 1905 try { 1906 writestate.wait(); 1907 } catch (InterruptedException iex) { 1908 // essentially ignore and propagate the interrupt back up 1909 LOG.warn("Interrupted while waiting in region {}", this); 1910 interrupted = true; 1911 break; 1912 } 1913 } 1914 } finally { 1915 if (interrupted) { 1916 Thread.currentThread().interrupt(); 1917 } 1918 } 1919 } 1920 } 1921 1922 /** 1923 * Wait for all current flushes of the region to complete 1924 */ 1925 public void waitForFlushes() { 1926 waitForFlushes(0);// Unbound wait 1927 } 1928 1929 @Override 1930 public boolean waitForFlushes(long timeout) { 1931 synchronized (writestate) { 1932 if (this.writestate.readOnly) { 1933 // we should not wait for replayed flushed if we are read only (for example in case the 1934 // region is a secondary replica). 1935 return true; 1936 } 1937 if (!writestate.flushing) return true; 1938 long start = System.currentTimeMillis(); 1939 long duration = 0; 1940 boolean interrupted = false; 1941 LOG.debug("waiting for cache flush to complete for region " + this); 1942 try { 1943 while (writestate.flushing) { 1944 if (timeout > 0 && duration >= timeout) break; 1945 try { 1946 long toWait = timeout == 0 ? 0 : (timeout - duration); 1947 writestate.wait(toWait); 1948 } catch (InterruptedException iex) { 1949 // essentially ignore and propagate the interrupt back up 1950 LOG.warn("Interrupted while waiting in region {}", this); 1951 interrupted = true; 1952 break; 1953 } finally { 1954 duration = System.currentTimeMillis() - start; 1955 } 1956 } 1957 } finally { 1958 if (interrupted) { 1959 Thread.currentThread().interrupt(); 1960 } 1961 } 1962 LOG.debug("Waited {} ms for region {} flush to complete", duration, this); 1963 return !(writestate.flushing); 1964 } 1965 } 1966 1967 protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool( 1968 final String threadNamePrefix) { 1969 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount()); 1970 int maxThreads = Math.min(numStores, 1971 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX, 1972 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)); 1973 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix); 1974 } 1975 1976 protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool( 1977 final String threadNamePrefix) { 1978 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount()); 1979 int maxThreads = Math.max(1, 1980 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX, 1981 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX) 1982 / numStores); 1983 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix); 1984 } 1985 1986 static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads, 1987 final String threadNamePrefix) { 1988 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS, 1989 new ThreadFactory() { 1990 private int count = 1; 1991 1992 @Override 1993 public Thread newThread(Runnable r) { 1994 return new Thread(r, threadNamePrefix + "-" + count++); 1995 } 1996 }); 1997 } 1998 1999 /** 2000 * @return True if its worth doing a flush before we put up the close flag. 2001 */ 2002 private boolean worthPreFlushing() { 2003 return this.memStoreSizing.getDataSize() > 2004 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5); 2005 } 2006 2007 ////////////////////////////////////////////////////////////////////////////// 2008 // HRegion accessors 2009 ////////////////////////////////////////////////////////////////////////////// 2010 2011 @Override 2012 public TableDescriptor getTableDescriptor() { 2013 return this.htableDescriptor; 2014 } 2015 2016 void setTableDescriptor(TableDescriptor desc) { 2017 htableDescriptor = desc; 2018 } 2019 2020 /** @return WAL in use for this region */ 2021 public WAL getWAL() { 2022 return this.wal; 2023 } 2024 2025 public BlockCache getBlockCache() { 2026 return this.blockCache; 2027 } 2028 2029 /** 2030 * Only used for unit test which doesn't start region server. 2031 */ 2032 public void setBlockCache(BlockCache blockCache) { 2033 this.blockCache = blockCache; 2034 } 2035 2036 public MobFileCache getMobFileCache() { 2037 return this.mobFileCache; 2038 } 2039 2040 /** 2041 * Only used for unit test which doesn't start region server. 2042 */ 2043 public void setMobFileCache(MobFileCache mobFileCache) { 2044 this.mobFileCache = mobFileCache; 2045 } 2046 2047 /** 2048 * @return split policy for this region. 2049 */ 2050 RegionSplitPolicy getSplitPolicy() { 2051 return this.splitPolicy; 2052 } 2053 2054 /** 2055 * A split takes the config from the parent region & passes it to the daughter 2056 * region's constructor. If 'conf' was passed, you would end up using the HTD 2057 * of the parent region in addition to the new daughter HTD. Pass 'baseConf' 2058 * to the daughter regions to avoid this tricky dedupe problem. 2059 * @return Configuration object 2060 */ 2061 Configuration getBaseConf() { 2062 return this.baseConf; 2063 } 2064 2065 /** @return {@link FileSystem} being used by this region */ 2066 public FileSystem getFilesystem() { 2067 return fs.getFileSystem(); 2068 } 2069 2070 /** @return the {@link HRegionFileSystem} used by this region */ 2071 public HRegionFileSystem getRegionFileSystem() { 2072 return this.fs; 2073 } 2074 2075 /** @return the WAL {@link HRegionFileSystem} used by this region */ 2076 HRegionWALFileSystem getRegionWALFileSystem() throws IOException { 2077 return new HRegionWALFileSystem(conf, getWalFileSystem(), 2078 CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo()); 2079 } 2080 2081 /** @return the WAL {@link FileSystem} being used by this region */ 2082 FileSystem getWalFileSystem() throws IOException { 2083 if (walFS == null) { 2084 walFS = CommonFSUtils.getWALFileSystem(conf); 2085 } 2086 return walFS; 2087 } 2088 2089 /** 2090 * @return the Region directory under WALRootDirectory 2091 * @throws IOException if there is an error getting WALRootDir 2092 */ 2093 public Path getWALRegionDir() throws IOException { 2094 if (regionDir == null) { 2095 regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(), 2096 getRegionInfo().getEncodedName()); 2097 } 2098 return regionDir; 2099 } 2100 2101 @Override 2102 public long getEarliestFlushTimeForAllStores() { 2103 return Collections.min(lastStoreFlushTimeMap.values()); 2104 } 2105 2106 @Override 2107 public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException { 2108 long result = Long.MAX_VALUE; 2109 for (HStore store : stores.values()) { 2110 Collection<HStoreFile> storeFiles = store.getStorefiles(); 2111 if (storeFiles == null) { 2112 continue; 2113 } 2114 for (HStoreFile file : storeFiles) { 2115 StoreFileReader sfReader = file.getReader(); 2116 if (sfReader == null) { 2117 continue; 2118 } 2119 HFile.Reader reader = sfReader.getHFileReader(); 2120 if (reader == null) { 2121 continue; 2122 } 2123 if (majorCompactionOnly) { 2124 byte[] val = reader.getHFileInfo().get(MAJOR_COMPACTION_KEY); 2125 if (val == null || !Bytes.toBoolean(val)) { 2126 continue; 2127 } 2128 } 2129 result = Math.min(result, reader.getFileContext().getFileCreateTime()); 2130 } 2131 } 2132 return result == Long.MAX_VALUE ? 0 : result; 2133 } 2134 2135 RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) { 2136 long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId; 2137 byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes(); 2138 regionLoadBldr.clearStoreCompleteSequenceId(); 2139 for (byte[] familyName : this.stores.keySet()) { 2140 long earliest = this.wal.getEarliestMemStoreSeqNum(encodedRegionName, familyName); 2141 // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will 2142 // give us a sequence id that is for sure flushed. We want edit replay to start after this 2143 // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id. 2144 long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1; 2145 regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.newBuilder() 2146 .setFamilyName(UnsafeByteOperations.unsafeWrap(familyName)).setSequenceId(csid).build()); 2147 } 2148 return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId()); 2149 } 2150 2151 ////////////////////////////////////////////////////////////////////////////// 2152 // HRegion maintenance. 2153 // 2154 // These methods are meant to be called periodically by the HRegionServer for 2155 // upkeep. 2156 ////////////////////////////////////////////////////////////////////////////// 2157 /** 2158 * Do preparation for pending compaction. 2159 * @throws IOException 2160 */ 2161 protected void doRegionCompactionPrep() throws IOException { 2162 } 2163 2164 /** 2165 * Synchronously compact all stores in the region. 2166 * <p>This operation could block for a long time, so don't call it from a 2167 * time-sensitive thread. 2168 * <p>Note that no locks are taken to prevent possible conflicts between 2169 * compaction and splitting activities. The regionserver does not normally compact 2170 * and split in parallel. However by calling this method you may introduce 2171 * unexpected and unhandled concurrency. Don't do this unless you know what 2172 * you are doing. 2173 * 2174 * @param majorCompaction True to force a major compaction regardless of thresholds 2175 * @throws IOException 2176 */ 2177 public void compact(boolean majorCompaction) throws IOException { 2178 if (majorCompaction) { 2179 stores.values().forEach(HStore::triggerMajorCompaction); 2180 } 2181 for (HStore s : stores.values()) { 2182 Optional<CompactionContext> compaction = s.requestCompaction(); 2183 if (compaction.isPresent()) { 2184 ThroughputController controller = null; 2185 if (rsServices != null) { 2186 controller = CompactionThroughputControllerFactory.create(rsServices, conf); 2187 } 2188 if (controller == null) { 2189 controller = NoLimitThroughputController.INSTANCE; 2190 } 2191 compact(compaction.get(), s, controller, null); 2192 } 2193 } 2194 } 2195 2196 /** 2197 * This is a helper function that compact all the stores synchronously. 2198 * <p> 2199 * It is used by utilities and testing 2200 */ 2201 public void compactStores() throws IOException { 2202 for (HStore s : stores.values()) { 2203 Optional<CompactionContext> compaction = s.requestCompaction(); 2204 if (compaction.isPresent()) { 2205 compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null); 2206 } 2207 } 2208 } 2209 2210 /** 2211 * This is a helper function that compact the given store. 2212 * <p> 2213 * It is used by utilities and testing 2214 */ 2215 void compactStore(byte[] family, ThroughputController throughputController) throws IOException { 2216 HStore s = getStore(family); 2217 Optional<CompactionContext> compaction = s.requestCompaction(); 2218 if (compaction.isPresent()) { 2219 compact(compaction.get(), s, throughputController, null); 2220 } 2221 } 2222 2223 /** 2224 * Called by compaction thread and after region is opened to compact the 2225 * HStores if necessary. 2226 * 2227 * <p>This operation could block for a long time, so don't call it from a 2228 * time-sensitive thread. 2229 * 2230 * Note that no locking is necessary at this level because compaction only 2231 * conflicts with a region split, and that cannot happen because the region 2232 * server does them sequentially and not in parallel. 2233 * 2234 * @param compaction Compaction details, obtained by requestCompaction() 2235 * @param throughputController 2236 * @return whether the compaction completed 2237 */ 2238 public boolean compact(CompactionContext compaction, HStore store, 2239 ThroughputController throughputController) throws IOException { 2240 return compact(compaction, store, throughputController, null); 2241 } 2242 2243 public boolean compact(CompactionContext compaction, HStore store, 2244 ThroughputController throughputController, User user) throws IOException { 2245 assert compaction != null && compaction.hasSelection(); 2246 assert !compaction.getRequest().getFiles().isEmpty(); 2247 if (this.closing.get() || this.closed.get()) { 2248 LOG.debug("Skipping compaction on " + this + " because closing/closed"); 2249 store.cancelRequestedCompaction(compaction); 2250 return false; 2251 } 2252 MonitoredTask status = null; 2253 boolean requestNeedsCancellation = true; 2254 /* 2255 * We are trying to remove / relax the region read lock for compaction. 2256 * Let's see what are the potential race conditions among the operations (user scan, 2257 * region split, region close and region bulk load). 2258 * 2259 * user scan ---> region read lock 2260 * region split --> region close first --> region write lock 2261 * region close --> region write lock 2262 * region bulk load --> region write lock 2263 * 2264 * read lock is compatible with read lock. ---> no problem with user scan/read 2265 * region bulk load does not cause problem for compaction (no consistency problem, store lock 2266 * will help the store file accounting). 2267 * They can run almost concurrently at the region level. 2268 * 2269 * The only remaining race condition is between the region close and compaction. 2270 * So we will evaluate, below, how region close intervenes with compaction if compaction does 2271 * not acquire region read lock. 2272 * 2273 * Here are the steps for compaction: 2274 * 1. obtain list of StoreFile's 2275 * 2. create StoreFileScanner's based on list from #1 2276 * 3. perform compaction and save resulting files under tmp dir 2277 * 4. swap in compacted files 2278 * 2279 * #1 is guarded by store lock. This patch does not change this --> no worse or better 2280 * For #2, we obtain smallest read point (for region) across all the Scanners (for both default 2281 * compactor and stripe compactor). 2282 * The read points are for user scans. Region keeps the read points for all currently open 2283 * user scanners. 2284 * Compaction needs to know the smallest read point so that during re-write of the hfiles, 2285 * it can remove the mvcc points for the cells if their mvccs are older than the smallest 2286 * since they are not needed anymore. 2287 * This will not conflict with compaction. 2288 * For #3, it can be performed in parallel to other operations. 2289 * For #4 bulk load and compaction don't conflict with each other on the region level 2290 * (for multi-family atomicy). 2291 * Region close and compaction are guarded pretty well by the 'writestate'. 2292 * In HRegion#doClose(), we have : 2293 * synchronized (writestate) { 2294 * // Disable compacting and flushing by background threads for this 2295 * // region. 2296 * canFlush = !writestate.readOnly; 2297 * writestate.writesEnabled = false; 2298 * LOG.debug("Closing " + this + ": disabling compactions & flushes"); 2299 * waitForFlushesAndCompactions(); 2300 * } 2301 * waitForFlushesAndCompactions() would wait for writestate.compacting to come down to 0. 2302 * and in HRegion.compact() 2303 * try { 2304 * synchronized (writestate) { 2305 * if (writestate.writesEnabled) { 2306 * wasStateSet = true; 2307 * ++writestate.compacting; 2308 * } else { 2309 * String msg = "NOT compacting region " + this + ". Writes disabled."; 2310 * LOG.info(msg); 2311 * status.abort(msg); 2312 * return false; 2313 * } 2314 * } 2315 * Also in compactor.performCompaction(): 2316 * check periodically to see if a system stop is requested 2317 * if (closeCheckInterval > 0) { 2318 * bytesWritten += len; 2319 * if (bytesWritten > closeCheckInterval) { 2320 * bytesWritten = 0; 2321 * if (!store.areWritesEnabled()) { 2322 * progress.cancel(); 2323 * return false; 2324 * } 2325 * } 2326 * } 2327 */ 2328 try { 2329 byte[] cf = Bytes.toBytes(store.getColumnFamilyName()); 2330 if (stores.get(cf) != store) { 2331 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this 2332 + " has been re-instantiated, cancel this compaction request. " 2333 + " It may be caused by the roll back of split transaction"); 2334 return false; 2335 } 2336 2337 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this); 2338 status.enableStatusJournal(false); 2339 if (this.closed.get()) { 2340 String msg = "Skipping compaction on " + this + " because closed"; 2341 LOG.debug(msg); 2342 status.abort(msg); 2343 return false; 2344 } 2345 boolean wasStateSet = false; 2346 try { 2347 synchronized (writestate) { 2348 if (writestate.writesEnabled) { 2349 wasStateSet = true; 2350 writestate.compacting.incrementAndGet(); 2351 } else { 2352 String msg = "NOT compacting region " + this + ". Writes disabled."; 2353 LOG.info(msg); 2354 status.abort(msg); 2355 return false; 2356 } 2357 } 2358 LOG.info("Starting compaction of {} in {}{}", store, this, 2359 (compaction.getRequest().isOffPeak()?" as an off-peak compaction":"")); 2360 doRegionCompactionPrep(); 2361 try { 2362 status.setStatus("Compacting store " + store); 2363 // We no longer need to cancel the request on the way out of this 2364 // method because Store#compact will clean up unconditionally 2365 requestNeedsCancellation = false; 2366 store.compact(compaction, throughputController, user); 2367 } catch (InterruptedIOException iioe) { 2368 String msg = "region " + this + " compaction interrupted"; 2369 LOG.info(msg, iioe); 2370 status.abort(msg); 2371 return false; 2372 } 2373 } finally { 2374 if (wasStateSet) { 2375 synchronized (writestate) { 2376 writestate.compacting.decrementAndGet(); 2377 if (writestate.compacting.get() <= 0) { 2378 writestate.notifyAll(); 2379 } 2380 } 2381 } 2382 } 2383 status.markComplete("Compaction complete"); 2384 return true; 2385 } finally { 2386 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction); 2387 if (status != null) { 2388 LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 2389 status.prettyPrintJournal()); 2390 status.cleanup(); 2391 } 2392 } 2393 } 2394 2395 /** 2396 * Flush the cache. 2397 * 2398 * <p>When this method is called the cache will be flushed unless: 2399 * <ol> 2400 * <li>the cache is empty</li> 2401 * <li>the region is closed.</li> 2402 * <li>a flush is already in progress</li> 2403 * <li>writes are disabled</li> 2404 * </ol> 2405 * 2406 * <p>This method may block for some time, so it should not be called from a 2407 * time-sensitive thread. 2408 * @param flushAllStores whether we want to force a flush of all stores 2409 * @return FlushResult indicating whether the flush was successful or not and if 2410 * the region needs compacting 2411 * 2412 * @throws IOException general io exceptions 2413 * because a snapshot was not properly persisted. 2414 */ 2415 // TODO HBASE-18905. We might have to expose a requestFlush API for CPs 2416 public FlushResult flush(boolean flushAllStores) throws IOException { 2417 return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY); 2418 } 2419 2420 public interface FlushResult { 2421 enum Result { 2422 FLUSHED_NO_COMPACTION_NEEDED, 2423 FLUSHED_COMPACTION_NEEDED, 2424 // Special case where a flush didn't run because there's nothing in the memstores. Used when 2425 // bulk loading to know when we can still load even if a flush didn't happen. 2426 CANNOT_FLUSH_MEMSTORE_EMPTY, 2427 CANNOT_FLUSH 2428 } 2429 2430 /** @return the detailed result code */ 2431 Result getResult(); 2432 2433 /** @return true if the memstores were flushed, else false */ 2434 boolean isFlushSucceeded(); 2435 2436 /** @return True if the flush requested a compaction, else false */ 2437 boolean isCompactionNeeded(); 2438 } 2439 2440 public FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker, 2441 FlushLifeCycleTracker tracker) throws IOException { 2442 List families = null; 2443 if (flushAllStores) { 2444 families = new ArrayList(); 2445 families.addAll(this.getTableDescriptor().getColumnFamilyNames()); 2446 } 2447 return this.flushcache(families, writeFlushRequestWalMarker, tracker); 2448 } 2449 2450 /** 2451 * Flush the cache. 2452 * 2453 * When this method is called the cache will be flushed unless: 2454 * <ol> 2455 * <li>the cache is empty</li> 2456 * <li>the region is closed.</li> 2457 * <li>a flush is already in progress</li> 2458 * <li>writes are disabled</li> 2459 * </ol> 2460 * 2461 * <p>This method may block for some time, so it should not be called from a 2462 * time-sensitive thread. 2463 * @param families stores of region to flush. 2464 * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL 2465 * @param tracker used to track the life cycle of this flush 2466 * @return whether the flush is success and whether the region needs compacting 2467 * 2468 * @throws IOException general io exceptions 2469 * @throws DroppedSnapshotException Thrown when replay of wal is required 2470 * because a Snapshot was not properly persisted. The region is put in closing mode, and the 2471 * caller MUST abort after this. 2472 */ 2473 public FlushResultImpl flushcache(List<byte[]> families, 2474 boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException { 2475 // fail-fast instead of waiting on the lock 2476 if (this.closing.get()) { 2477 String msg = "Skipping flush on " + this + " because closing"; 2478 LOG.debug(msg); 2479 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2480 } 2481 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this); 2482 status.enableStatusJournal(false); 2483 status.setStatus("Acquiring readlock on region"); 2484 // block waiting for the lock for flushing cache 2485 lock.readLock().lock(); 2486 boolean flushed = true; 2487 try { 2488 if (this.closed.get()) { 2489 String msg = "Skipping flush on " + this + " because closed"; 2490 LOG.debug(msg); 2491 status.abort(msg); 2492 flushed = false; 2493 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2494 } 2495 if (coprocessorHost != null) { 2496 status.setStatus("Running coprocessor pre-flush hooks"); 2497 coprocessorHost.preFlush(tracker); 2498 } 2499 // TODO: this should be managed within memstore with the snapshot, updated only after flush 2500 // successful 2501 if (numMutationsWithoutWAL.sum() > 0) { 2502 numMutationsWithoutWAL.reset(); 2503 dataInMemoryWithoutWAL.reset(); 2504 } 2505 synchronized (writestate) { 2506 if (!writestate.flushing && writestate.writesEnabled) { 2507 this.writestate.flushing = true; 2508 } else { 2509 String msg = "NOT flushing " + this + " as " + (writestate.flushing ? "already flushing" 2510 : "writes are not enabled"); 2511 LOG.debug(msg); 2512 status.abort(msg); 2513 flushed = false; 2514 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2515 } 2516 } 2517 2518 try { 2519 // The reason that we do not always use flushPolicy is, when the flush is 2520 // caused by logRoller, we should select stores which must be flushed 2521 // rather than could be flushed. 2522 Collection<HStore> specificStoresToFlush = null; 2523 if (families != null) { 2524 specificStoresToFlush = getSpecificStores(families); 2525 } else { 2526 specificStoresToFlush = flushPolicy.selectStoresToFlush(); 2527 } 2528 FlushResultImpl fs = 2529 internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker); 2530 2531 if (coprocessorHost != null) { 2532 status.setStatus("Running post-flush coprocessor hooks"); 2533 coprocessorHost.postFlush(tracker); 2534 } 2535 2536 if(fs.isFlushSucceeded()) { 2537 flushesQueued.reset(); 2538 } 2539 2540 status.markComplete("Flush successful " + fs.toString()); 2541 return fs; 2542 } finally { 2543 synchronized (writestate) { 2544 writestate.flushing = false; 2545 this.writestate.flushRequested = false; 2546 writestate.notifyAll(); 2547 } 2548 } 2549 } finally { 2550 lock.readLock().unlock(); 2551 if (flushed) { 2552 // Don't log this journal stuff if no flush -- confusing. 2553 LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 2554 status.prettyPrintJournal()); 2555 } 2556 status.cleanup(); 2557 } 2558 } 2559 2560 /** 2561 * get stores which matches the specified families 2562 * 2563 * @return the stores need to be flushed. 2564 */ 2565 private Collection<HStore> getSpecificStores(List<byte[]> families) { 2566 Collection<HStore> specificStoresToFlush = new ArrayList<>(); 2567 for (byte[] family : families) { 2568 specificStoresToFlush.add(stores.get(family)); 2569 } 2570 return specificStoresToFlush; 2571 } 2572 2573 /** 2574 * Should the store be flushed because it is old enough. 2575 * <p> 2576 * Every FlushPolicy should call this to determine whether a store is old enough to flush (except 2577 * that you always flush all stores). Otherwise the method will always 2578 * returns true which will make a lot of flush requests. 2579 */ 2580 boolean shouldFlushStore(HStore store) { 2581 long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), 2582 store.getColumnFamilyDescriptor().getName()) - 1; 2583 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) { 2584 if (LOG.isDebugEnabled()) { 2585 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " + 2586 getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest + 2587 " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint()); 2588 } 2589 return true; 2590 } 2591 if (this.flushCheckInterval <= 0) { 2592 return false; 2593 } 2594 long now = EnvironmentEdgeManager.currentTime(); 2595 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) { 2596 if (LOG.isDebugEnabled()) { 2597 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " + 2598 getRegionInfo().getEncodedName() + " because time of oldest edit=" + 2599 store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now); 2600 } 2601 return true; 2602 } 2603 return false; 2604 } 2605 2606 /** 2607 * Should the memstore be flushed now 2608 */ 2609 boolean shouldFlush(final StringBuilder whyFlush) { 2610 whyFlush.setLength(0); 2611 // This is a rough measure. 2612 if (this.maxFlushedSeqId > 0 2613 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) { 2614 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush"); 2615 return true; 2616 } 2617 long modifiedFlushCheckInterval = flushCheckInterval; 2618 if (getRegionInfo().getTable().isSystemTable() && 2619 getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) { 2620 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL; 2621 } 2622 if (modifiedFlushCheckInterval <= 0) { //disabled 2623 return false; 2624 } 2625 long now = EnvironmentEdgeManager.currentTime(); 2626 //if we flushed in the recent past, we don't need to do again now 2627 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) { 2628 return false; 2629 } 2630 //since we didn't flush in the recent past, flush now if certain conditions 2631 //are met. Return true on first such memstore hit. 2632 for (HStore s : stores.values()) { 2633 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) { 2634 // we have an old enough edit in the memstore, flush 2635 whyFlush.append(s.toString() + " has an old edit so flush to free WALs"); 2636 return true; 2637 } 2638 } 2639 return false; 2640 } 2641 2642 /** 2643 * Flushing all stores. 2644 * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker) 2645 */ 2646 private FlushResult internalFlushcache(MonitoredTask status) throws IOException { 2647 return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY); 2648 } 2649 2650 /** 2651 * Flushing given stores. 2652 * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker) 2653 */ 2654 private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status, 2655 boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException { 2656 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status, 2657 writeFlushWalMarker, tracker); 2658 } 2659 2660 /** 2661 * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the 2662 * memstore, all of which have also been written to the wal. We need to write those updates in the 2663 * memstore out to disk, while being able to process reads/writes as much as possible during the 2664 * flush operation. 2665 * <p> 2666 * This method may block for some time. Every time you call it, we up the regions sequence id even 2667 * if we don't flush; i.e. the returned region id will be at least one larger than the last edit 2668 * applied to this region. The returned id does not refer to an actual edit. The returned id can 2669 * be used for say installing a bulk loaded file just ahead of the last hfile that was the result 2670 * of this flush, etc. 2671 * @param wal Null if we're NOT to go via wal. 2672 * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file. 2673 * @param storesToFlush The list of stores to flush. 2674 * @return object describing the flush's state 2675 * @throws IOException general io exceptions 2676 * @throws DroppedSnapshotException Thrown when replay of WAL is required. 2677 */ 2678 protected FlushResultImpl internalFlushcache(WAL wal, long myseqid, 2679 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, 2680 FlushLifeCycleTracker tracker) throws IOException { 2681 PrepareFlushResult result = 2682 internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker); 2683 if (result.result == null) { 2684 return internalFlushCacheAndCommit(wal, status, result, storesToFlush); 2685 } else { 2686 return result.result; // early exit due to failure from prepare stage 2687 } 2688 } 2689 2690 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE", 2691 justification="FindBugs seems confused about trxId") 2692 protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid, 2693 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, 2694 FlushLifeCycleTracker tracker) throws IOException { 2695 if (this.rsServices != null && this.rsServices.isAborted()) { 2696 // Don't flush when server aborting, it's unsafe 2697 throw new IOException("Aborting flush because server is aborted..."); 2698 } 2699 final long startTime = EnvironmentEdgeManager.currentTime(); 2700 // If nothing to flush, return, but return with a valid unused sequenceId. 2701 // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a 2702 // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs 2703 // to no other that it can use to associate with the bulk load. Hence this little dance below 2704 // to go get one. 2705 if (this.memStoreSizing.getDataSize() <= 0) { 2706 // Take an update lock so no edits can come into memory just yet. 2707 this.updatesLock.writeLock().lock(); 2708 WriteEntry writeEntry = null; 2709 try { 2710 if (this.memStoreSizing.getDataSize() <= 0) { 2711 // Presume that if there are still no edits in the memstore, then there are no edits for 2712 // this region out in the WAL subsystem so no need to do any trickery clearing out 2713 // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for 2714 // sure just beyond the last appended region edit and not associated with any edit 2715 // (useful as marker when bulk loading, etc.). 2716 if (wal != null) { 2717 writeEntry = mvcc.begin(); 2718 long flushOpSeqId = writeEntry.getWriteNumber(); 2719 FlushResultImpl flushResult = 2720 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId, 2721 "Nothing to flush", writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker)); 2722 mvcc.completeAndWait(writeEntry); 2723 // Set to null so we don't complete it again down in finally block. 2724 writeEntry = null; 2725 return new PrepareFlushResult(flushResult, myseqid); 2726 } else { 2727 return new PrepareFlushResult(new FlushResultImpl( 2728 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid); 2729 } 2730 } 2731 } finally { 2732 if (writeEntry != null) { 2733 // If writeEntry is non-null, this operation failed; the mvcc transaction failed... 2734 // but complete it anyways so it doesn't block the mvcc queue. 2735 mvcc.complete(writeEntry); 2736 } 2737 this.updatesLock.writeLock().unlock(); 2738 } 2739 } 2740 logFatLineOnFlush(storesToFlush, myseqid); 2741 // Stop updates while we snapshot the memstore of all of these regions' stores. We only have 2742 // to do this for a moment. It is quick. We also set the memstore size to zero here before we 2743 // allow updates again so its value will represent the size of the updates received 2744 // during flush 2745 2746 // We have to take an update lock during snapshot, or else a write could end up in both snapshot 2747 // and memstore (makes it difficult to do atomic rows then) 2748 status.setStatus("Obtaining lock to block concurrent updates"); 2749 // block waiting for the lock for internal flush 2750 this.updatesLock.writeLock().lock(); 2751 status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName()); 2752 MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing(); 2753 2754 Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>(); 2755 for (HStore store : storesToFlush) { 2756 flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(), 2757 store.preFlushSeqIDEstimation()); 2758 } 2759 2760 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2761 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2762 TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2763 // The sequence id of this flush operation which is used to log FlushMarker and pass to 2764 // createFlushContext to use as the store file's sequence id. It can be in advance of edits 2765 // still in the memstore, edits that are in other column families yet to be flushed. 2766 long flushOpSeqId = HConstants.NO_SEQNUM; 2767 // The max flushed sequence id after this flush operation completes. All edits in memstore 2768 // will be in advance of this sequence id. 2769 long flushedSeqId = HConstants.NO_SEQNUM; 2770 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes(); 2771 try { 2772 if (wal != null) { 2773 Long earliestUnflushedSequenceIdForTheRegion = 2774 wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq); 2775 if (earliestUnflushedSequenceIdForTheRegion == null) { 2776 // This should never happen. This is how startCacheFlush signals flush cannot proceed. 2777 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing."; 2778 status.setStatus(msg); 2779 return new PrepareFlushResult( 2780 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false), 2781 myseqid); 2782 } 2783 flushOpSeqId = getNextSequenceId(wal); 2784 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit 2785 flushedSeqId = 2786 earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM? 2787 flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1; 2788 } else { 2789 // use the provided sequence Id as WAL is not being used for this flush. 2790 flushedSeqId = flushOpSeqId = myseqid; 2791 } 2792 2793 for (HStore s : storesToFlush) { 2794 storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), 2795 s.createFlushContext(flushOpSeqId, tracker)); 2796 // for writing stores to WAL 2797 committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); 2798 } 2799 2800 // write the snapshot start to WAL 2801 if (wal != null && !writestate.readOnly) { 2802 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH, 2803 getRegionInfo(), flushOpSeqId, committedFiles); 2804 // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH 2805 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, 2806 mvcc); 2807 } 2808 2809 // Prepare flush (take a snapshot) 2810 storeFlushCtxs.forEach((name, flush) -> { 2811 MemStoreSize snapshotSize = flush.prepare(); 2812 totalSizeOfFlushableStores.incMemStoreSize(snapshotSize); 2813 storeFlushableSize.put(name, snapshotSize); 2814 }); 2815 } catch (IOException ex) { 2816 doAbortFlushToWAL(wal, flushOpSeqId, committedFiles); 2817 throw ex; 2818 } finally { 2819 this.updatesLock.writeLock().unlock(); 2820 } 2821 String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " + 2822 "flushsize=" + totalSizeOfFlushableStores; 2823 status.setStatus(s); 2824 doSyncOfUnflushedWALChanges(wal, getRegionInfo()); 2825 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, 2826 flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores); 2827 } 2828 2829 /** 2830 * Utility method broken out of internalPrepareFlushCache so that method is smaller. 2831 */ 2832 private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) { 2833 if (!LOG.isInfoEnabled()) { 2834 return; 2835 } 2836 // Log a fat line detailing what is being flushed. 2837 StringBuilder perCfExtras = null; 2838 if (!isAllFamilies(storesToFlush)) { 2839 perCfExtras = new StringBuilder(); 2840 for (HStore store: storesToFlush) { 2841 MemStoreSize mss = store.getFlushableSize(); 2842 perCfExtras.append("; ").append(store.getColumnFamilyName()); 2843 perCfExtras.append("={dataSize=") 2844 .append(StringUtils.byteDesc(mss.getDataSize())); 2845 perCfExtras.append(", heapSize=") 2846 .append(StringUtils.byteDesc(mss.getHeapSize())); 2847 perCfExtras.append(", offHeapSize=") 2848 .append(StringUtils.byteDesc(mss.getOffHeapSize())); 2849 perCfExtras.append("}"); 2850 } 2851 } 2852 MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); 2853 LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " + 2854 storesToFlush.size() + "/" + stores.size() + " column families," + 2855 " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) + 2856 " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) + 2857 ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") + 2858 ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId)); 2859 } 2860 2861 private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId, 2862 final Map<byte[], List<Path>> committedFiles) { 2863 if (wal == null) return; 2864 try { 2865 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, 2866 getRegionInfo(), flushOpSeqId, committedFiles); 2867 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, 2868 mvcc); 2869 } catch (Throwable t) { 2870 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in " 2871 + " region {}", StringUtils.stringifyException(t), this); 2872 // ignore this since we will be aborting the RS with DSE. 2873 } 2874 // we have called wal.startCacheFlush(), now we have to abort it 2875 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes()); 2876 } 2877 2878 /** 2879 * Sync unflushed WAL changes. See HBASE-8208 for details 2880 */ 2881 private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri) 2882 throws IOException { 2883 if (wal == null) { 2884 return; 2885 } 2886 try { 2887 wal.sync(); // ensure that flush marker is sync'ed 2888 } catch (IOException ioe) { 2889 wal.abortCacheFlush(hri.getEncodedNameAsBytes()); 2890 throw ioe; 2891 } 2892 } 2893 2894 /** 2895 * @return True if passed Set is all families in the region. 2896 */ 2897 private boolean isAllFamilies(Collection<HStore> families) { 2898 return families == null || this.stores.size() == families.size(); 2899 } 2900 2901 /** 2902 * Writes a marker to WAL indicating a flush is requested but cannot be complete due to various 2903 * reasons. Ignores exceptions from WAL. Returns whether the write succeeded. 2904 * @param wal 2905 * @return whether WAL write was successful 2906 */ 2907 private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) { 2908 if (writeFlushWalMarker && wal != null && !writestate.readOnly) { 2909 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH, 2910 getRegionInfo(), -1, new TreeMap<>(Bytes.BYTES_COMPARATOR)); 2911 try { 2912 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, 2913 mvcc); 2914 return true; 2915 } catch (IOException e) { 2916 LOG.warn(getRegionInfo().getEncodedName() + " : " 2917 + "Received exception while trying to write the flush request to wal", e); 2918 } 2919 } 2920 return false; 2921 } 2922 2923 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", 2924 justification="Intentional; notify is about completed flush") 2925 protected FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status, 2926 PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException { 2927 // prepare flush context is carried via PrepareFlushResult 2928 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs; 2929 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles; 2930 long startTime = prepareResult.startTime; 2931 long flushOpSeqId = prepareResult.flushOpSeqId; 2932 long flushedSeqId = prepareResult.flushedSeqId; 2933 2934 String s = "Flushing stores of " + this; 2935 status.setStatus(s); 2936 if (LOG.isTraceEnabled()) LOG.trace(s); 2937 2938 // Any failure from here on out will be catastrophic requiring server 2939 // restart so wal content can be replayed and put back into the memstore. 2940 // Otherwise, the snapshot content while backed up in the wal, it will not 2941 // be part of the current running servers state. 2942 boolean compactionRequested = false; 2943 long flushedOutputFileSize = 0; 2944 try { 2945 // A. Flush memstore to all the HStores. 2946 // Keep running vector of all store files that includes both old and the 2947 // just-made new flush store file. The new flushed file is still in the 2948 // tmp directory. 2949 2950 for (StoreFlushContext flush : storeFlushCtxs.values()) { 2951 flush.flushCache(status); 2952 } 2953 2954 // Switch snapshot (in memstore) -> new hfile (thus causing 2955 // all the store scanners to reset/reseek). 2956 for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) { 2957 StoreFlushContext sfc = flushEntry.getValue(); 2958 boolean needsCompaction = sfc.commit(status); 2959 if (needsCompaction) { 2960 compactionRequested = true; 2961 } 2962 byte[] storeName = flushEntry.getKey(); 2963 List<Path> storeCommittedFiles = sfc.getCommittedFiles(); 2964 committedFiles.put(storeName, storeCommittedFiles); 2965 // Flush committed no files, indicating flush is empty or flush was canceled 2966 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) { 2967 MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName); 2968 prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize); 2969 } 2970 flushedOutputFileSize += sfc.getOutputFileSize(); 2971 } 2972 storeFlushCtxs.clear(); 2973 2974 // Set down the memstore size by amount of flush. 2975 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); 2976 this.decrMemStoreSize(mss); 2977 2978 // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled. 2979 // During startup, quota manager may not be initialized yet. 2980 if (rsServices != null) { 2981 RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager(); 2982 if (quotaManager != null) { 2983 quotaManager.getRegionSizeStore().incrementRegionSize( 2984 this.getRegionInfo(), flushedOutputFileSize); 2985 } 2986 } 2987 2988 if (wal != null) { 2989 // write flush marker to WAL. If fail, we should throw DroppedSnapshotException 2990 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH, 2991 getRegionInfo(), flushOpSeqId, committedFiles); 2992 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, 2993 mvcc); 2994 } 2995 } catch (Throwable t) { 2996 // An exception here means that the snapshot was not persisted. 2997 // The wal needs to be replayed so its content is restored to memstore. 2998 // Currently, only a server restart will do this. 2999 // We used to only catch IOEs but its possible that we'd get other 3000 // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch 3001 // all and sundry. 3002 if (wal != null) { 3003 try { 3004 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, 3005 getRegionInfo(), flushOpSeqId, committedFiles); 3006 WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc); 3007 } catch (Throwable ex) { 3008 LOG.warn(getRegionInfo().getEncodedName() + " : " 3009 + "failed writing ABORT_FLUSH marker to WAL", ex); 3010 // ignore this since we will be aborting the RS with DSE. 3011 } 3012 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes()); 3013 } 3014 DroppedSnapshotException dse = new DroppedSnapshotException("region: " + 3015 Bytes.toStringBinary(getRegionInfo().getRegionName()), t); 3016 status.abort("Flush failed: " + StringUtils.stringifyException(t)); 3017 3018 // Callers for flushcache() should catch DroppedSnapshotException and abort the region server. 3019 // However, since we may have the region read lock, we cannot call close(true) here since 3020 // we cannot promote to a write lock. Instead we are setting closing so that all other region 3021 // operations except for close will be rejected. 3022 this.closing.set(true); 3023 3024 if (rsServices != null) { 3025 // This is a safeguard against the case where the caller fails to explicitly handle aborting 3026 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse); 3027 } 3028 3029 throw dse; 3030 } 3031 3032 // If we get to here, the HStores have been written. 3033 if (wal != null) { 3034 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId); 3035 } 3036 3037 // Record latest flush time 3038 for (HStore store: storesToFlush) { 3039 this.lastStoreFlushTimeMap.put(store, startTime); 3040 } 3041 3042 this.maxFlushedSeqId = flushedSeqId; 3043 this.lastFlushOpSeqId = flushOpSeqId; 3044 3045 // C. Finally notify anyone waiting on memstore to clear: 3046 // e.g. checkResources(). 3047 synchronized (this) { 3048 notifyAll(); // FindBugs NN_NAKED_NOTIFY 3049 } 3050 3051 long time = EnvironmentEdgeManager.currentTime() - startTime; 3052 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); 3053 long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize(); 3054 String msg = "Finished flush of" 3055 + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" + mss.getDataSize() 3056 + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/" + mss.getHeapSize() 3057 + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/" + memstoresize 3058 + " for " + this.getRegionInfo().getEncodedName() + " in " + time + "ms, sequenceid=" 3059 + flushOpSeqId + ", compaction requested=" + compactionRequested 3060 + ((wal == null) ? "; wal=null" : ""); 3061 LOG.info(msg); 3062 status.setStatus(msg); 3063 3064 if (rsServices != null && rsServices.getMetrics() != null) { 3065 rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(), 3066 time, 3067 mss.getDataSize(), flushedOutputFileSize); 3068 } 3069 3070 return new FlushResultImpl(compactionRequested ? 3071 FlushResult.Result.FLUSHED_COMPACTION_NEEDED : 3072 FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId); 3073 } 3074 3075 /** 3076 * Method to safely get the next sequence number. 3077 * @return Next sequence number unassociated with any actual edit. 3078 * @throws IOException 3079 */ 3080 protected long getNextSequenceId(final WAL wal) throws IOException { 3081 WriteEntry we = mvcc.begin(); 3082 mvcc.completeAndWait(we); 3083 return we.getWriteNumber(); 3084 } 3085 3086 ////////////////////////////////////////////////////////////////////////////// 3087 // get() methods for client use. 3088 ////////////////////////////////////////////////////////////////////////////// 3089 3090 @Override 3091 public RegionScannerImpl getScanner(Scan scan) throws IOException { 3092 return getScanner(scan, null); 3093 } 3094 3095 @Override 3096 public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners) 3097 throws IOException { 3098 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE); 3099 } 3100 3101 private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners, 3102 long nonceGroup, long nonce) throws IOException { 3103 startRegionOperation(Operation.SCAN); 3104 try { 3105 // Verify families are all valid 3106 if (!scan.hasFamilies()) { 3107 // Adding all families to scanner 3108 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) { 3109 scan.addFamily(family); 3110 } 3111 } else { 3112 for (byte[] family : scan.getFamilyMap().keySet()) { 3113 checkFamily(family); 3114 } 3115 } 3116 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce); 3117 } finally { 3118 closeRegionOperation(Operation.SCAN); 3119 } 3120 } 3121 3122 protected RegionScanner instantiateRegionScanner(Scan scan, 3123 List<KeyValueScanner> additionalScanners) throws IOException { 3124 return instantiateRegionScanner(scan, additionalScanners, HConstants.NO_NONCE, 3125 HConstants.NO_NONCE); 3126 } 3127 3128 protected RegionScannerImpl instantiateRegionScanner(Scan scan, 3129 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException { 3130 if (scan.isReversed()) { 3131 if (scan.getFilter() != null) { 3132 scan.getFilter().setReversed(true); 3133 } 3134 return new ReversedRegionScannerImpl(scan, additionalScanners, this); 3135 } 3136 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce); 3137 } 3138 3139 /** 3140 * Prepare a delete for a row mutation processor 3141 * @param delete The passed delete is modified by this method. WARNING! 3142 * @throws IOException 3143 */ 3144 public void prepareDelete(Delete delete) throws IOException { 3145 // Check to see if this is a deleteRow insert 3146 if(delete.getFamilyCellMap().isEmpty()){ 3147 for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){ 3148 // Don't eat the timestamp 3149 delete.addFamily(family, delete.getTimestamp()); 3150 } 3151 } else { 3152 for(byte [] family : delete.getFamilyCellMap().keySet()) { 3153 if(family == null) { 3154 throw new NoSuchColumnFamilyException("Empty family is invalid"); 3155 } 3156 checkFamily(family); 3157 } 3158 } 3159 } 3160 3161 @Override 3162 public void delete(Delete delete) throws IOException { 3163 checkReadOnly(); 3164 checkResources(); 3165 startRegionOperation(Operation.DELETE); 3166 try { 3167 // All edits for the given row (across all column families) must happen atomically. 3168 doBatchMutate(delete); 3169 } finally { 3170 closeRegionOperation(Operation.DELETE); 3171 } 3172 } 3173 3174 /** 3175 * Row needed by below method. 3176 */ 3177 private static final byte [] FOR_UNIT_TESTS_ONLY = Bytes.toBytes("ForUnitTestsOnly"); 3178 3179 /** 3180 * This is used only by unit tests. Not required to be a public API. 3181 * @param familyMap map of family to edits for the given family. 3182 * @throws IOException 3183 */ 3184 void delete(NavigableMap<byte[], List<Cell>> familyMap, 3185 Durability durability) throws IOException { 3186 Delete delete = new Delete(FOR_UNIT_TESTS_ONLY); 3187 delete.setFamilyCellMap(familyMap); 3188 delete.setDurability(durability); 3189 doBatchMutate(delete); 3190 } 3191 3192 /** 3193 * Set up correct timestamps in the KVs in Delete object. 3194 * <p>Caller should have the row and region locks. 3195 * @param mutation 3196 * @param familyMap 3197 * @param byteNow 3198 * @throws IOException 3199 */ 3200 public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap, 3201 byte[] byteNow) throws IOException { 3202 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) { 3203 3204 byte[] family = e.getKey(); 3205 List<Cell> cells = e.getValue(); 3206 assert cells instanceof RandomAccess; 3207 3208 Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR); 3209 int listSize = cells.size(); 3210 for (int i=0; i < listSize; i++) { 3211 Cell cell = cells.get(i); 3212 // Check if time is LATEST, change to time of most recent addition if so 3213 // This is expensive. 3214 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP 3215 && PrivateCellUtil.isDeleteType(cell)) { 3216 byte[] qual = CellUtil.cloneQualifier(cell); 3217 3218 Integer count = kvCount.get(qual); 3219 if (count == null) { 3220 kvCount.put(qual, 1); 3221 } else { 3222 kvCount.put(qual, count + 1); 3223 } 3224 count = kvCount.get(qual); 3225 3226 Get get = new Get(CellUtil.cloneRow(cell)); 3227 get.setMaxVersions(count); 3228 get.addColumn(family, qual); 3229 if (coprocessorHost != null) { 3230 if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell, 3231 byteNow, get)) { 3232 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow); 3233 } 3234 } else { 3235 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow); 3236 } 3237 } else { 3238 PrivateCellUtil.updateLatestStamp(cell, byteNow); 3239 } 3240 } 3241 } 3242 } 3243 3244 void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow) 3245 throws IOException { 3246 List<Cell> result = get(get, false); 3247 3248 if (result.size() < count) { 3249 // Nothing to delete 3250 PrivateCellUtil.updateLatestStamp(cell, byteNow); 3251 return; 3252 } 3253 if (result.size() > count) { 3254 throw new RuntimeException("Unexpected size: " + result.size()); 3255 } 3256 Cell getCell = result.get(count - 1); 3257 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp()); 3258 } 3259 3260 @Override 3261 public void put(Put put) throws IOException { 3262 checkReadOnly(); 3263 3264 // Do a rough check that we have resources to accept a write. The check is 3265 // 'rough' in that between the resource check and the call to obtain a 3266 // read lock, resources may run out. For now, the thought is that this 3267 // will be extremely rare; we'll deal with it when it happens. 3268 checkResources(); 3269 startRegionOperation(Operation.PUT); 3270 try { 3271 // All edits for the given row (across all column families) must happen atomically. 3272 doBatchMutate(put); 3273 } finally { 3274 closeRegionOperation(Operation.PUT); 3275 } 3276 } 3277 3278 /** 3279 * Class that tracks the progress of a batch operations, accumulating status codes and tracking 3280 * the index at which processing is proceeding. These batch operations may get split into 3281 * mini-batches for processing. 3282 */ 3283 private abstract static class BatchOperation<T> { 3284 protected final T[] operations; 3285 protected final OperationStatus[] retCodeDetails; 3286 protected final WALEdit[] walEditsFromCoprocessors; 3287 // reference family cell maps directly so coprocessors can mutate them if desired 3288 protected final Map<byte[], List<Cell>>[] familyCellMaps; 3289 // For Increment/Append operations 3290 protected final Result[] results; 3291 // For nonce operations 3292 protected final boolean[] canProceed; 3293 3294 protected final HRegion region; 3295 protected int nextIndexToProcess = 0; 3296 protected final ObservedExceptionsInBatch observedExceptions; 3297 //Durability of the batch (highest durability of all operations) 3298 protected Durability durability; 3299 protected boolean atomic = false; 3300 3301 public BatchOperation(final HRegion region, T[] operations) { 3302 this.operations = operations; 3303 this.retCodeDetails = new OperationStatus[operations.length]; 3304 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN); 3305 this.walEditsFromCoprocessors = new WALEdit[operations.length]; 3306 familyCellMaps = new Map[operations.length]; 3307 this.results = new Result[operations.length]; 3308 this.canProceed = new boolean[operations.length]; 3309 3310 this.region = region; 3311 observedExceptions = new ObservedExceptionsInBatch(); 3312 durability = Durability.USE_DEFAULT; 3313 } 3314 3315 /** 3316 * Visitor interface for batch operations 3317 */ 3318 @FunctionalInterface 3319 public interface Visitor { 3320 /** 3321 * @param index operation index 3322 * @return If true continue visiting remaining entries, break otherwise 3323 */ 3324 boolean visit(int index) throws IOException; 3325 } 3326 3327 /** 3328 * Helper method for visiting pending/ all batch operations 3329 */ 3330 public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor) 3331 throws IOException { 3332 assert lastIndexExclusive <= this.size(); 3333 for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) { 3334 if (!pendingOnly || isOperationPending(i)) { 3335 if (!visitor.visit(i)) { 3336 break; 3337 } 3338 } 3339 } 3340 } 3341 3342 public abstract Mutation getMutation(int index); 3343 3344 public abstract long getNonceGroup(int index); 3345 3346 public abstract long getNonce(int index); 3347 3348 /** 3349 * This method is potentially expensive and useful mostly for non-replay CP path. 3350 */ 3351 public abstract Mutation[] getMutationsForCoprocs(); 3352 3353 public abstract boolean isInReplay(); 3354 3355 public abstract long getOrigLogSeqNum(); 3356 3357 public abstract void startRegionOperation() throws IOException; 3358 3359 public abstract void closeRegionOperation() throws IOException; 3360 3361 /** 3362 * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs 3363 * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This 3364 * is intended to operate on entire batch and will be called from outside of class to check 3365 * and prepare batch. This can be implemented by calling helper method 3366 * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations. 3367 */ 3368 public abstract void checkAndPrepare() throws IOException; 3369 3370 /** 3371 * Implement any Put request specific check and prepare logic here. Please refer to 3372 * {@link #checkAndPrepareMutation(Mutation, long)} for how its used. 3373 */ 3374 protected abstract void checkAndPreparePut(final Put p) throws IOException; 3375 3376 /** 3377 * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell 3378 * count, tags and timestamp for all cells of all operations in a mini-batch. 3379 */ 3380 public abstract void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> 3381 miniBatchOp, long timestamp, final List<RowLock> acquiredRowLocks) throws IOException; 3382 3383 /** 3384 * Write mini-batch operations to MemStore 3385 */ 3386 public abstract WriteEntry writeMiniBatchOperationsToMemStore( 3387 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 3388 throws IOException; 3389 3390 protected void writeMiniBatchOperationsToMemStore( 3391 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber) 3392 throws IOException { 3393 MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing(); 3394 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 3395 // We need to update the sequence id for following reasons. 3396 // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id. 3397 // 2) If no WAL, FSWALEntry won't be used 3398 // we use durability of the original mutation for the mutation passed by CP. 3399 if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) { 3400 region.updateSequenceId(familyCellMaps[index].values(), writeNumber); 3401 } 3402 applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting); 3403 return true; 3404 }); 3405 // update memStore size 3406 region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(), 3407 memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount()); 3408 } 3409 3410 public boolean isDone() { 3411 return nextIndexToProcess == operations.length; 3412 } 3413 3414 public int size() { 3415 return operations.length; 3416 } 3417 3418 public boolean isOperationPending(int index) { 3419 return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN; 3420 } 3421 3422 public List<UUID> getClusterIds() { 3423 assert size() != 0; 3424 return getMutation(0).getClusterIds(); 3425 } 3426 3427 boolean isAtomic() { 3428 return atomic; 3429 } 3430 3431 /** 3432 * Helper method that checks and prepares only one mutation. This can be used to implement 3433 * {@link #checkAndPrepare()} for entire Batch. 3434 * NOTE: As CP prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations, 3435 * this method should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks 3436 * are run for the mutation 3437 */ 3438 protected void checkAndPrepareMutation(Mutation mutation, final long timestamp) 3439 throws IOException { 3440 region.checkRow(mutation.getRow(), "batchMutate"); 3441 if (mutation instanceof Put) { 3442 // Check the families in the put. If bad, skip this one. 3443 checkAndPreparePut((Put) mutation); 3444 region.checkTimestamps(mutation.getFamilyCellMap(), timestamp); 3445 } else if (mutation instanceof Delete) { 3446 region.prepareDelete((Delete) mutation); 3447 } else if (mutation instanceof Increment || mutation instanceof Append) { 3448 region.checkFamilies(mutation.getFamilyCellMap().keySet()); 3449 } 3450 } 3451 3452 protected void checkAndPrepareMutation(int index, long timestamp) throws IOException { 3453 Mutation mutation = getMutation(index); 3454 try { 3455 this.checkAndPrepareMutation(mutation, timestamp); 3456 3457 if (mutation instanceof Put || mutation instanceof Delete) { 3458 // store the family map reference to allow for mutations 3459 familyCellMaps[index] = mutation.getFamilyCellMap(); 3460 } 3461 3462 // store durability for the batch (highest durability of all operations in the batch) 3463 Durability tmpDur = region.getEffectiveDurability(mutation.getDurability()); 3464 if (tmpDur.ordinal() > durability.ordinal()) { 3465 durability = tmpDur; 3466 } 3467 } catch (NoSuchColumnFamilyException nscfe) { 3468 final String msg = "No such column family in batch mutation in region " + this; 3469 if (observedExceptions.hasSeenNoSuchFamily()) { 3470 LOG.warn(msg + nscfe.getMessage()); 3471 } else { 3472 LOG.warn(msg, nscfe); 3473 observedExceptions.sawNoSuchFamily(); 3474 } 3475 retCodeDetails[index] = new OperationStatus( 3476 OperationStatusCode.BAD_FAMILY, nscfe.getMessage()); 3477 if (isAtomic()) { // fail, atomic means all or none 3478 throw nscfe; 3479 } 3480 } catch (FailedSanityCheckException fsce) { 3481 final String msg = "Batch Mutation did not pass sanity check in region " + this; 3482 if (observedExceptions.hasSeenFailedSanityCheck()) { 3483 LOG.warn(msg + fsce.getMessage()); 3484 } else { 3485 LOG.warn(msg, fsce); 3486 observedExceptions.sawFailedSanityCheck(); 3487 } 3488 retCodeDetails[index] = new OperationStatus( 3489 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage()); 3490 if (isAtomic()) { 3491 throw fsce; 3492 } 3493 } catch (WrongRegionException we) { 3494 final String msg = "Batch mutation had a row that does not belong to this region " + this; 3495 if (observedExceptions.hasSeenWrongRegion()) { 3496 LOG.warn(msg + we.getMessage()); 3497 } else { 3498 LOG.warn(msg, we); 3499 observedExceptions.sawWrongRegion(); 3500 } 3501 retCodeDetails[index] = new OperationStatus( 3502 OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage()); 3503 if (isAtomic()) { 3504 throw we; 3505 } 3506 } 3507 } 3508 3509 /** 3510 * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which 3511 * a row lock can be acquired. All mutations with locked rows are considered to be 3512 * In-progress operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch 3513 * is window over {@link BatchOperation} and contains contiguous pending operations. 3514 * 3515 * @param acquiredRowLocks keeps track of rowLocks acquired. 3516 */ 3517 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch( 3518 List<RowLock> acquiredRowLocks) throws IOException { 3519 int readyToWriteCount = 0; 3520 int lastIndexExclusive = 0; 3521 RowLock prevRowLock = null; 3522 for (; lastIndexExclusive < size(); lastIndexExclusive++) { 3523 // It reaches the miniBatchSize, stop here and process the miniBatch 3524 // This only applies to non-atomic batch operations. 3525 if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) { 3526 break; 3527 } 3528 3529 if (!isOperationPending(lastIndexExclusive)) { 3530 continue; 3531 } 3532 3533 // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting 3534 // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation 3535 // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't 3536 // pass the isOperationPending check 3537 Map<byte[], List<Cell>> curFamilyCellMap = 3538 getMutation(lastIndexExclusive).getFamilyCellMap(); 3539 try { 3540 // start the protector before acquiring row lock considering performance, and will finish 3541 // it when encountering exception 3542 region.storeHotnessProtector.start(curFamilyCellMap); 3543 } catch (RegionTooBusyException rtbe) { 3544 region.storeHotnessProtector.finish(curFamilyCellMap); 3545 if (isAtomic()) { 3546 throw rtbe; 3547 } 3548 retCodeDetails[lastIndexExclusive] = 3549 new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage()); 3550 continue; 3551 } 3552 3553 Mutation mutation = getMutation(lastIndexExclusive); 3554 // If we haven't got any rows in our batch, we should block to get the next one. 3555 RowLock rowLock = null; 3556 boolean throwException = false; 3557 try { 3558 // if atomic then get exclusive lock, else shared lock 3559 rowLock = region.getRowLockInternal(mutation.getRow(), !isAtomic(), prevRowLock); 3560 } catch (TimeoutIOException | InterruptedIOException e) { 3561 // NOTE: We will retry when other exceptions, but we should stop if we receive 3562 // TimeoutIOException or InterruptedIOException as operation has timed out or 3563 // interrupted respectively. 3564 throwException = true; 3565 throw e; 3566 } catch (IOException ioe) { 3567 LOG.warn("Failed getting lock, row={}, in region {}", 3568 Bytes.toStringBinary(mutation.getRow()), this, ioe); 3569 if (isAtomic()) { // fail, atomic means all or none 3570 throwException = true; 3571 throw ioe; 3572 } 3573 } catch (Throwable throwable) { 3574 throwException = true; 3575 throw throwable; 3576 } finally { 3577 if (throwException) { 3578 region.storeHotnessProtector.finish(curFamilyCellMap); 3579 } 3580 } 3581 if (rowLock == null) { 3582 // We failed to grab another lock 3583 if (isAtomic()) { 3584 region.storeHotnessProtector.finish(curFamilyCellMap); 3585 throw new IOException("Can't apply all operations atomically!"); 3586 } 3587 break; // Stop acquiring more rows for this batch 3588 } else { 3589 if (rowLock != prevRowLock) { 3590 // It is a different row now, add this to the acquiredRowLocks and 3591 // set prevRowLock to the new returned rowLock 3592 acquiredRowLocks.add(rowLock); 3593 prevRowLock = rowLock; 3594 } 3595 } 3596 3597 readyToWriteCount++; 3598 } 3599 return createMiniBatch(lastIndexExclusive, readyToWriteCount); 3600 } 3601 3602 protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive, 3603 final int readyToWriteCount) { 3604 return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails, 3605 walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount); 3606 } 3607 3608 /** 3609 * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are 3610 * present, they are merged to result WALEdit. 3611 */ 3612 public List<Pair<NonceKey, WALEdit>> buildWALEdits( 3613 final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException { 3614 List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>(); 3615 3616 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() { 3617 private Pair<NonceKey, WALEdit> curWALEditForNonce; 3618 3619 @Override 3620 public boolean visit(int index) throws IOException { 3621 Mutation m = getMutation(index); 3622 // we use durability of the original mutation for the mutation passed by CP. 3623 if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) { 3624 region.recordMutationWithoutWal(m.getFamilyCellMap()); 3625 return true; 3626 } 3627 3628 // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each. 3629 // Given how nonce keys are originally written, these should be contiguous. 3630 // They don't have to be, it will still work, just write more WALEdits than needed. 3631 long nonceGroup = getNonceGroup(index); 3632 long nonce = getNonce(index); 3633 if (curWALEditForNonce == null || 3634 curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup || 3635 curWALEditForNonce.getFirst().getNonce() != nonce) { 3636 curWALEditForNonce = new Pair<>(new NonceKey(nonceGroup, nonce), 3637 new WALEdit(miniBatchOp.getCellCount(), isInReplay())); 3638 walEdits.add(curWALEditForNonce); 3639 } 3640 WALEdit walEdit = curWALEditForNonce.getSecond(); 3641 3642 // Add WAL edits from CPs. 3643 WALEdit fromCP = walEditsFromCoprocessors[index]; 3644 if (fromCP != null) { 3645 for (Cell cell : fromCP.getCells()) { 3646 walEdit.add(cell); 3647 } 3648 } 3649 walEdit.add(familyCellMaps[index]); 3650 3651 return true; 3652 } 3653 }); 3654 return walEdits; 3655 } 3656 3657 /** 3658 * This method completes mini-batch operations by calling postBatchMutate() CP hook (if 3659 * required) and completing mvcc. 3660 */ 3661 public void completeMiniBatchOperations( 3662 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 3663 throws IOException { 3664 if (writeEntry != null) { 3665 region.mvcc.completeAndWait(writeEntry); 3666 } 3667 } 3668 3669 public void doPostOpCleanupForMiniBatch( 3670 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit, 3671 boolean success) throws IOException { 3672 doFinishHotnessProtector(miniBatchOp); 3673 } 3674 3675 private void doFinishHotnessProtector( 3676 final MiniBatchOperationInProgress<Mutation> miniBatchOp) { 3677 // check and return if the protector is not enabled 3678 if (!region.storeHotnessProtector.isEnable()) { 3679 return; 3680 } 3681 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception. 3682 // This case was handled. 3683 if (miniBatchOp == null) { 3684 return; 3685 } 3686 3687 final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive(); 3688 3689 for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) { 3690 switch (retCodeDetails[i].getOperationStatusCode()) { 3691 case SUCCESS: 3692 case FAILURE: 3693 region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap()); 3694 break; 3695 default: 3696 // do nothing 3697 // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the 3698 // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start 3699 break; 3700 } 3701 } 3702 } 3703 3704 /** 3705 * Atomically apply the given map of family->edits to the memstore. 3706 * This handles the consistency control on its own, but the caller 3707 * should already have locked updatesLock.readLock(). This also does 3708 * <b>not</b> check the families for validity. 3709 * 3710 * @param familyMap Map of Cells by family 3711 */ 3712 protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap, 3713 MemStoreSizing memstoreAccounting) throws IOException { 3714 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) { 3715 byte[] family = e.getKey(); 3716 List<Cell> cells = e.getValue(); 3717 assert cells instanceof RandomAccess; 3718 region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting); 3719 } 3720 } 3721 } 3722 3723 3724 /** 3725 * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most 3726 * of the logic is same. 3727 */ 3728 static class MutationBatchOperation extends BatchOperation<Mutation> { 3729 private long nonceGroup; 3730 private long nonce; 3731 public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic, 3732 long nonceGroup, long nonce) { 3733 super(region, operations); 3734 this.atomic = atomic; 3735 this.nonceGroup = nonceGroup; 3736 this.nonce = nonce; 3737 } 3738 3739 @Override 3740 public Mutation getMutation(int index) { 3741 return this.operations[index]; 3742 } 3743 3744 @Override 3745 public long getNonceGroup(int index) { 3746 return nonceGroup; 3747 } 3748 3749 @Override 3750 public long getNonce(int index) { 3751 return nonce; 3752 } 3753 3754 @Override 3755 public Mutation[] getMutationsForCoprocs() { 3756 return this.operations; 3757 } 3758 3759 @Override 3760 public boolean isInReplay() { 3761 return false; 3762 } 3763 3764 @Override 3765 public long getOrigLogSeqNum() { 3766 return SequenceId.NO_SEQUENCE_ID; 3767 } 3768 3769 @Override 3770 public void startRegionOperation() throws IOException { 3771 region.startRegionOperation(Operation.BATCH_MUTATE); 3772 } 3773 3774 @Override 3775 public void closeRegionOperation() throws IOException { 3776 region.closeRegionOperation(Operation.BATCH_MUTATE); 3777 } 3778 3779 @Override 3780 public void checkAndPreparePut(Put p) throws IOException { 3781 region.checkFamilies(p.getFamilyCellMap().keySet()); 3782 } 3783 3784 @Override 3785 public void checkAndPrepare() throws IOException { 3786 // index 0: puts, index 1: deletes, index 2: increments, index 3: append 3787 final int[] metrics = {0, 0, 0, 0}; 3788 3789 visitBatchOperations(true, this.size(), new Visitor() { 3790 private long now = EnvironmentEdgeManager.currentTime(); 3791 private WALEdit walEdit; 3792 @Override 3793 public boolean visit(int index) throws IOException { 3794 // Run coprocessor pre hook outside of locks to avoid deadlock 3795 if (region.coprocessorHost != null) { 3796 if (walEdit == null) { 3797 walEdit = new WALEdit(); 3798 } 3799 callPreMutateCPHook(index, walEdit, metrics); 3800 if (!walEdit.isEmpty()) { 3801 walEditsFromCoprocessors[index] = walEdit; 3802 walEdit = null; 3803 } 3804 } 3805 if (isOperationPending(index)) { 3806 // TODO: Currently validation is done with current time before acquiring locks and 3807 // updates are done with different timestamps after acquiring locks. This behavior is 3808 // inherited from the code prior to this change. Can this be changed? 3809 checkAndPrepareMutation(index, now); 3810 } 3811 return true; 3812 } 3813 }); 3814 3815 // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in 3816 // normal processing. 3817 // Update metrics in same way as it is done when we go the normal processing route (we now 3818 // update general metrics though a Coprocessor did the work). 3819 if (region.metricsRegion != null) { 3820 if (metrics[0] > 0) { 3821 // There were some Puts in the batch. 3822 region.metricsRegion.updatePut(); 3823 } 3824 if (metrics[1] > 0) { 3825 // There were some Deletes in the batch. 3826 region.metricsRegion.updateDelete(); 3827 } 3828 if (metrics[2] > 0) { 3829 // There were some Increment in the batch. 3830 region.metricsRegion.updateIncrement(); 3831 } 3832 if (metrics[3] > 0) { 3833 // There were some Append in the batch. 3834 region.metricsRegion.updateAppend(); 3835 } 3836 } 3837 } 3838 3839 @Override 3840 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp, 3841 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException { 3842 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 3843 Mutation mutation = getMutation(index); 3844 if (mutation instanceof Put) { 3845 HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp)); 3846 miniBatchOp.incrementNumOfPuts(); 3847 } else if (mutation instanceof Delete) { 3848 region.prepareDeleteTimestamps(mutation, familyCellMaps[index], 3849 Bytes.toBytes(timestamp)); 3850 miniBatchOp.incrementNumOfDeletes(); 3851 } else if (mutation instanceof Increment || mutation instanceof Append) { 3852 boolean returnResults; 3853 if (mutation instanceof Increment) { 3854 returnResults = ((Increment) mutation).isReturnResults(); 3855 } else { 3856 returnResults = ((Append) mutation).isReturnResults(); 3857 } 3858 3859 // For nonce operations 3860 canProceed[index] = startNonceOperation(nonceGroup, nonce); 3861 if (!canProceed[index]) { 3862 Result result; 3863 if (returnResults) { 3864 // convert duplicate increment/append to get 3865 List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce); 3866 result = Result.create(results); 3867 } else { 3868 result = Result.EMPTY_RESULT; 3869 } 3870 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 3871 return true; 3872 } 3873 3874 Result result = null; 3875 if (region.coprocessorHost != null) { 3876 if (mutation instanceof Increment) { 3877 result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation); 3878 } else { 3879 result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation); 3880 } 3881 } 3882 if (result != null) { 3883 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, 3884 returnResults ? result : Result.EMPTY_RESULT); 3885 return true; 3886 } 3887 3888 List<Cell> results = returnResults ? new ArrayList<>(mutation.size()) : null; 3889 familyCellMaps[index] = reckonDeltas(mutation, results, timestamp); 3890 this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT; 3891 3892 if (mutation instanceof Increment) { 3893 miniBatchOp.incrementNumOfIncrements(); 3894 } else { 3895 miniBatchOp.incrementNumOfAppends(); 3896 } 3897 } 3898 region.rewriteCellTags(familyCellMaps[index], mutation); 3899 3900 // update cell count 3901 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) { 3902 for (List<Cell> cells : mutation.getFamilyCellMap().values()) { 3903 miniBatchOp.addCellCount(cells.size()); 3904 } 3905 } 3906 3907 WALEdit fromCP = walEditsFromCoprocessors[index]; 3908 if (fromCP != null) { 3909 miniBatchOp.addCellCount(fromCP.size()); 3910 } 3911 return true; 3912 }); 3913 3914 if (region.coprocessorHost != null) { 3915 // calling the pre CP hook for batch mutation 3916 region.coprocessorHost.preBatchMutate(miniBatchOp); 3917 checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp); 3918 } 3919 } 3920 3921 /** 3922 * Starts the nonce operation for a mutation, if needed. 3923 * @param nonceGroup Nonce group from the request. 3924 * @param nonce Nonce. 3925 * @return whether to proceed this mutation. 3926 */ 3927 private boolean startNonceOperation(long nonceGroup, long nonce) throws IOException { 3928 if (region.rsServices == null || region.rsServices.getNonceManager() == null 3929 || nonce == HConstants.NO_NONCE) { 3930 return true; 3931 } 3932 boolean canProceed; 3933 try { 3934 canProceed = region.rsServices.getNonceManager() 3935 .startOperation(nonceGroup, nonce, region.rsServices); 3936 } catch (InterruptedException ex) { 3937 throw new InterruptedIOException("Nonce start operation interrupted"); 3938 } 3939 return canProceed; 3940 } 3941 3942 /** 3943 * Ends nonce operation for a mutation, if needed. 3944 * @param nonceGroup Nonce group from the request. Always 0 in initial implementation. 3945 * @param nonce Nonce. 3946 * @param success Whether the operation for this nonce has succeeded. 3947 */ 3948 private void endNonceOperation(long nonceGroup, long nonce, boolean success) { 3949 if (region.rsServices != null && region.rsServices.getNonceManager() != null 3950 && nonce != HConstants.NO_NONCE) { 3951 region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success); 3952 } 3953 } 3954 3955 private static Get toGet(final Mutation mutation) throws IOException { 3956 assert mutation instanceof Increment || mutation instanceof Append; 3957 Get get = new Get(mutation.getRow()); 3958 CellScanner cellScanner = mutation.cellScanner(); 3959 while (!cellScanner.advance()) { 3960 Cell cell = cellScanner.current(); 3961 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell)); 3962 } 3963 if (mutation instanceof Increment) { 3964 // Increment 3965 Increment increment = (Increment) mutation; 3966 get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax()); 3967 } else { 3968 // Append 3969 Append append = (Append) mutation; 3970 get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax()); 3971 } 3972 for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) { 3973 get.setAttribute(entry.getKey(), entry.getValue()); 3974 } 3975 return get; 3976 } 3977 3978 private Map<byte[], List<Cell>> reckonDeltas(Mutation mutation, List<Cell> results, 3979 long now) throws IOException { 3980 assert mutation instanceof Increment || mutation instanceof Append; 3981 Map<byte[], List<Cell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR); 3982 // Process a Store/family at a time. 3983 for (Map.Entry<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) { 3984 final byte[] columnFamilyName = entry.getKey(); 3985 List<Cell> deltas = entry.getValue(); 3986 // Reckon for the Store what to apply to WAL and MemStore. 3987 List<Cell> toApply = reckonDeltasByStore(region.stores.get(columnFamilyName), mutation, 3988 now, deltas, results); 3989 if (!toApply.isEmpty()) { 3990 for (Cell cell : toApply) { 3991 HStore store = region.getStore(cell); 3992 if (store == null) { 3993 region.checkFamily(CellUtil.cloneFamily(cell)); 3994 } else { 3995 ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(), 3996 key -> new ArrayList<>()).add(cell); 3997 } 3998 } 3999 } 4000 } 4001 return ret; 4002 } 4003 4004 /** 4005 * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed 4006 * column family/Store. 4007 * 4008 * Does Get of current value and then adds passed in deltas for this Store returning the 4009 * result. 4010 * 4011 * @param mutation The encompassing Mutation object 4012 * @param deltas Changes to apply to this Store; either increment amount or data to append 4013 * @param results In here we accumulate all the Cells we are to return to the client. If null, 4014 * client doesn't want results returned. 4015 * @return Resulting Cells after <code>deltas</code> have been applied to current 4016 * values. Side effect is our filling out of the <code>results</code> List. 4017 */ 4018 private List<Cell> reckonDeltasByStore(HStore store, Mutation mutation, long now, 4019 List<Cell> deltas, List<Cell> results) throws IOException { 4020 assert mutation instanceof Increment || mutation instanceof Append; 4021 byte[] columnFamily = store.getColumnFamilyDescriptor().getName(); 4022 List<Pair<Cell, Cell>> cellPairs = new ArrayList<>(deltas.size()); 4023 4024 // Sort the cells so that they match the order that they appear in the Get results. 4025 // Otherwise, we won't be able to find the existing values if the cells are not specified 4026 // in order by the client since cells are in an array list. 4027 sort(deltas, store.getComparator()); 4028 4029 // Get previous values for all columns in this family. 4030 Get get = new Get(mutation.getRow()); 4031 for (Cell cell: deltas) { 4032 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell)); 4033 } 4034 TimeRange tr; 4035 if (mutation instanceof Increment) { 4036 tr = ((Increment) mutation).getTimeRange(); 4037 } else { 4038 tr = ((Append) mutation).getTimeRange(); 4039 } 4040 4041 if (tr != null) { 4042 get.setTimeRange(tr.getMin(), tr.getMax()); 4043 } 4044 4045 List<Cell> currentValues = region.get(get, false); 4046 4047 // Iterate the input columns and update existing values if they were found, otherwise 4048 // add new column initialized to the delta amount 4049 int currentValuesIndex = 0; 4050 for (int i = 0; i < deltas.size(); i++) { 4051 Cell delta = deltas.get(i); 4052 Cell currentValue = null; 4053 if (currentValuesIndex < currentValues.size() && 4054 CellUtil.matchingQualifier(currentValues.get(currentValuesIndex), delta)) { 4055 currentValue = currentValues.get(currentValuesIndex); 4056 if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) { 4057 currentValuesIndex++; 4058 } 4059 } 4060 // Switch on whether this an increment or an append building the new Cell to apply. 4061 Cell newCell; 4062 if (mutation instanceof Increment) { 4063 long deltaAmount = getLongValue(delta); 4064 final long newValue = currentValue == null ? 4065 deltaAmount : getLongValue(currentValue) + deltaAmount; 4066 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, 4067 (oldCell) -> Bytes.toBytes(newValue)); 4068 } else { 4069 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, 4070 (oldCell) -> 4071 ByteBuffer.wrap(new byte[delta.getValueLength() + oldCell.getValueLength()]) 4072 .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength()) 4073 .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength()) 4074 .array() 4075 ); 4076 } 4077 if (region.maxCellSize > 0) { 4078 int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell); 4079 if (newCellSize > region.maxCellSize) { 4080 String msg = "Cell with size " + newCellSize + " exceeds limit of " 4081 + region.maxCellSize + " bytes in region " + this; 4082 LOG.debug(msg); 4083 throw new DoNotRetryIOException(msg); 4084 } 4085 } 4086 cellPairs.add(new Pair<>(currentValue, newCell)); 4087 // Add to results to get returned to the Client. If null, cilent does not want results. 4088 if (results != null) { 4089 results.add(newCell); 4090 } 4091 } 4092 // Give coprocessors a chance to update the new cells before apply to WAL or memstore 4093 if (region.coprocessorHost != null) { 4094 // Here the operation must be increment or append. 4095 cellPairs = mutation instanceof Increment ? 4096 region.coprocessorHost.postIncrementBeforeWAL(mutation, cellPairs) : 4097 region.coprocessorHost.postAppendBeforeWAL(mutation, cellPairs); 4098 } 4099 return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList()); 4100 } 4101 4102 private static Cell reckonDelta(final Cell delta, final Cell currentCell, 4103 final byte[] columnFamily, final long now, Mutation mutation, 4104 Function<Cell, byte[]> supplier) throws IOException { 4105 // Forward any tags found on the delta. 4106 List<Tag> tags = TagUtil.carryForwardTags(delta); 4107 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL()); 4108 if (currentCell != null) { 4109 tags = TagUtil.carryForwardTags(tags, currentCell); 4110 byte[] newValue = supplier.apply(currentCell); 4111 return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 4112 .setRow(mutation.getRow(), 0, mutation.getRow().length) 4113 .setFamily(columnFamily, 0, columnFamily.length) 4114 // copy the qualifier if the cell is located in shared memory. 4115 .setQualifier(CellUtil.cloneQualifier(delta)) 4116 .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now)) 4117 .setType(KeyValue.Type.Put.getCode()) 4118 .setValue(newValue, 0, newValue.length) 4119 .setTags(TagUtil.fromList(tags)) 4120 .build(); 4121 } else { 4122 PrivateCellUtil.updateLatestStamp(delta, now); 4123 return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags); 4124 } 4125 } 4126 4127 /** 4128 * @return Get the long out of the passed in Cell 4129 */ 4130 private static long getLongValue(final Cell cell) throws DoNotRetryIOException { 4131 int len = cell.getValueLength(); 4132 if (len != Bytes.SIZEOF_LONG) { 4133 // throw DoNotRetryIOException instead of IllegalArgumentException 4134 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide"); 4135 } 4136 return PrivateCellUtil.getValueAsLong(cell); 4137 } 4138 4139 @Override 4140 public List<Pair<NonceKey, WALEdit>> buildWALEdits(final MiniBatchOperationInProgress<Mutation> 4141 miniBatchOp) throws IOException { 4142 List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp); 4143 // for MutationBatchOperation, more than one nonce is not allowed 4144 if (walEdits.size() > 1) { 4145 throw new IOException("Found multiple nonce keys per batch!"); 4146 } 4147 return walEdits; 4148 } 4149 4150 @Override 4151 public WriteEntry writeMiniBatchOperationsToMemStore( 4152 final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry) 4153 throws IOException { 4154 if (writeEntry == null) { 4155 writeEntry = region.mvcc.begin(); 4156 } 4157 super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber()); 4158 return writeEntry; 4159 } 4160 4161 @Override 4162 public void completeMiniBatchOperations( 4163 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 4164 throws IOException { 4165 // TODO: can it be done after completing mvcc? 4166 // calling the post CP hook for batch mutation 4167 if (region.coprocessorHost != null) { 4168 region.coprocessorHost.postBatchMutate(miniBatchOp); 4169 } 4170 super.completeMiniBatchOperations(miniBatchOp, writeEntry); 4171 4172 if (nonce != HConstants.NO_NONCE) { 4173 if (region.rsServices != null && region.rsServices.getNonceManager() != null) { 4174 region.rsServices.getNonceManager() 4175 .addMvccToOperationContext(nonceGroup, nonce, writeEntry.getWriteNumber()); 4176 } 4177 } 4178 } 4179 4180 @Override 4181 public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp, 4182 final WALEdit walEdit, boolean success) throws IOException { 4183 4184 super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success); 4185 if (miniBatchOp != null) { 4186 // synced so that the coprocessor contract is adhered to. 4187 if (region.coprocessorHost != null) { 4188 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> { 4189 // only for successful puts/deletes/increments/appends 4190 if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) { 4191 Mutation m = getMutation(i); 4192 if (m instanceof Put) { 4193 region.coprocessorHost.postPut((Put) m, walEdit, m.getDurability()); 4194 } else if (m instanceof Delete) { 4195 region.coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability()); 4196 } else if (m instanceof Increment) { 4197 Result result = region.getCoprocessorHost().postIncrement((Increment) m, 4198 results[i]); 4199 if (result != results[i]) { 4200 retCodeDetails[i] = 4201 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result); 4202 } 4203 } else if (m instanceof Append) { 4204 Result result = region.getCoprocessorHost().postAppend((Append) m, results[i]); 4205 if (result != results[i]) { 4206 retCodeDetails[i] = 4207 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result); 4208 } 4209 } 4210 } 4211 return true; 4212 }); 4213 } 4214 4215 // For nonce operations 4216 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> { 4217 if (canProceed[i]) { 4218 endNonceOperation(nonceGroup, nonce, 4219 retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS); 4220 } 4221 return true; 4222 }); 4223 4224 // See if the column families were consistent through the whole thing. 4225 // if they were then keep them. If they were not then pass a null. 4226 // null will be treated as unknown. 4227 // Total time taken might be involving Puts, Deletes, Increments and Appends. 4228 // Split the time for puts and deletes based on the total number of Puts, Deletes, 4229 // Increments and Appends. 4230 if (region.metricsRegion != null) { 4231 if (miniBatchOp.getNumOfPuts() > 0) { 4232 // There were some Puts in the batch. 4233 region.metricsRegion.updatePut(); 4234 } 4235 if (miniBatchOp.getNumOfDeletes() > 0) { 4236 // There were some Deletes in the batch. 4237 region.metricsRegion.updateDelete(); 4238 } 4239 if (miniBatchOp.getNumOfIncrements() > 0) { 4240 // There were some Increments in the batch. 4241 region.metricsRegion.updateIncrement(); 4242 } 4243 if (miniBatchOp.getNumOfAppends() > 0) { 4244 // There were some Appends in the batch. 4245 region.metricsRegion.updateAppend(); 4246 } 4247 } 4248 } 4249 4250 if (region.coprocessorHost != null) { 4251 // call the coprocessor hook to do any finalization steps after the put is done 4252 region.coprocessorHost.postBatchMutateIndispensably( 4253 miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success); 4254 } 4255 } 4256 4257 /** 4258 * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch 4259 * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2: 4260 * count of increments and 3: count of appends 4261 */ 4262 private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics) 4263 throws IOException { 4264 Mutation m = getMutation(index); 4265 if (m instanceof Put) { 4266 if (region.coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) { 4267 // pre hook says skip this Put 4268 // mark as success and skip in doMiniBatchMutation 4269 metrics[0]++; 4270 retCodeDetails[index] = OperationStatus.SUCCESS; 4271 } 4272 } else if (m instanceof Delete) { 4273 Delete curDel = (Delete) m; 4274 if (curDel.getFamilyCellMap().isEmpty()) { 4275 // handle deleting a row case 4276 // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook. 4277 // Can this be avoided? 4278 region.prepareDelete(curDel); 4279 } 4280 if (region.coprocessorHost.preDelete(curDel, walEdit, m.getDurability())) { 4281 // pre hook says skip this Delete 4282 // mark as success and skip in doMiniBatchMutation 4283 metrics[1]++; 4284 retCodeDetails[index] = OperationStatus.SUCCESS; 4285 } 4286 } else if (m instanceof Increment) { 4287 Increment increment = (Increment) m; 4288 Result result = region.coprocessorHost.preIncrement(increment); 4289 if (result != null) { 4290 // pre hook says skip this Increment 4291 // mark as success and skip in doMiniBatchMutation 4292 metrics[2]++; 4293 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 4294 } 4295 } else if (m instanceof Append) { 4296 Append append = (Append) m; 4297 Result result = region.coprocessorHost.preAppend(append); 4298 if (result != null) { 4299 // pre hook says skip this Append 4300 // mark as success and skip in doMiniBatchMutation 4301 metrics[3]++; 4302 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 4303 } 4304 } else { 4305 String msg = "Put/Delete/Increment/Append mutations only supported in a batch"; 4306 retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg); 4307 if (isAtomic()) { // fail, atomic means all or none 4308 throw new IOException(msg); 4309 } 4310 } 4311 } 4312 4313 // TODO Support Increment/Append operations 4314 private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp, 4315 final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException { 4316 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> { 4317 // we pass (i - firstIndex) below since the call expects a relative index 4318 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess); 4319 if (cpMutations == null) { 4320 return true; 4321 } 4322 // Else Coprocessor added more Mutations corresponding to the Mutation at this index. 4323 Mutation mutation = getMutation(i); 4324 for (Mutation cpMutation : cpMutations) { 4325 this.checkAndPrepareMutation(cpMutation, timestamp); 4326 4327 // Acquire row locks. If not, the whole batch will fail. 4328 acquiredRowLocks.add(region.getRowLockInternal(cpMutation.getRow(), true, null)); 4329 4330 // Returned mutations from coprocessor correspond to the Mutation at index i. We can 4331 // directly add the cells from those mutations to the familyMaps of this mutation. 4332 Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap(); 4333 region.rewriteCellTags(cpFamilyMap, mutation); 4334 // will get added to the memStore later 4335 mergeFamilyMaps(familyCellMaps[i], cpFamilyMap); 4336 4337 // The durability of returned mutation is replaced by the corresponding mutation. 4338 // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the 4339 // cells of returned mutation. 4340 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) { 4341 for (List<Cell> cells : cpFamilyMap.values()) { 4342 miniBatchOp.addCellCount(cells.size()); 4343 } 4344 } 4345 } 4346 return true; 4347 }); 4348 } 4349 4350 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap, 4351 Map<byte[], List<Cell>> toBeMerged) { 4352 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) { 4353 List<Cell> cells = familyMap.get(entry.getKey()); 4354 if (cells == null) { 4355 familyMap.put(entry.getKey(), entry.getValue()); 4356 } else { 4357 cells.addAll(entry.getValue()); 4358 } 4359 } 4360 } 4361 } 4362 4363 /** 4364 * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most 4365 * of the logic is same. 4366 */ 4367 static class ReplayBatchOperation extends BatchOperation<MutationReplay> { 4368 private long origLogSeqNum = 0; 4369 public ReplayBatchOperation(final HRegion region, MutationReplay[] operations, 4370 long origLogSeqNum) { 4371 super(region, operations); 4372 this.origLogSeqNum = origLogSeqNum; 4373 } 4374 4375 @Override 4376 public Mutation getMutation(int index) { 4377 return this.operations[index].mutation; 4378 } 4379 4380 @Override 4381 public long getNonceGroup(int index) { 4382 return this.operations[index].nonceGroup; 4383 } 4384 4385 @Override 4386 public long getNonce(int index) { 4387 return this.operations[index].nonce; 4388 } 4389 4390 @Override 4391 public Mutation[] getMutationsForCoprocs() { 4392 return null; 4393 } 4394 4395 @Override 4396 public boolean isInReplay() { 4397 return true; 4398 } 4399 4400 @Override 4401 public long getOrigLogSeqNum() { 4402 return this.origLogSeqNum; 4403 } 4404 4405 @Override 4406 public void startRegionOperation() throws IOException { 4407 region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE); 4408 } 4409 4410 @Override 4411 public void closeRegionOperation() throws IOException { 4412 region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE); 4413 } 4414 4415 /** 4416 * During replay, there could exist column families which are removed between region server 4417 * failure and replay 4418 */ 4419 @Override 4420 protected void checkAndPreparePut(Put p) throws IOException { 4421 Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap(); 4422 List<byte[]> nonExistentList = null; 4423 for (byte[] family : familyCellMap.keySet()) { 4424 if (!region.htableDescriptor.hasColumnFamily(family)) { 4425 if (nonExistentList == null) { 4426 nonExistentList = new ArrayList<>(); 4427 } 4428 nonExistentList.add(family); 4429 } 4430 } 4431 if (nonExistentList != null) { 4432 for (byte[] family : nonExistentList) { 4433 // Perhaps schema was changed between crash and replay 4434 LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this); 4435 familyCellMap.remove(family); 4436 } 4437 } 4438 } 4439 4440 @Override 4441 public void checkAndPrepare() throws IOException { 4442 long now = EnvironmentEdgeManager.currentTime(); 4443 visitBatchOperations(true, this.size(), (int index) -> { 4444 checkAndPrepareMutation(index, now); 4445 return true; 4446 }); 4447 } 4448 4449 @Override 4450 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp, 4451 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException { 4452 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 4453 // update cell count 4454 for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) { 4455 miniBatchOp.addCellCount(cells.size()); 4456 } 4457 return true; 4458 }); 4459 } 4460 4461 @Override 4462 public WriteEntry writeMiniBatchOperationsToMemStore( 4463 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 4464 throws IOException { 4465 super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum()); 4466 return writeEntry; 4467 } 4468 4469 @Override 4470 public void completeMiniBatchOperations( 4471 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 4472 throws IOException { 4473 super.completeMiniBatchOperations(miniBatchOp, writeEntry); 4474 region.mvcc.advanceTo(getOrigLogSeqNum()); 4475 } 4476 } 4477 4478 public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup, 4479 long nonce) throws IOException { 4480 // As it stands, this is used for 3 things 4481 // * batchMutate with single mutation - put/delete/increment/append, separate or from 4482 // checkAndMutate. 4483 // * coprocessor calls (see ex. BulkDeleteEndpoint). 4484 // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd... 4485 return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce)); 4486 } 4487 4488 @Override 4489 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException { 4490 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically 4491 boolean atomic = Arrays.stream(mutations) 4492 .anyMatch(m -> m instanceof Increment || m instanceof Append); 4493 return batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE); 4494 } 4495 4496 public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) 4497 throws IOException { 4498 if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo()) 4499 && replaySeqId < lastReplayedOpenRegionSeqId) { 4500 // if it is a secondary replica we should ignore these entries silently 4501 // since they are coming out of order 4502 if (LOG.isTraceEnabled()) { 4503 LOG.trace(getRegionInfo().getEncodedName() + " : " 4504 + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId 4505 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId); 4506 for (MutationReplay mut : mutations) { 4507 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation); 4508 } 4509 } 4510 4511 OperationStatus[] statuses = new OperationStatus[mutations.length]; 4512 for (int i = 0; i < statuses.length; i++) { 4513 statuses[i] = OperationStatus.SUCCESS; 4514 } 4515 return statuses; 4516 } 4517 return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId)); 4518 } 4519 4520 /** 4521 * Perform a batch of mutations. 4522 * 4523 * Operations in a batch are stored with highest durability specified of for all operations in a 4524 * batch, except for {@link Durability#SKIP_WAL}. 4525 * 4526 * <p>This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with 4527 * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with 4528 * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch 4529 * and mutation batch is very similar, lot of code is shared by providing generic methods in 4530 * base class {@link BatchOperation}. The logic for this method and 4531 * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which 4532 * are overridden by derived classes to implement special behavior. 4533 * 4534 * @param batchOp contains the list of mutations 4535 * @return an array of OperationStatus which internally contains the 4536 * OperationStatusCode and the exceptionMessage if any. 4537 * @throws IOException if an IO problem is encountered 4538 */ 4539 OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException { 4540 boolean initialized = false; 4541 batchOp.startRegionOperation(); 4542 try { 4543 while (!batchOp.isDone()) { 4544 if (!batchOp.isInReplay()) { 4545 checkReadOnly(); 4546 } 4547 checkResources(); 4548 4549 if (!initialized) { 4550 this.writeRequestsCount.add(batchOp.size()); 4551 // validate and prepare batch for write, for MutationBatchOperation it also calls CP 4552 // prePut()/preDelete()/preIncrement()/preAppend() hooks 4553 batchOp.checkAndPrepare(); 4554 initialized = true; 4555 } 4556 doMiniBatchMutate(batchOp); 4557 requestFlushIfNeeded(); 4558 } 4559 } finally { 4560 if (rsServices != null && rsServices.getMetrics() != null) { 4561 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor. 4562 getTableName(), batchOp.size()); 4563 } 4564 batchOp.closeRegionOperation(); 4565 } 4566 return batchOp.retCodeDetails; 4567 } 4568 4569 /** 4570 * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])} 4571 * In here we also handle replay of edits on region recover. Also gets change in size brought 4572 * about by applying {@code batchOp}. 4573 */ 4574 private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException { 4575 boolean success = false; 4576 WALEdit walEdit = null; 4577 WriteEntry writeEntry = null; 4578 boolean locked = false; 4579 // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive) 4580 MiniBatchOperationInProgress<Mutation> miniBatchOp = null; 4581 /** Keep track of the locks we hold so we can release them in finally clause */ 4582 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size()); 4583 4584 // Check for thread interrupt status in case we have been signaled from 4585 // #interruptRegionOperation. 4586 checkInterrupt(); 4587 4588 try { 4589 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with 4590 // locked rows 4591 miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks); 4592 4593 // We've now grabbed as many mutations off the list as we can 4594 // Ensure we acquire at least one. 4595 if (miniBatchOp.getReadyToWriteCount() <= 0) { 4596 // Nothing to put/delete/increment/append -- an exception in the above such as 4597 // NoSuchColumnFamily? 4598 return; 4599 } 4600 4601 // Check for thread interrupt status in case we have been signaled from 4602 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for 4603 // the WAL append. 4604 checkInterrupt(); 4605 4606 lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount()); 4607 locked = true; 4608 4609 // From this point until memstore update this operation should not be interrupted. 4610 disableInterrupts(); 4611 4612 // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp 4613 // We should record the timestamp only after we have acquired the rowLock, 4614 // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer 4615 // timestamp 4616 4617 long now = EnvironmentEdgeManager.currentTime(); 4618 batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks); 4619 4620 // STEP 3. Build WAL edit 4621 4622 List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp); 4623 4624 // STEP 4. Append the WALEdits to WAL and sync. 4625 4626 for(Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) { 4627 Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next(); 4628 walEdit = nonceKeyWALEditPair.getSecond(); 4629 NonceKey nonceKey = nonceKeyWALEditPair.getFirst(); 4630 4631 if (walEdit != null && !walEdit.isEmpty()) { 4632 writeEntry = doWALAppend(walEdit, batchOp.durability, batchOp.getClusterIds(), now, 4633 nonceKey.getNonceGroup(), nonceKey.getNonce(), batchOp.getOrigLogSeqNum()); 4634 } 4635 4636 // Complete mvcc for all but last writeEntry (for replay case) 4637 if (it.hasNext() && writeEntry != null) { 4638 mvcc.complete(writeEntry); 4639 writeEntry = null; 4640 } 4641 } 4642 4643 // STEP 5. Write back to memStore 4644 // NOTE: writeEntry can be null here 4645 writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry); 4646 4647 // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and 4648 // complete mvcc for last writeEntry 4649 batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry); 4650 writeEntry = null; 4651 success = true; 4652 } finally { 4653 // Call complete rather than completeAndWait because we probably had error if walKey != null 4654 if (writeEntry != null) mvcc.complete(writeEntry); 4655 4656 if (locked) { 4657 this.updatesLock.readLock().unlock(); 4658 } 4659 releaseRowLocks(acquiredRowLocks); 4660 4661 enableInterrupts(); 4662 4663 final int finalLastIndexExclusive = 4664 miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size(); 4665 final boolean finalSuccess = success; 4666 batchOp.visitBatchOperations(true, finalLastIndexExclusive, 4667 (int i) -> { 4668 Mutation mutation = batchOp.getMutation(i); 4669 if (mutation instanceof Increment || mutation instanceof Append) { 4670 if (finalSuccess) { 4671 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.SUCCESS, 4672 batchOp.results[i]); 4673 } else { 4674 batchOp.retCodeDetails[i] = OperationStatus.FAILURE; 4675 } 4676 } else { 4677 batchOp.retCodeDetails[i] = 4678 finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE; 4679 } 4680 return true; 4681 }); 4682 4683 batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess); 4684 4685 batchOp.nextIndexToProcess = finalLastIndexExclusive; 4686 } 4687 } 4688 4689 /** 4690 * Returns effective durability from the passed durability and 4691 * the table descriptor. 4692 */ 4693 protected Durability getEffectiveDurability(Durability d) { 4694 return d == Durability.USE_DEFAULT ? this.regionDurability : d; 4695 } 4696 4697 @Override 4698 @Deprecated 4699 public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, 4700 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException { 4701 CheckAndMutate checkAndMutate; 4702 try { 4703 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row) 4704 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange); 4705 if (mutation instanceof Put) { 4706 checkAndMutate = builder.build((Put) mutation); 4707 } else if (mutation instanceof Delete) { 4708 checkAndMutate = builder.build((Delete) mutation); 4709 } else { 4710 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass() 4711 .getSimpleName().toUpperCase()); 4712 } 4713 } catch (IllegalArgumentException e) { 4714 throw new DoNotRetryIOException(e.getMessage()); 4715 } 4716 return checkAndMutate(checkAndMutate).isSuccess(); 4717 } 4718 4719 @Override 4720 @Deprecated 4721 public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) 4722 throws IOException { 4723 CheckAndMutate checkAndMutate; 4724 try { 4725 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row).ifMatches(filter) 4726 .timeRange(timeRange); 4727 if (mutation instanceof Put) { 4728 checkAndMutate = builder.build((Put) mutation); 4729 } else if (mutation instanceof Delete) { 4730 checkAndMutate = builder.build((Delete) mutation); 4731 } else { 4732 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass() 4733 .getSimpleName().toUpperCase()); 4734 } 4735 } catch (IllegalArgumentException e) { 4736 throw new DoNotRetryIOException(e.getMessage()); 4737 } 4738 return checkAndMutate(checkAndMutate).isSuccess(); 4739 } 4740 4741 @Override 4742 @Deprecated 4743 public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, 4744 ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException { 4745 CheckAndMutate checkAndMutate; 4746 try { 4747 checkAndMutate = CheckAndMutate.newBuilder(row) 4748 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm); 4749 } catch (IllegalArgumentException e) { 4750 throw new DoNotRetryIOException(e.getMessage()); 4751 } 4752 return checkAndMutate(checkAndMutate).isSuccess(); 4753 } 4754 4755 @Override 4756 @Deprecated 4757 public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm) 4758 throws IOException { 4759 CheckAndMutate checkAndMutate; 4760 try { 4761 checkAndMutate = CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange) 4762 .build(rm); 4763 } catch (IllegalArgumentException e) { 4764 throw new DoNotRetryIOException(e.getMessage()); 4765 } 4766 return checkAndMutate(checkAndMutate).isSuccess(); 4767 } 4768 4769 @Override 4770 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException { 4771 byte[] row = checkAndMutate.getRow(); 4772 Filter filter = null; 4773 byte[] family = null; 4774 byte[] qualifier = null; 4775 CompareOperator op = null; 4776 ByteArrayComparable comparator = null; 4777 if (checkAndMutate.hasFilter()) { 4778 filter = checkAndMutate.getFilter(); 4779 } else { 4780 family = checkAndMutate.getFamily(); 4781 qualifier = checkAndMutate.getQualifier(); 4782 op = checkAndMutate.getCompareOp(); 4783 comparator = new BinaryComparator(checkAndMutate.getValue()); 4784 } 4785 TimeRange timeRange = checkAndMutate.getTimeRange(); 4786 4787 Mutation mutation = null; 4788 RowMutations rowMutations = null; 4789 if (checkAndMutate.getAction() instanceof Mutation) { 4790 mutation = (Mutation) checkAndMutate.getAction(); 4791 } else { 4792 rowMutations = (RowMutations) checkAndMutate.getAction(); 4793 } 4794 4795 if (mutation != null) { 4796 checkMutationType(mutation); 4797 checkRow(mutation, row); 4798 } else { 4799 checkRow(rowMutations, row); 4800 } 4801 checkReadOnly(); 4802 // TODO, add check for value length also move this check to the client 4803 checkResources(); 4804 startRegionOperation(); 4805 try { 4806 Get get = new Get(row); 4807 if (family != null) { 4808 checkFamily(family); 4809 get.addColumn(family, qualifier); 4810 } 4811 if (filter != null) { 4812 get.setFilter(filter); 4813 } 4814 if (timeRange != null) { 4815 get.setTimeRange(timeRange.getMin(), timeRange.getMax()); 4816 } 4817 // Lock row - note that doBatchMutate will relock this row if called 4818 checkRow(row, "doCheckAndRowMutate"); 4819 RowLock rowLock = getRowLockInternal(get.getRow(), false, null); 4820 try { 4821 if (this.getCoprocessorHost() != null) { 4822 CheckAndMutateResult result = 4823 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate); 4824 if (result != null) { 4825 return result; 4826 } 4827 } 4828 4829 // NOTE: We used to wait here until mvcc caught up: mvcc.await(); 4830 // Supposition is that now all changes are done under row locks, then when we go to read, 4831 // we'll get the latest on this row. 4832 List<Cell> result = get(get, false); 4833 boolean matches = false; 4834 long cellTs = 0; 4835 if (filter != null) { 4836 if (!result.isEmpty()) { 4837 matches = true; 4838 cellTs = result.get(0).getTimestamp(); 4839 } 4840 } else { 4841 boolean valueIsNull = comparator.getValue() == null || comparator.getValue().length == 0; 4842 if (result.isEmpty() && valueIsNull) { 4843 matches = true; 4844 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 && valueIsNull) { 4845 matches = true; 4846 cellTs = result.get(0).getTimestamp(); 4847 } else if (result.size() == 1 && !valueIsNull) { 4848 Cell kv = result.get(0); 4849 cellTs = kv.getTimestamp(); 4850 int compareResult = PrivateCellUtil.compareValue(kv, comparator); 4851 matches = matches(op, compareResult); 4852 } 4853 } 4854 4855 // If matches, perform the mutation or the rowMutations 4856 if (matches) { 4857 // We have acquired the row lock already. If the system clock is NOT monotonically 4858 // non-decreasing (see HBASE-14070) we should make sure that the mutation has a 4859 // larger timestamp than what was observed via Get. doBatchMutate already does this, but 4860 // there is no way to pass the cellTs. See HBASE-14054. 4861 long now = EnvironmentEdgeManager.currentTime(); 4862 long ts = Math.max(now, cellTs); // ensure write is not eclipsed 4863 byte[] byteTs = Bytes.toBytes(ts); 4864 if (mutation != null) { 4865 if (mutation instanceof Put) { 4866 updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs); 4867 } 4868 // And else 'delete' is not needed since it already does a second get, and sets the 4869 // timestamp from get (see prepareDeleteTimestamps). 4870 } else { 4871 for (Mutation m: rowMutations.getMutations()) { 4872 if (m instanceof Put) { 4873 updateCellTimestamps(m.getFamilyCellMap().values(), byteTs); 4874 } 4875 } 4876 // And else 'delete' is not needed since it already does a second get, and sets the 4877 // timestamp from get (see prepareDeleteTimestamps). 4878 } 4879 // All edits for the given row (across all column families) must happen atomically. 4880 Result r; 4881 if (mutation != null) { 4882 r = doBatchMutate(mutation, true).getResult(); 4883 } else { 4884 r = mutateRow(rowMutations); 4885 } 4886 this.checkAndMutateChecksPassed.increment(); 4887 return new CheckAndMutateResult(true, r); 4888 } 4889 this.checkAndMutateChecksFailed.increment(); 4890 return new CheckAndMutateResult(false, null); 4891 } finally { 4892 rowLock.release(); 4893 } 4894 } finally { 4895 closeRegionOperation(); 4896 } 4897 } 4898 4899 private void checkMutationType(final Mutation mutation) 4900 throws DoNotRetryIOException { 4901 if (!(mutation instanceof Put) && !(mutation instanceof Delete) && 4902 !(mutation instanceof Increment) && !(mutation instanceof Append)) { 4903 throw new org.apache.hadoop.hbase.DoNotRetryIOException( 4904 "Action must be Put or Delete or Increment or Delete"); 4905 } 4906 } 4907 4908 private void checkRow(final Row action, final byte[] row) 4909 throws DoNotRetryIOException { 4910 if (!Bytes.equals(row, action.getRow())) { 4911 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match"); 4912 } 4913 } 4914 4915 private boolean matches(final CompareOperator op, final int compareResult) { 4916 boolean matches = false; 4917 switch (op) { 4918 case LESS: 4919 matches = compareResult < 0; 4920 break; 4921 case LESS_OR_EQUAL: 4922 matches = compareResult <= 0; 4923 break; 4924 case EQUAL: 4925 matches = compareResult == 0; 4926 break; 4927 case NOT_EQUAL: 4928 matches = compareResult != 0; 4929 break; 4930 case GREATER_OR_EQUAL: 4931 matches = compareResult >= 0; 4932 break; 4933 case GREATER: 4934 matches = compareResult > 0; 4935 break; 4936 default: 4937 throw new RuntimeException("Unknown Compare op " + op.name()); 4938 } 4939 return matches; 4940 } 4941 4942 private OperationStatus doBatchMutate(Mutation mutation) throws IOException { 4943 return doBatchMutate(mutation, false); 4944 } 4945 4946 private OperationStatus doBatchMutate(Mutation mutation, boolean atomic) throws IOException { 4947 return doBatchMutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE); 4948 } 4949 4950 private OperationStatus doBatchMutate(Mutation mutation, boolean atomic, long nonceGroup, 4951 long nonce) throws IOException { 4952 OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation}, atomic, 4953 nonceGroup, nonce); 4954 if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) { 4955 throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg()); 4956 } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) { 4957 throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg()); 4958 } else if (batchMutate[0].getOperationStatusCode().equals( 4959 OperationStatusCode.STORE_TOO_BUSY)) { 4960 throw new RegionTooBusyException(batchMutate[0].getExceptionMsg()); 4961 } 4962 return batchMutate[0]; 4963 } 4964 4965 /** 4966 * Complete taking the snapshot on the region. Writes the region info and adds references to the 4967 * working snapshot directory. 4968 * 4969 * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare} 4970 * arg. (In the future other cancellable HRegion methods could eventually add a 4971 * {@link ForeignExceptionSnare}, or we could do something fancier). 4972 * 4973 * @param desc snapshot description object 4974 * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to 4975 * bail out. This is allowed to be null and will just be ignored in that case. 4976 * @throws IOException if there is an external or internal error causing the snapshot to fail 4977 */ 4978 public void addRegionToSnapshot(SnapshotDescription desc, 4979 ForeignExceptionSnare exnSnare) throws IOException { 4980 Path rootDir = CommonFSUtils.getRootDir(conf); 4981 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf); 4982 4983 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(), 4984 snapshotDir, desc, exnSnare); 4985 manifest.addRegion(this); 4986 } 4987 4988 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId) 4989 throws IOException { 4990 for (List<Cell> cells: cellItr) { 4991 if (cells == null) return; 4992 for (Cell cell : cells) { 4993 PrivateCellUtil.setSequenceId(cell, sequenceId); 4994 } 4995 } 4996 } 4997 4998 /** 4999 * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP} 5000 * provided current timestamp. 5001 * @param cellItr 5002 * @param now 5003 */ 5004 private static void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now) 5005 throws IOException { 5006 for (List<Cell> cells: cellItr) { 5007 if (cells == null) continue; 5008 // Optimization: 'foreach' loop is not used. See: 5009 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 5010 assert cells instanceof RandomAccess; 5011 int listSize = cells.size(); 5012 for (int i = 0; i < listSize; i++) { 5013 PrivateCellUtil.updateLatestStamp(cells.get(i), now); 5014 } 5015 } 5016 } 5017 5018 /** 5019 * Possibly rewrite incoming cell tags. 5020 */ 5021 void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) { 5022 // Check if we have any work to do and early out otherwise 5023 // Update these checks as more logic is added here 5024 if (m.getTTL() == Long.MAX_VALUE) { 5025 return; 5026 } 5027 5028 // From this point we know we have some work to do 5029 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) { 5030 List<Cell> cells = e.getValue(); 5031 assert cells instanceof RandomAccess; 5032 int listSize = cells.size(); 5033 for (int i = 0; i < listSize; i++) { 5034 Cell cell = cells.get(i); 5035 List<Tag> newTags = TagUtil.carryForwardTags(null, cell); 5036 newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL()); 5037 // Rewrite the cell with the updated set of tags 5038 cells.set(i, PrivateCellUtil.createCell(cell, newTags)); 5039 } 5040 } 5041 } 5042 5043 /* 5044 * Check if resources to support an update. 5045 * 5046 * We throw RegionTooBusyException if above memstore limit 5047 * and expect client to retry using some kind of backoff 5048 */ 5049 void checkResources() throws RegionTooBusyException { 5050 // If catalog region, do not impose resource constraints or block updates. 5051 if (this.getRegionInfo().isMetaRegion()) return; 5052 5053 MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); 5054 if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) { 5055 blockedRequestsCount.increment(); 5056 requestFlush(); 5057 // Don't print current limit because it will vary too much. The message is used as a key 5058 // over in RetriesExhaustedWithDetailsException processing. 5059 final String regionName = 5060 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName(); 5061 final String serverName = this.getRegionServerServices() == null ? 5062 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" : 5063 this.getRegionServerServices().getServerName().toString()); 5064 RegionTooBusyException rtbe = new RegionTooBusyException( 5065 "Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils 5066 .humanSize(this.blockingMemStoreSize) + ", regionName=" + regionName + ", server=" 5067 + serverName); 5068 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe); 5069 throw rtbe; 5070 } 5071 } 5072 5073 /** 5074 * @throws IOException Throws exception if region is in read-only mode. 5075 */ 5076 protected void checkReadOnly() throws IOException { 5077 if (isReadOnly()) { 5078 throw new DoNotRetryIOException("region is read only"); 5079 } 5080 } 5081 5082 protected void checkReadsEnabled() throws IOException { 5083 if (!this.writestate.readsEnabled) { 5084 throw new IOException(getRegionInfo().getEncodedName() 5085 + ": The region's reads are disabled. Cannot serve the request"); 5086 } 5087 } 5088 5089 public void setReadsEnabled(boolean readsEnabled) { 5090 if (readsEnabled && !this.writestate.readsEnabled) { 5091 LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName()); 5092 } 5093 this.writestate.setReadsEnabled(readsEnabled); 5094 } 5095 5096 /** 5097 * Add updates first to the wal and then add values to memstore. 5098 * Warning: Assumption is caller has lock on passed in row. 5099 * @param edits Cell updates by column 5100 * @throws IOException 5101 */ 5102 void put(final byte [] row, byte [] family, List<Cell> edits) 5103 throws IOException { 5104 NavigableMap<byte[], List<Cell>> familyMap; 5105 familyMap = new TreeMap<>(Bytes.BYTES_COMPARATOR); 5106 5107 familyMap.put(family, edits); 5108 Put p = new Put(row); 5109 p.setFamilyCellMap(familyMap); 5110 doBatchMutate(p); 5111 } 5112 5113 /** 5114 * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be 5115 * set; when set we will run operations that make sense in the increment/append scenario 5116 * but that do not make sense otherwise. 5117 * @see #applyToMemStore(HStore, Cell, MemStoreSizing) 5118 */ 5119 private void applyToMemStore(HStore store, List<Cell> cells, boolean delta, 5120 MemStoreSizing memstoreAccounting) throws IOException { 5121 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! 5122 boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1; 5123 if (upsert) { 5124 store.upsert(cells, getSmallestReadPoint(), memstoreAccounting); 5125 } else { 5126 store.add(cells, memstoreAccounting); 5127 } 5128 } 5129 5130 /** 5131 * @see #applyToMemStore(HStore, List, boolean, MemStoreSizing) 5132 */ 5133 private void applyToMemStore(HStore store, Cell cell, MemStoreSizing memstoreAccounting) 5134 throws IOException { 5135 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! 5136 if (store == null) { 5137 checkFamily(CellUtil.cloneFamily(cell)); 5138 // Unreachable because checkFamily will throw exception 5139 } 5140 store.add(cell, memstoreAccounting); 5141 } 5142 5143 /** 5144 * Check the collection of families for validity. 5145 * @param families 5146 * @throws NoSuchColumnFamilyException 5147 */ 5148 public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException { 5149 for (byte[] family : families) { 5150 checkFamily(family); 5151 } 5152 } 5153 5154 /** 5155 * Check the collection of families for valid timestamps 5156 * @param familyMap 5157 * @param now current timestamp 5158 * @throws FailedSanityCheckException 5159 */ 5160 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now) 5161 throws FailedSanityCheckException { 5162 if (timestampSlop == HConstants.LATEST_TIMESTAMP) { 5163 return; 5164 } 5165 long maxTs = now + timestampSlop; 5166 for (List<Cell> kvs : familyMap.values()) { 5167 // Optimization: 'foreach' loop is not used. See: 5168 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 5169 assert kvs instanceof RandomAccess; 5170 int listSize = kvs.size(); 5171 for (int i=0; i < listSize; i++) { 5172 Cell cell = kvs.get(i); 5173 // see if the user-side TS is out of range. latest = server-side 5174 long ts = cell.getTimestamp(); 5175 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) { 5176 throw new FailedSanityCheckException("Timestamp for KV out of range " 5177 + cell + " (too.new=" + timestampSlop + ")"); 5178 } 5179 } 5180 } 5181 } 5182 5183 /* 5184 * @param size 5185 * @return True if size is over the flush threshold 5186 */ 5187 private boolean isFlushSize(MemStoreSize size) { 5188 return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize(); 5189 } 5190 5191 private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException { 5192 for (Path file : files) { 5193 if (!fs.delete(file, false)) { 5194 LOG.error("Failed delete of {}", file); 5195 } else { 5196 LOG.debug("Deleted recovered.edits file={}", file); 5197 } 5198 } 5199 } 5200 5201 /** 5202 * Read the edits put under this region by wal splitting process. Put 5203 * the recovered edits back up into this region. 5204 * 5205 * <p>We can ignore any wal message that has a sequence ID that's equal to or 5206 * lower than minSeqId. (Because we know such messages are already 5207 * reflected in the HFiles.) 5208 * 5209 * <p>While this is running we are putting pressure on memory yet we are 5210 * outside of our usual accounting because we are not yet an onlined region 5211 * (this stuff is being run as part of Region initialization). This means 5212 * that if we're up against global memory limits, we'll not be flagged to flush 5213 * because we are not online. We can't be flushed by usual mechanisms anyways; 5214 * we're not yet online so our relative sequenceids are not yet aligned with 5215 * WAL sequenceids -- not till we come up online, post processing of split 5216 * edits. 5217 * 5218 * <p>But to help relieve memory pressure, at least manage our own heap size 5219 * flushing if are in excess of per-region limits. Flushing, though, we have 5220 * to be careful and avoid using the regionserver/wal sequenceid. Its running 5221 * on a different line to whats going on in here in this region context so if we 5222 * crashed replaying these edits, but in the midst had a flush that used the 5223 * regionserver wal with a sequenceid in excess of whats going on in here 5224 * in this region and with its split editlogs, then we could miss edits the 5225 * next time we go to recover. So, we have to flush inline, using seqids that 5226 * make sense in a this single region context only -- until we online. 5227 * 5228 * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of 5229 * the maxSeqId for the store to be applied, else its skipped. 5230 * @return the sequence id of the last edit added to this region out of the 5231 * recovered edits log or <code>minSeqId</code> if nothing added from editlogs. 5232 */ 5233 long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores, 5234 final CancelableProgressable reporter, final MonitoredTask status) throws IOException { 5235 long minSeqIdForTheRegion = -1; 5236 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) { 5237 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) { 5238 minSeqIdForTheRegion = maxSeqIdInStore; 5239 } 5240 } 5241 long seqId = minSeqIdForTheRegion; 5242 String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR); 5243 if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) { 5244 FileSystem walFS = getWalFileSystem(); 5245 FileSystem rootFS = getFilesystem(); 5246 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(), 5247 getRegionInfo().getEncodedName()); 5248 Path regionWALDir = getWALRegionDir(); 5249 Path regionDir = 5250 FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo()); 5251 5252 // We made a mistake in HBASE-20734 so we need to do this dirty hack... 5253 NavigableSet<Path> filesUnderWrongRegionWALDir = 5254 WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir); 5255 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, 5256 filesUnderWrongRegionWALDir, reporter, regionDir)); 5257 // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear 5258 // under the root dir even if walDir is set. 5259 NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet(); 5260 if (!regionWALDir.equals(regionDir)) { 5261 filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir); 5262 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS, 5263 filesUnderRootDir, reporter, regionDir)); 5264 } 5265 5266 NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir); 5267 seqId = Math.max(seqId, 5268 replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir)); 5269 if (seqId > minSeqIdForTheRegion) { 5270 // Then we added some edits to memory. Flush and cleanup split edit files. 5271 internalFlushcache(null, seqId, stores.values(), status, false, 5272 FlushLifeCycleTracker.DUMMY); 5273 } 5274 // Now delete the content of recovered edits. We're done w/ them. 5275 if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) { 5276 // For debugging data loss issues! 5277 // If this flag is set, make use of the hfile archiving by making recovered.edits a fake 5278 // column family. Have to fake out file type too by casting our recovered.edits as 5279 // storefiles 5280 String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName(); 5281 Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size()); 5282 for (Path file : files) { 5283 fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true)); 5284 } 5285 getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles); 5286 } else { 5287 deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir)); 5288 deleteRecoveredEdits(rootFS, filesUnderRootDir); 5289 } 5290 } else { 5291 Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr); 5292 FileSystem fs = recoveredEditsDir.getFileSystem(conf); 5293 FileStatus[] files = fs.listStatus(recoveredEditsDir); 5294 LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length, 5295 recoveredEditsDir); 5296 if (files != null) { 5297 for (FileStatus file : files) { 5298 // it is safe to trust the zero-length in this case because we've been through rename and 5299 // lease recovery in the above. 5300 if (isZeroLengthThenDelete(fs, file, file.getPath())) { 5301 continue; 5302 } 5303 seqId = 5304 Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs)); 5305 } 5306 } 5307 if (seqId > minSeqIdForTheRegion) { 5308 // Then we added some edits to memory. Flush and cleanup split edit files. 5309 internalFlushcache(null, seqId, stores.values(), status, false, 5310 FlushLifeCycleTracker.DUMMY); 5311 } 5312 deleteRecoveredEdits(fs, 5313 Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList())); 5314 } 5315 5316 return seqId; 5317 } 5318 5319 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs, 5320 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir) 5321 throws IOException { 5322 long seqid = minSeqIdForTheRegion; 5323 if (LOG.isDebugEnabled()) { 5324 LOG.debug("Found " + (files == null ? 0 : files.size()) 5325 + " recovered edits file(s) under " + regionDir); 5326 } 5327 5328 if (files == null || files.isEmpty()) { 5329 return minSeqIdForTheRegion; 5330 } 5331 5332 for (Path edits: files) { 5333 if (edits == null || !fs.exists(edits)) { 5334 LOG.warn("Null or non-existent edits file: " + edits); 5335 continue; 5336 } 5337 if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) { 5338 continue; 5339 } 5340 5341 long maxSeqId; 5342 String fileName = edits.getName(); 5343 maxSeqId = Math.abs(Long.parseLong(fileName)); 5344 if (maxSeqId <= minSeqIdForTheRegion) { 5345 if (LOG.isDebugEnabled()) { 5346 String msg = "Maximum sequenceid for this wal is " + maxSeqId 5347 + " and minimum sequenceid for the region " + this + " is " + minSeqIdForTheRegion 5348 + ", skipped the whole file, path=" + edits; 5349 LOG.debug(msg); 5350 } 5351 continue; 5352 } 5353 5354 try { 5355 // replay the edits. Replay can return -1 if everything is skipped, only update 5356 // if seqId is greater 5357 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs)); 5358 } catch (IOException e) { 5359 handleException(fs, edits, e); 5360 } 5361 } 5362 return seqid; 5363 } 5364 5365 private void handleException(FileSystem fs, Path edits, IOException e) throws IOException { 5366 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS, 5367 conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS)); 5368 if (conf.get("hbase.skip.errors") != null) { 5369 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use " 5370 + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead."); 5371 } 5372 if (skipErrors) { 5373 Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits); 5374 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed " 5375 + edits + " as " + p, 5376 e); 5377 } else { 5378 throw e; 5379 } 5380 } 5381 5382 /** 5383 * @param edits File of recovered edits. 5384 * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger 5385 * than this to be replayed for each store. 5386 * @return the sequence id of the last edit added to this region out of the recovered edits log or 5387 * <code>minSeqId</code> if nothing added from editlogs. 5388 */ 5389 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores, 5390 final CancelableProgressable reporter, FileSystem fs) throws IOException { 5391 String msg = "Replaying edits from " + edits; 5392 LOG.info(msg); 5393 MonitoredTask status = TaskMonitor.get().createStatus(msg); 5394 5395 status.setStatus("Opening recovered edits"); 5396 WAL.Reader reader = null; 5397 try { 5398 reader = WALFactory.createReader(fs, edits, conf); 5399 long currentEditSeqId = -1; 5400 long currentReplaySeqId = -1; 5401 long firstSeqIdInLog = -1; 5402 long skippedEdits = 0; 5403 long editsCount = 0; 5404 long intervalEdits = 0; 5405 WAL.Entry entry; 5406 HStore store = null; 5407 boolean reported_once = false; 5408 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager(); 5409 5410 try { 5411 // How many edits seen before we check elapsed time 5412 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000); 5413 // How often to send a progress report (default 1/2 master timeout) 5414 int period = this.conf.getInt("hbase.hstore.report.period", 300000); 5415 long lastReport = EnvironmentEdgeManager.currentTime(); 5416 5417 if (coprocessorHost != null) { 5418 coprocessorHost.preReplayWALs(this.getRegionInfo(), edits); 5419 } 5420 5421 while ((entry = reader.next()) != null) { 5422 WALKey key = entry.getKey(); 5423 WALEdit val = entry.getEdit(); 5424 5425 if (ng != null) { // some test, or nonces disabled 5426 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime()); 5427 } 5428 5429 if (reporter != null) { 5430 intervalEdits += val.size(); 5431 if (intervalEdits >= interval) { 5432 // Number of edits interval reached 5433 intervalEdits = 0; 5434 long cur = EnvironmentEdgeManager.currentTime(); 5435 if (lastReport + period <= cur) { 5436 status.setStatus("Replaying edits..." + 5437 " skipped=" + skippedEdits + 5438 " edits=" + editsCount); 5439 // Timeout reached 5440 if(!reporter.progress()) { 5441 msg = "Progressable reporter failed, stopping replay for region " + this; 5442 LOG.warn(msg); 5443 status.abort(msg); 5444 throw new IOException(msg); 5445 } 5446 reported_once = true; 5447 lastReport = cur; 5448 } 5449 } 5450 } 5451 5452 if (firstSeqIdInLog == -1) { 5453 firstSeqIdInLog = key.getSequenceId(); 5454 } 5455 if (currentEditSeqId > key.getSequenceId()) { 5456 // when this condition is true, it means we have a serious defect because we need to 5457 // maintain increasing SeqId for WAL edits per region 5458 LOG.error(getRegionInfo().getEncodedName() + " : " 5459 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key 5460 + "; edit=" + val); 5461 } else { 5462 currentEditSeqId = key.getSequenceId(); 5463 } 5464 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ? 5465 key.getOrigLogSeqNum() : currentEditSeqId; 5466 5467 // Start coprocessor replay here. The coprocessor is for each WALEdit 5468 // instead of a KeyValue. 5469 if (coprocessorHost != null) { 5470 status.setStatus("Running pre-WAL-restore hook in coprocessors"); 5471 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) { 5472 // if bypass this wal entry, ignore it ... 5473 continue; 5474 } 5475 } 5476 boolean checkRowWithinBoundary = false; 5477 // Check this edit is for this region. 5478 if (!Bytes.equals(key.getEncodedRegionName(), 5479 this.getRegionInfo().getEncodedNameAsBytes())) { 5480 checkRowWithinBoundary = true; 5481 } 5482 5483 boolean flush = false; 5484 MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing(); 5485 for (Cell cell: val.getCells()) { 5486 // Check this edit is for me. Also, guard against writing the special 5487 // METACOLUMN info such as HBASE::CACHEFLUSH entries 5488 if (WALEdit.isMetaEditFamily(cell)) { 5489 // if region names don't match, skipp replaying compaction marker 5490 if (!checkRowWithinBoundary) { 5491 //this is a special edit, we should handle it 5492 CompactionDescriptor compaction = WALEdit.getCompaction(cell); 5493 if (compaction != null) { 5494 //replay the compaction 5495 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE); 5496 } 5497 } 5498 skippedEdits++; 5499 continue; 5500 } 5501 // Figure which store the edit is meant for. 5502 if (store == null || !CellUtil.matchingFamily(cell, 5503 store.getColumnFamilyDescriptor().getName())) { 5504 store = getStore(cell); 5505 } 5506 if (store == null) { 5507 // This should never happen. Perhaps schema was changed between 5508 // crash and redeploy? 5509 LOG.warn("No family for cell {} in region {}", cell, this); 5510 skippedEdits++; 5511 continue; 5512 } 5513 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(), 5514 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) { 5515 LOG.warn("Row of {} is not within region boundary for region {}", cell, this); 5516 skippedEdits++; 5517 continue; 5518 } 5519 // Now, figure if we should skip this edit. 5520 if (key.getSequenceId() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor() 5521 .getName())) { 5522 skippedEdits++; 5523 continue; 5524 } 5525 PrivateCellUtil.setSequenceId(cell, currentReplaySeqId); 5526 5527 restoreEdit(store, cell, memStoreSizing); 5528 editsCount++; 5529 } 5530 MemStoreSize mss = memStoreSizing.getMemStoreSize(); 5531 incMemStoreSize(mss); 5532 flush = isFlushSize(this.memStoreSizing.getMemStoreSize()); 5533 if (flush) { 5534 internalFlushcache(null, currentEditSeqId, stores.values(), status, false, 5535 FlushLifeCycleTracker.DUMMY); 5536 } 5537 5538 if (coprocessorHost != null) { 5539 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val); 5540 } 5541 } 5542 5543 if (coprocessorHost != null) { 5544 coprocessorHost.postReplayWALs(this.getRegionInfo(), edits); 5545 } 5546 } catch (EOFException eof) { 5547 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits); 5548 msg = "EnLongAddered EOF. Most likely due to Master failure during " 5549 + "wal splitting, so we have this data in another edit. Continuing, but renaming " 5550 + edits + " as " + p + " for region " + this; 5551 LOG.warn(msg, eof); 5552 status.abort(msg); 5553 } catch (IOException ioe) { 5554 // If the IOE resulted from bad file format, 5555 // then this problem is idempotent and retrying won't help 5556 if (ioe.getCause() instanceof ParseException) { 5557 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits); 5558 msg = "File corruption enLongAddered! " + 5559 "Continuing, but renaming " + edits + " as " + p; 5560 LOG.warn(msg, ioe); 5561 status.setStatus(msg); 5562 } else { 5563 status.abort(StringUtils.stringifyException(ioe)); 5564 // other IO errors may be transient (bad network connection, 5565 // checksum exception on one datanode, etc). throw & retry 5566 throw ioe; 5567 } 5568 } 5569 if (reporter != null && !reported_once) { 5570 reporter.progress(); 5571 } 5572 msg = "Applied " + editsCount + ", skipped " + skippedEdits + 5573 ", firstSequenceIdInLog=" + firstSeqIdInLog + 5574 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits; 5575 status.markComplete(msg); 5576 LOG.debug(msg); 5577 return currentEditSeqId; 5578 } finally { 5579 status.cleanup(); 5580 if (reader != null) { 5581 reader.close(); 5582 } 5583 } 5584 } 5585 5586 /** 5587 * Call to complete a compaction. Its for the case where we find in the WAL a compaction 5588 * that was not finished. We could find one recovering a WAL after a regionserver crash. 5589 * See HBASE-2331. 5590 */ 5591 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles, 5592 boolean removeFiles, long replaySeqId) 5593 throws IOException { 5594 try { 5595 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(), 5596 "Compaction marker from WAL ", compaction); 5597 } catch (WrongRegionException wre) { 5598 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 5599 // skip the compaction marker since it is not for this region 5600 return; 5601 } 5602 throw wre; 5603 } 5604 5605 synchronized (writestate) { 5606 if (replaySeqId < lastReplayedOpenRegionSeqId) { 5607 LOG.warn(getRegionInfo().getEncodedName() + " : " 5608 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction) 5609 + " because its sequence id " + replaySeqId + " is smaller than this regions " 5610 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId); 5611 return; 5612 } 5613 if (replaySeqId < lastReplayedCompactionSeqId) { 5614 LOG.warn(getRegionInfo().getEncodedName() + " : " 5615 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction) 5616 + " because its sequence id " + replaySeqId + " is smaller than this regions " 5617 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId); 5618 return; 5619 } else { 5620 lastReplayedCompactionSeqId = replaySeqId; 5621 } 5622 5623 if (LOG.isDebugEnabled()) { 5624 LOG.debug(getRegionInfo().getEncodedName() + " : " 5625 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction) 5626 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId=" 5627 + lastReplayedOpenRegionSeqId); 5628 } 5629 5630 startRegionOperation(Operation.REPLAY_EVENT); 5631 try { 5632 HStore store = this.getStore(compaction.getFamilyName().toByteArray()); 5633 if (store == null) { 5634 LOG.warn(getRegionInfo().getEncodedName() + " : " 5635 + "Found Compaction WAL edit for deleted family:" 5636 + Bytes.toString(compaction.getFamilyName().toByteArray())); 5637 return; 5638 } 5639 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles); 5640 logRegionFiles(); 5641 } catch (FileNotFoundException ex) { 5642 LOG.warn(getRegionInfo().getEncodedName() + " : " 5643 + "At least one of the store files in compaction: " 5644 + TextFormat.shortDebugString(compaction) 5645 + " doesn't exist any more. Skip loading the file(s)", ex); 5646 } finally { 5647 closeRegionOperation(Operation.REPLAY_EVENT); 5648 } 5649 } 5650 } 5651 5652 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException { 5653 checkTargetRegion(flush.getEncodedRegionName().toByteArray(), 5654 "Flush marker from WAL ", flush); 5655 5656 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 5657 return; // if primary nothing to do 5658 } 5659 5660 if (LOG.isDebugEnabled()) { 5661 LOG.debug(getRegionInfo().getEncodedName() + " : " 5662 + "Replaying flush marker " + TextFormat.shortDebugString(flush)); 5663 } 5664 5665 startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close 5666 try { 5667 FlushAction action = flush.getAction(); 5668 switch (action) { 5669 case START_FLUSH: 5670 replayWALFlushStartMarker(flush); 5671 break; 5672 case COMMIT_FLUSH: 5673 replayWALFlushCommitMarker(flush); 5674 break; 5675 case ABORT_FLUSH: 5676 replayWALFlushAbortMarker(flush); 5677 break; 5678 case CANNOT_FLUSH: 5679 replayWALFlushCannotFlushMarker(flush, replaySeqId); 5680 break; 5681 default: 5682 LOG.warn(getRegionInfo().getEncodedName() + " : " + 5683 "Received a flush event with unknown action, ignoring. " + 5684 TextFormat.shortDebugString(flush)); 5685 break; 5686 } 5687 5688 logRegionFiles(); 5689 } finally { 5690 closeRegionOperation(Operation.REPLAY_EVENT); 5691 } 5692 } 5693 5694 /** Replay the flush marker from primary region by creating a corresponding snapshot of 5695 * the store memstores, only if the memstores do not have a higher seqId from an earlier wal 5696 * edit (because the events may be coming out of order). 5697 */ 5698 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException { 5699 long flushSeqId = flush.getFlushSequenceNumber(); 5700 5701 HashSet<HStore> storesToFlush = new HashSet<>(); 5702 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) { 5703 byte[] family = storeFlush.getFamilyName().toByteArray(); 5704 HStore store = getStore(family); 5705 if (store == null) { 5706 LOG.warn(getRegionInfo().getEncodedName() + " : " 5707 + "Received a flush start marker from primary, but the family is not found. Ignoring" 5708 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush)); 5709 continue; 5710 } 5711 storesToFlush.add(store); 5712 } 5713 5714 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this); 5715 5716 // we will use writestate as a coarse-grain lock for all the replay events 5717 // (flush, compaction, region open etc) 5718 synchronized (writestate) { 5719 try { 5720 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) { 5721 LOG.warn(getRegionInfo().getEncodedName() + " : " 5722 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush) 5723 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 5724 + " of " + lastReplayedOpenRegionSeqId); 5725 return null; 5726 } 5727 if (numMutationsWithoutWAL.sum() > 0) { 5728 numMutationsWithoutWAL.reset(); 5729 dataInMemoryWithoutWAL.reset(); 5730 } 5731 5732 if (!writestate.flushing) { 5733 // we do not have an active snapshot and corresponding this.prepareResult. This means 5734 // we can just snapshot our memstores and continue as normal. 5735 5736 // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal 5737 PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId, 5738 storesToFlush, status, false, FlushLifeCycleTracker.DUMMY); 5739 if (prepareResult.result == null) { 5740 // save the PrepareFlushResult so that we can use it later from commit flush 5741 this.writestate.flushing = true; 5742 this.prepareFlushResult = prepareResult; 5743 status.markComplete("Flush prepare successful"); 5744 if (LOG.isDebugEnabled()) { 5745 LOG.debug(getRegionInfo().getEncodedName() + " : " 5746 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber()); 5747 } 5748 } else { 5749 // special case empty memstore. We will still save the flush result in this case, since 5750 // our memstore ie empty, but the primary is still flushing 5751 if (prepareResult.getResult().getResult() == 5752 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { 5753 this.writestate.flushing = true; 5754 this.prepareFlushResult = prepareResult; 5755 if (LOG.isDebugEnabled()) { 5756 LOG.debug(getRegionInfo().getEncodedName() + " : " 5757 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber()); 5758 } 5759 } 5760 status.abort("Flush prepare failed with " + prepareResult.result); 5761 // nothing much to do. prepare flush failed because of some reason. 5762 } 5763 return prepareResult; 5764 } else { 5765 // we already have an active snapshot. 5766 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) { 5767 // They define the same flush. Log and continue. 5768 LOG.warn(getRegionInfo().getEncodedName() + " : " 5769 + "Received a flush prepare marker with the same seqId: " + 5770 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 5771 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 5772 // ignore 5773 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) { 5774 // We received a flush with a smaller seqNum than what we have prepared. We can only 5775 // ignore this prepare flush request. 5776 LOG.warn(getRegionInfo().getEncodedName() + " : " 5777 + "Received a flush prepare marker with a smaller seqId: " + 5778 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 5779 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 5780 // ignore 5781 } else { 5782 // We received a flush with a larger seqNum than what we have prepared 5783 LOG.warn(getRegionInfo().getEncodedName() + " : " 5784 + "Received a flush prepare marker with a larger seqId: " + 5785 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 5786 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 5787 // We do not have multiple active snapshots in the memstore or a way to merge current 5788 // memstore snapshot with the contents and resnapshot for now. We cannot take 5789 // another snapshot and drop the previous one because that will cause temporary 5790 // data loss in the secondary. So we ignore this for now, deferring the resolution 5791 // to happen when we see the corresponding flush commit marker. If we have a memstore 5792 // snapshot with x, and later received another prepare snapshot with y (where x < y), 5793 // when we see flush commit for y, we will drop snapshot for x, and can also drop all 5794 // the memstore edits if everything in memstore is < y. This is the usual case for 5795 // RS crash + recovery where we might see consequtive prepare flush wal markers. 5796 // Otherwise, this will cause more memory to be used in secondary replica until a 5797 // further prapare + commit flush is seen and replayed. 5798 } 5799 } 5800 } finally { 5801 status.cleanup(); 5802 writestate.notifyAll(); 5803 } 5804 } 5805 return null; 5806 } 5807 5808 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", 5809 justification="Intentional; post memstore flush") 5810 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException { 5811 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this); 5812 5813 // check whether we have the memstore snapshot with the corresponding seqId. Replay to 5814 // secondary region replicas are in order, except for when the region moves or then the 5815 // region server crashes. In those cases, we may receive replay requests out of order from 5816 // the original seqIds. 5817 synchronized (writestate) { 5818 try { 5819 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) { 5820 LOG.warn(getRegionInfo().getEncodedName() + " : " 5821 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush) 5822 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 5823 + " of " + lastReplayedOpenRegionSeqId); 5824 return; 5825 } 5826 5827 if (writestate.flushing) { 5828 PrepareFlushResult prepareFlushResult = this.prepareFlushResult; 5829 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) { 5830 if (LOG.isDebugEnabled()) { 5831 LOG.debug(getRegionInfo().getEncodedName() + " : " 5832 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber() 5833 + " and a previous prepared snapshot was found"); 5834 } 5835 // This is the regular case where we received commit flush after prepare flush 5836 // corresponding to the same seqId. 5837 replayFlushInStores(flush, prepareFlushResult, true); 5838 5839 // Set down the memstore size by amount of flush. 5840 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize()); 5841 this.prepareFlushResult = null; 5842 writestate.flushing = false; 5843 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) { 5844 // This should not happen normally. However, lets be safe and guard against these cases 5845 // we received a flush commit with a smaller seqId than what we have prepared 5846 // we will pick the flush file up from this commit (if we have not seen it), but we 5847 // will not drop the memstore 5848 LOG.warn(getRegionInfo().getEncodedName() + " : " 5849 + "Received a flush commit marker with smaller seqId: " 5850 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " 5851 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping" 5852 +" prepared memstore snapshot"); 5853 replayFlushInStores(flush, prepareFlushResult, false); 5854 5855 // snapshot is not dropped, so memstore sizes should not be decremented 5856 // we still have the prepared snapshot, flushing should still be true 5857 } else { 5858 // This should not happen normally. However, lets be safe and guard against these cases 5859 // we received a flush commit with a larger seqId than what we have prepared 5860 // we will pick the flush file for this. We will also obtain the updates lock and 5861 // look for contents of the memstore to see whether we have edits after this seqId. 5862 // If not, we will drop all the memstore edits and the snapshot as well. 5863 LOG.warn(getRegionInfo().getEncodedName() + " : " 5864 + "Received a flush commit marker with larger seqId: " 5865 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " + 5866 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared" 5867 +" memstore snapshot"); 5868 5869 replayFlushInStores(flush, prepareFlushResult, true); 5870 5871 // Set down the memstore size by amount of flush. 5872 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize()); 5873 5874 // Inspect the memstore contents to see whether the memstore contains only edits 5875 // with seqId smaller than the flush seqId. If so, we can discard those edits. 5876 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); 5877 5878 this.prepareFlushResult = null; 5879 writestate.flushing = false; 5880 } 5881 // If we were waiting for observing a flush or region opening event for not showing 5882 // partial data after a secondary region crash, we can allow reads now. We can only make 5883 // sure that we are not showing partial data (for example skipping some previous edits) 5884 // until we observe a full flush start and flush commit. So if we were not able to find 5885 // a previous flush we will not enable reads now. 5886 this.setReadsEnabled(true); 5887 } else { 5888 LOG.warn(getRegionInfo().getEncodedName() + " : " 5889 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber() 5890 + ", but no previous prepared snapshot was found"); 5891 // There is no corresponding prepare snapshot from before. 5892 // We will pick up the new flushed file 5893 replayFlushInStores(flush, null, false); 5894 5895 // Inspect the memstore contents to see whether the memstore contains only edits 5896 // with seqId smaller than the flush seqId. If so, we can discard those edits. 5897 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); 5898 } 5899 5900 status.markComplete("Flush commit successful"); 5901 5902 // Update the last flushed sequence id for region. 5903 this.maxFlushedSeqId = flush.getFlushSequenceNumber(); 5904 5905 // advance the mvcc read point so that the new flushed file is visible. 5906 mvcc.advanceTo(flush.getFlushSequenceNumber()); 5907 5908 } catch (FileNotFoundException ex) { 5909 LOG.warn(getRegionInfo().getEncodedName() + " : " 5910 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush) 5911 + " doesn't exist any more. Skip loading the file(s)", ex); 5912 } 5913 finally { 5914 status.cleanup(); 5915 writestate.notifyAll(); 5916 } 5917 } 5918 5919 // C. Finally notify anyone waiting on memstore to clear: 5920 // e.g. checkResources(). 5921 synchronized (this) { 5922 notifyAll(); // FindBugs NN_NAKED_NOTIFY 5923 } 5924 } 5925 5926 /** 5927 * Replays the given flush descriptor by opening the flush files in stores and dropping the 5928 * memstore snapshots if requested. 5929 * @param flush 5930 * @param prepareFlushResult 5931 * @param dropMemstoreSnapshot 5932 * @throws IOException 5933 */ 5934 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult, 5935 boolean dropMemstoreSnapshot) 5936 throws IOException { 5937 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) { 5938 byte[] family = storeFlush.getFamilyName().toByteArray(); 5939 HStore store = getStore(family); 5940 if (store == null) { 5941 LOG.warn(getRegionInfo().getEncodedName() + " : " 5942 + "Received a flush commit marker from primary, but the family is not found." 5943 + "Ignoring StoreFlushDescriptor:" + storeFlush); 5944 continue; 5945 } 5946 List<String> flushFiles = storeFlush.getFlushOutputList(); 5947 StoreFlushContext ctx = null; 5948 long startTime = EnvironmentEdgeManager.currentTime(); 5949 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) { 5950 ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY); 5951 } else { 5952 ctx = prepareFlushResult.storeFlushCtxs.get(family); 5953 startTime = prepareFlushResult.startTime; 5954 } 5955 5956 if (ctx == null) { 5957 LOG.warn(getRegionInfo().getEncodedName() + " : " 5958 + "Unexpected: flush commit marker received from store " 5959 + Bytes.toString(family) + " but no associated flush context. Ignoring"); 5960 continue; 5961 } 5962 5963 ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush 5964 5965 // Record latest flush time 5966 this.lastStoreFlushTimeMap.put(store, startTime); 5967 } 5968 } 5969 5970 private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException { 5971 Path regionDir = fs.getRegionDir(); 5972 long maxSeqId = -1; 5973 for (HStore store : stores) { 5974 String familyName = store.getColumnFamilyName(); 5975 FileStatus[] files = 5976 WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName); 5977 if (files != null && files.length != 0) { 5978 for (FileStatus file : files) { 5979 Path filePath = file.getPath(); 5980 // If file length is zero then delete it 5981 if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) { 5982 continue; 5983 } 5984 try { 5985 HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath()); 5986 maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID()); 5987 } catch (IOException e) { 5988 handleException(fs.getFileSystem(), filePath, e); 5989 continue; 5990 } 5991 } 5992 if (this.rsServices != null && store.needsCompaction()) { 5993 this.rsServices.getCompactionRequestor() 5994 .requestCompaction(this, store, "load recovered hfiles request compaction", 5995 Store.PRIORITY_USER + 1, CompactionLifeCycleTracker.DUMMY, null); 5996 } 5997 } 5998 } 5999 return maxSeqId; 6000 } 6001 6002 /** 6003 * Be careful, this method will drop all data in the memstore of this region. 6004 * Currently, this method is used to drop memstore to prevent memory leak 6005 * when replaying recovered.edits while opening region. 6006 */ 6007 public MemStoreSize dropMemStoreContents() throws IOException { 6008 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing(); 6009 this.updatesLock.writeLock().lock(); 6010 try { 6011 for (HStore s : stores.values()) { 6012 MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM); 6013 LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region " 6014 + this.getRegionInfo().getRegionNameAsString() 6015 + " , dropped memstoresize: [" + memStoreSize + " }"); 6016 totalFreedSize.incMemStoreSize(memStoreSize); 6017 } 6018 return totalFreedSize.getMemStoreSize(); 6019 } finally { 6020 this.updatesLock.writeLock().unlock(); 6021 } 6022 } 6023 6024 /** 6025 * Drops the memstore contents after replaying a flush descriptor or region open event replay 6026 * if the memstore edits have seqNums smaller than the given seq id 6027 * @throws IOException 6028 */ 6029 private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException { 6030 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing(); 6031 this.updatesLock.writeLock().lock(); 6032 try { 6033 6034 long currentSeqId = mvcc.getReadPoint(); 6035 if (seqId >= currentSeqId) { 6036 // then we can drop the memstore contents since everything is below this seqId 6037 LOG.info(getRegionInfo().getEncodedName() + " : " 6038 + "Dropping memstore contents as well since replayed flush seqId: " 6039 + seqId + " is greater than current seqId:" + currentSeqId); 6040 6041 // Prepare flush (take a snapshot) and then abort (drop the snapshot) 6042 if (store == null) { 6043 for (HStore s : stores.values()) { 6044 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId)); 6045 } 6046 } else { 6047 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId)); 6048 } 6049 } else { 6050 LOG.info(getRegionInfo().getEncodedName() + " : " 6051 + "Not dropping memstore contents since replayed flush seqId: " 6052 + seqId + " is smaller than current seqId:" + currentSeqId); 6053 } 6054 } finally { 6055 this.updatesLock.writeLock().unlock(); 6056 } 6057 return totalFreedSize.getMemStoreSize(); 6058 } 6059 6060 private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId) 6061 throws IOException { 6062 MemStoreSize flushableSize = s.getFlushableSize(); 6063 this.decrMemStoreSize(flushableSize); 6064 StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY); 6065 ctx.prepare(); 6066 ctx.abort(); 6067 return flushableSize; 6068 } 6069 6070 private void replayWALFlushAbortMarker(FlushDescriptor flush) { 6071 // nothing to do for now. A flush abort will cause a RS abort which means that the region 6072 // will be opened somewhere else later. We will see the region open event soon, and replaying 6073 // that will drop the snapshot 6074 } 6075 6076 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) { 6077 synchronized (writestate) { 6078 if (this.lastReplayedOpenRegionSeqId > replaySeqId) { 6079 LOG.warn(getRegionInfo().getEncodedName() + " : " 6080 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush) 6081 + " because its sequence id " + replaySeqId + " is smaller than this regions " 6082 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId); 6083 return; 6084 } 6085 6086 // If we were waiting for observing a flush or region opening event for not showing partial 6087 // data after a secondary region crash, we can allow reads now. This event means that the 6088 // primary was not able to flush because memstore is empty when we requested flush. By the 6089 // time we observe this, we are guaranteed to have up to date seqId with our previous 6090 // assignment. 6091 this.setReadsEnabled(true); 6092 } 6093 } 6094 6095 PrepareFlushResult getPrepareFlushResult() { 6096 return prepareFlushResult; 6097 } 6098 6099 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY", 6100 justification="Intentional; cleared the memstore") 6101 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException { 6102 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(), 6103 "RegionEvent marker from WAL ", regionEvent); 6104 6105 startRegionOperation(Operation.REPLAY_EVENT); 6106 try { 6107 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6108 return; // if primary nothing to do 6109 } 6110 6111 if (regionEvent.getEventType() == EventType.REGION_CLOSE) { 6112 // nothing to do on REGION_CLOSE for now. 6113 return; 6114 } 6115 if (regionEvent.getEventType() != EventType.REGION_OPEN) { 6116 LOG.warn(getRegionInfo().getEncodedName() + " : " 6117 + "Unknown region event received, ignoring :" 6118 + TextFormat.shortDebugString(regionEvent)); 6119 return; 6120 } 6121 6122 if (LOG.isDebugEnabled()) { 6123 LOG.debug(getRegionInfo().getEncodedName() + " : " 6124 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent)); 6125 } 6126 6127 // we will use writestate as a coarse-grain lock for all the replay events 6128 synchronized (writestate) { 6129 // Replication can deliver events out of order when primary region moves or the region 6130 // server crashes, since there is no coordination between replication of different wal files 6131 // belonging to different region servers. We have to safe guard against this case by using 6132 // region open event's seqid. Since this is the first event that the region puts (after 6133 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit 6134 // smaller than this seqId 6135 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) { 6136 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber(); 6137 } else { 6138 LOG.warn(getRegionInfo().getEncodedName() + " : " 6139 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent) 6140 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 6141 + " of " + lastReplayedOpenRegionSeqId); 6142 return; 6143 } 6144 6145 // region open lists all the files that the region has at the time of the opening. Just pick 6146 // all the files and drop prepared flushes and empty memstores 6147 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) { 6148 // stores of primary may be different now 6149 byte[] family = storeDescriptor.getFamilyName().toByteArray(); 6150 HStore store = getStore(family); 6151 if (store == null) { 6152 LOG.warn(getRegionInfo().getEncodedName() + " : " 6153 + "Received a region open marker from primary, but the family is not found. " 6154 + "Ignoring. StoreDescriptor:" + storeDescriptor); 6155 continue; 6156 } 6157 6158 long storeSeqId = store.getMaxSequenceId().orElse(0L); 6159 List<String> storeFiles = storeDescriptor.getStoreFileList(); 6160 try { 6161 store.refreshStoreFiles(storeFiles); // replace the files with the new ones 6162 } catch (FileNotFoundException ex) { 6163 LOG.warn(getRegionInfo().getEncodedName() + " : " 6164 + "At least one of the store files: " + storeFiles 6165 + " doesn't exist any more. Skip loading the file(s)", ex); 6166 continue; 6167 } 6168 if (store.getMaxSequenceId().orElse(0L) != storeSeqId) { 6169 // Record latest flush time if we picked up new files 6170 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime()); 6171 } 6172 6173 if (writestate.flushing) { 6174 // only drop memstore snapshots if they are smaller than last flush for the store 6175 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) { 6176 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? 6177 null : this.prepareFlushResult.storeFlushCtxs.get(family); 6178 if (ctx != null) { 6179 MemStoreSize mss = store.getFlushableSize(); 6180 ctx.abort(); 6181 this.decrMemStoreSize(mss); 6182 this.prepareFlushResult.storeFlushCtxs.remove(family); 6183 } 6184 } 6185 } 6186 6187 // Drop the memstore contents if they are now smaller than the latest seen flushed file 6188 dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store); 6189 if (storeSeqId > this.maxFlushedSeqId) { 6190 this.maxFlushedSeqId = storeSeqId; 6191 } 6192 } 6193 6194 // if all stores ended up dropping their snapshots, we can safely drop the 6195 // prepareFlushResult 6196 dropPrepareFlushIfPossible(); 6197 6198 // advance the mvcc read point so that the new flushed file is visible. 6199 mvcc.await(); 6200 6201 // If we were waiting for observing a flush or region opening event for not showing partial 6202 // data after a secondary region crash, we can allow reads now. 6203 this.setReadsEnabled(true); 6204 6205 // C. Finally notify anyone waiting on memstore to clear: 6206 // e.g. checkResources(). 6207 synchronized (this) { 6208 notifyAll(); // FindBugs NN_NAKED_NOTIFY 6209 } 6210 } 6211 logRegionFiles(); 6212 } finally { 6213 closeRegionOperation(Operation.REPLAY_EVENT); 6214 } 6215 } 6216 6217 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException { 6218 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(), 6219 "BulkLoad marker from WAL ", bulkLoadEvent); 6220 6221 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6222 return; // if primary nothing to do 6223 } 6224 6225 if (LOG.isDebugEnabled()) { 6226 LOG.debug(getRegionInfo().getEncodedName() + " : " 6227 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent)); 6228 } 6229 // check if multiple families involved 6230 boolean multipleFamilies = false; 6231 byte[] family = null; 6232 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) { 6233 byte[] fam = storeDescriptor.getFamilyName().toByteArray(); 6234 if (family == null) { 6235 family = fam; 6236 } else if (!Bytes.equals(family, fam)) { 6237 multipleFamilies = true; 6238 break; 6239 } 6240 } 6241 6242 startBulkRegionOperation(multipleFamilies); 6243 try { 6244 // we will use writestate as a coarse-grain lock for all the replay events 6245 synchronized (writestate) { 6246 // Replication can deliver events out of order when primary region moves or the region 6247 // server crashes, since there is no coordination between replication of different wal files 6248 // belonging to different region servers. We have to safe guard against this case by using 6249 // region open event's seqid. Since this is the first event that the region puts (after 6250 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit 6251 // smaller than this seqId 6252 if (bulkLoadEvent.getBulkloadSeqNum() >= 0 6253 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) { 6254 LOG.warn(getRegionInfo().getEncodedName() + " : " 6255 + "Skipping replaying bulkload event :" 6256 + TextFormat.shortDebugString(bulkLoadEvent) 6257 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId" 6258 + " =" + lastReplayedOpenRegionSeqId); 6259 6260 return; 6261 } 6262 6263 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) { 6264 // stores of primary may be different now 6265 family = storeDescriptor.getFamilyName().toByteArray(); 6266 HStore store = getStore(family); 6267 if (store == null) { 6268 LOG.warn(getRegionInfo().getEncodedName() + " : " 6269 + "Received a bulk load marker from primary, but the family is not found. " 6270 + "Ignoring. StoreDescriptor:" + storeDescriptor); 6271 continue; 6272 } 6273 6274 List<String> storeFiles = storeDescriptor.getStoreFileList(); 6275 for (String storeFile : storeFiles) { 6276 StoreFileInfo storeFileInfo = null; 6277 try { 6278 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile); 6279 store.bulkLoadHFile(storeFileInfo); 6280 } catch(FileNotFoundException ex) { 6281 LOG.warn(getRegionInfo().getEncodedName() + " : " 6282 + ((storeFileInfo != null) ? storeFileInfo.toString() : 6283 (new Path(Bytes.toString(family), storeFile)).toString()) 6284 + " doesn't exist any more. Skip loading the file"); 6285 } 6286 } 6287 } 6288 } 6289 if (bulkLoadEvent.getBulkloadSeqNum() > 0) { 6290 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum()); 6291 } 6292 } finally { 6293 closeBulkRegionOperation(); 6294 } 6295 } 6296 6297 /** 6298 * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult 6299 */ 6300 private void dropPrepareFlushIfPossible() { 6301 if (writestate.flushing) { 6302 boolean canDrop = true; 6303 if (prepareFlushResult.storeFlushCtxs != null) { 6304 for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs 6305 .entrySet()) { 6306 HStore store = getStore(entry.getKey()); 6307 if (store == null) { 6308 continue; 6309 } 6310 if (store.getSnapshotSize().getDataSize() > 0) { 6311 canDrop = false; 6312 break; 6313 } 6314 } 6315 } 6316 6317 // this means that all the stores in the region has finished flushing, but the WAL marker 6318 // may not have been written or we did not receive it yet. 6319 if (canDrop) { 6320 writestate.flushing = false; 6321 this.prepareFlushResult = null; 6322 } 6323 } 6324 } 6325 6326 @Override 6327 public boolean refreshStoreFiles() throws IOException { 6328 return refreshStoreFiles(false); 6329 } 6330 6331 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY", 6332 justification = "Notify is about post replay. Intentional") 6333 protected boolean refreshStoreFiles(boolean force) throws IOException { 6334 if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6335 return false; // if primary nothing to do 6336 } 6337 6338 if (LOG.isDebugEnabled()) { 6339 LOG.debug(getRegionInfo().getEncodedName() + " : " 6340 + "Refreshing store files to see whether we can free up memstore"); 6341 } 6342 6343 long totalFreedDataSize = 0; 6344 6345 long smallestSeqIdInStores = Long.MAX_VALUE; 6346 6347 startRegionOperation(); // obtain region close lock 6348 try { 6349 Map<HStore, Long> map = new HashMap<>(); 6350 synchronized (writestate) { 6351 for (HStore store : stores.values()) { 6352 // TODO: some stores might see new data from flush, while others do not which 6353 // MIGHT break atomic edits across column families. 6354 long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L); 6355 6356 // refresh the store files. This is similar to observing a region open wal marker. 6357 store.refreshStoreFiles(); 6358 6359 long storeSeqId = store.getMaxSequenceId().orElse(0L); 6360 if (storeSeqId < smallestSeqIdInStores) { 6361 smallestSeqIdInStores = storeSeqId; 6362 } 6363 6364 // see whether we can drop the memstore or the snapshot 6365 if (storeSeqId > maxSeqIdBefore) { 6366 if (writestate.flushing) { 6367 // only drop memstore snapshots if they are smaller than last flush for the store 6368 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) { 6369 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? 6370 null : this.prepareFlushResult.storeFlushCtxs.get( 6371 store.getColumnFamilyDescriptor().getName()); 6372 if (ctx != null) { 6373 MemStoreSize mss = store.getFlushableSize(); 6374 ctx.abort(); 6375 this.decrMemStoreSize(mss); 6376 this.prepareFlushResult.storeFlushCtxs. 6377 remove(store.getColumnFamilyDescriptor().getName()); 6378 totalFreedDataSize += mss.getDataSize(); 6379 } 6380 } 6381 } 6382 6383 map.put(store, storeSeqId); 6384 } 6385 } 6386 6387 // if all stores ended up dropping their snapshots, we can safely drop the 6388 // prepareFlushResult 6389 dropPrepareFlushIfPossible(); 6390 6391 // advance the mvcc read point so that the new flushed files are visible. 6392 // either greater than flush seq number or they were already picked up via flush. 6393 for (HStore s : stores.values()) { 6394 mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L)); 6395 } 6396 6397 6398 // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely 6399 // skip all edits that are to be replayed in the future with that has a smaller seqId 6400 // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits 6401 // that we have picked the flush files for 6402 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) { 6403 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores; 6404 } 6405 } 6406 if (!map.isEmpty()) { 6407 for (Map.Entry<HStore, Long> entry : map.entrySet()) { 6408 // Drop the memstore contents if they are now smaller than the latest seen flushed file 6409 totalFreedDataSize += dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey()) 6410 .getDataSize(); 6411 } 6412 } 6413 // C. Finally notify anyone waiting on memstore to clear: 6414 // e.g. checkResources(). 6415 synchronized (this) { 6416 notifyAll(); // FindBugs NN_NAKED_NOTIFY 6417 } 6418 return totalFreedDataSize > 0; 6419 } finally { 6420 closeRegionOperation(); 6421 } 6422 } 6423 6424 private void logRegionFiles() { 6425 if (LOG.isTraceEnabled()) { 6426 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: "); 6427 stores.values().stream().filter(s -> s.getStorefiles() != null) 6428 .flatMap(s -> s.getStorefiles().stream()) 6429 .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf)); 6430 } 6431 } 6432 6433 /** Checks whether the given regionName is either equal to our region, or that 6434 * the regionName is the primary region to our corresponding range for the secondary replica. 6435 */ 6436 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload) 6437 throws WrongRegionException { 6438 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) { 6439 return; 6440 } 6441 6442 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) && 6443 Bytes.equals(encodedRegionName, 6444 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) { 6445 return; 6446 } 6447 6448 throw new WrongRegionException(exceptionMsg + payload 6449 + " targetted for region " + Bytes.toStringBinary(encodedRegionName) 6450 + " does not match this region: " + this.getRegionInfo()); 6451 } 6452 6453 /** 6454 * Used by tests 6455 * @param s Store to add edit too. 6456 * @param cell Cell to add. 6457 */ 6458 protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) { 6459 s.add(cell, memstoreAccounting); 6460 } 6461 6462 /** 6463 * make sure have been through lease recovery before get file status, so the file length can be 6464 * trusted. 6465 * @param p File to check. 6466 * @return True if file was zero-length (and if so, we'll delete it in here). 6467 * @throws IOException 6468 */ 6469 private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat, 6470 final Path p) throws IOException { 6471 if (stat.getLen() > 0) { 6472 return false; 6473 } 6474 LOG.warn("File " + p + " is zero-length, deleting."); 6475 fs.delete(p, false); 6476 return true; 6477 } 6478 6479 protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup) 6480 throws IOException { 6481 if (family.isMobEnabled()) { 6482 if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) { 6483 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS + 6484 " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY + 6485 " accordingly."); 6486 } 6487 return new HMobStore(this, family, this.conf, warmup); 6488 } 6489 return new HStore(this, family, this.conf, warmup); 6490 } 6491 6492 @Override 6493 public HStore getStore(byte[] column) { 6494 return this.stores.get(column); 6495 } 6496 6497 /** 6498 * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on 6499 * the list. 6500 */ 6501 private HStore getStore(Cell cell) { 6502 return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey())) 6503 .map(e -> e.getValue()).findFirst().orElse(null); 6504 } 6505 6506 @Override 6507 public List<HStore> getStores() { 6508 return new ArrayList<>(stores.values()); 6509 } 6510 6511 @Override 6512 public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException { 6513 List<String> storeFileNames = new ArrayList<>(); 6514 synchronized (closeLock) { 6515 for (byte[] column : columns) { 6516 HStore store = this.stores.get(column); 6517 if (store == null) { 6518 throw new IllegalArgumentException( 6519 "No column family : " + new String(column, StandardCharsets.UTF_8) + " available"); 6520 } 6521 Collection<HStoreFile> storeFiles = store.getStorefiles(); 6522 if (storeFiles == null) { 6523 continue; 6524 } 6525 for (HStoreFile storeFile : storeFiles) { 6526 storeFileNames.add(storeFile.getPath().toString()); 6527 } 6528 6529 logRegionFiles(); 6530 } 6531 } 6532 return storeFileNames; 6533 } 6534 6535 ////////////////////////////////////////////////////////////////////////////// 6536 // Support code 6537 ////////////////////////////////////////////////////////////////////////////// 6538 6539 /** Make sure this is a valid row for the HRegion */ 6540 void checkRow(byte[] row, String op) throws IOException { 6541 if (!rowIsInRange(getRegionInfo(), row)) { 6542 throw new WrongRegionException("Requested row out of range for " + 6543 op + " on HRegion " + this + ", startKey='" + 6544 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" + 6545 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" + 6546 Bytes.toStringBinary(row) + "'"); 6547 } 6548 } 6549 6550 6551 /** 6552 * Get an exclusive ( write lock ) lock on a given row. 6553 * @param row Which row to lock. 6554 * @return A locked RowLock. The lock is exclusive and already aqquired. 6555 * @throws IOException 6556 */ 6557 public RowLock getRowLock(byte[] row) throws IOException { 6558 return getRowLock(row, false); 6559 } 6560 6561 @Override 6562 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException { 6563 checkRow(row, "row lock"); 6564 return getRowLockInternal(row, readLock, null); 6565 } 6566 6567 protected RowLock getRowLockInternal(byte[] row, boolean readLock, final RowLock prevRowLock) 6568 throws IOException { 6569 // create an object to use a a key in the row lock map 6570 HashedBytes rowKey = new HashedBytes(row); 6571 6572 RowLockContext rowLockContext = null; 6573 RowLockImpl result = null; 6574 6575 boolean success = false; 6576 try (TraceScope scope = TraceUtil.createTrace("HRegion.getRowLock")) { 6577 TraceUtil.addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock")); 6578 // Keep trying until we have a lock or error out. 6579 // TODO: do we need to add a time component here? 6580 while (result == null) { 6581 rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey)); 6582 // Now try an get the lock. 6583 // This can fail as 6584 if (readLock) { 6585 // For read lock, if the caller has locked the same row previously, it will not try 6586 // to acquire the same read lock. It simply returns the previous row lock. 6587 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock; 6588 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() == 6589 rowLockContext.readWriteLock.readLock())) { 6590 success = true; 6591 return prevRowLock; 6592 } 6593 result = rowLockContext.newReadLock(); 6594 } else { 6595 result = rowLockContext.newWriteLock(); 6596 } 6597 } 6598 6599 int timeout = rowLockWaitDuration; 6600 boolean reachDeadlineFirst = false; 6601 Optional<RpcCall> call = RpcServer.getCurrentCall(); 6602 if (call.isPresent()) { 6603 long deadline = call.get().getDeadline(); 6604 if (deadline < Long.MAX_VALUE) { 6605 int timeToDeadline = (int) (deadline - System.currentTimeMillis()); 6606 if (timeToDeadline <= this.rowLockWaitDuration) { 6607 reachDeadlineFirst = true; 6608 timeout = timeToDeadline; 6609 } 6610 } 6611 } 6612 6613 if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) { 6614 TraceUtil.addTimelineAnnotation("Failed to get row lock"); 6615 String message = "Timed out waiting for lock for row: " + rowKey + " in region " 6616 + getRegionInfo().getEncodedName(); 6617 if (reachDeadlineFirst) { 6618 throw new TimeoutIOException(message); 6619 } else { 6620 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request. 6621 throw new IOException(message); 6622 } 6623 } 6624 rowLockContext.setThreadName(Thread.currentThread().getName()); 6625 success = true; 6626 return result; 6627 } catch (InterruptedException ie) { 6628 if (LOG.isDebugEnabled()) { 6629 LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, 6630 getRegionInfo().getRegionNameAsString()); 6631 } 6632 TraceUtil.addTimelineAnnotation("Interrupted exception getting row lock"); 6633 throw throwOnInterrupt(ie); 6634 } catch (Error error) { 6635 // The maximum lock count for read lock is 64K (hardcoded), when this maximum count 6636 // is reached, it will throw out an Error. This Error needs to be caught so it can 6637 // go ahead to process the minibatch with lock acquired. 6638 LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row), 6639 getRegionInfo().getRegionNameAsString(), error); 6640 IOException ioe = new IOException(error); 6641 TraceUtil.addTimelineAnnotation("Error getting row lock"); 6642 throw ioe; 6643 } finally { 6644 // Clean up the counts just in case this was the thing keeping the context alive. 6645 if (!success && rowLockContext != null) { 6646 rowLockContext.cleanUp(); 6647 } 6648 } 6649 } 6650 6651 private void releaseRowLocks(List<RowLock> rowLocks) { 6652 if (rowLocks != null) { 6653 for (RowLock rowLock : rowLocks) { 6654 rowLock.release(); 6655 } 6656 rowLocks.clear(); 6657 } 6658 } 6659 6660 public int getReadLockCount() { 6661 return lock.getReadLockCount(); 6662 } 6663 6664 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() { 6665 return lockedRows; 6666 } 6667 6668 class RowLockContext { 6669 private final HashedBytes row; 6670 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true); 6671 final AtomicBoolean usable = new AtomicBoolean(true); 6672 final AtomicInteger count = new AtomicInteger(0); 6673 final Object lock = new Object(); 6674 private String threadName; 6675 6676 RowLockContext(HashedBytes row) { 6677 this.row = row; 6678 } 6679 6680 RowLockImpl newWriteLock() { 6681 Lock l = readWriteLock.writeLock(); 6682 return getRowLock(l); 6683 } 6684 RowLockImpl newReadLock() { 6685 Lock l = readWriteLock.readLock(); 6686 return getRowLock(l); 6687 } 6688 6689 private RowLockImpl getRowLock(Lock l) { 6690 count.incrementAndGet(); 6691 synchronized (lock) { 6692 if (usable.get()) { 6693 return new RowLockImpl(this, l); 6694 } else { 6695 return null; 6696 } 6697 } 6698 } 6699 6700 void cleanUp() { 6701 long c = count.decrementAndGet(); 6702 if (c <= 0) { 6703 synchronized (lock) { 6704 if (count.get() <= 0 && usable.get()){ // Don't attempt to remove row if already removed 6705 usable.set(false); 6706 RowLockContext removed = lockedRows.remove(row); 6707 assert removed == this: "we should never remove a different context"; 6708 } 6709 } 6710 } 6711 } 6712 6713 public void setThreadName(String threadName) { 6714 this.threadName = threadName; 6715 } 6716 6717 @Override 6718 public String toString() { 6719 return "RowLockContext{" + 6720 "row=" + row + 6721 ", readWriteLock=" + readWriteLock + 6722 ", count=" + count + 6723 ", threadName=" + threadName + 6724 '}'; 6725 } 6726 } 6727 6728 /** 6729 * Class used to represent a lock on a row. 6730 */ 6731 public static class RowLockImpl implements RowLock { 6732 private final RowLockContext context; 6733 private final Lock lock; 6734 6735 public RowLockImpl(RowLockContext context, Lock lock) { 6736 this.context = context; 6737 this.lock = lock; 6738 } 6739 6740 public Lock getLock() { 6741 return lock; 6742 } 6743 6744 public RowLockContext getContext() { 6745 return context; 6746 } 6747 6748 @Override 6749 public void release() { 6750 lock.unlock(); 6751 context.cleanUp(); 6752 } 6753 6754 @Override 6755 public String toString() { 6756 return "RowLockImpl{" + 6757 "context=" + context + 6758 ", lock=" + lock + 6759 '}'; 6760 } 6761 } 6762 6763 /** 6764 * Determines whether multiple column families are present 6765 * Precondition: familyPaths is not null 6766 * 6767 * @param familyPaths List of (column family, hfilePath) 6768 */ 6769 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) { 6770 boolean multipleFamilies = false; 6771 byte[] family = null; 6772 for (Pair<byte[], String> pair : familyPaths) { 6773 byte[] fam = pair.getFirst(); 6774 if (family == null) { 6775 family = fam; 6776 } else if (!Bytes.equals(family, fam)) { 6777 multipleFamilies = true; 6778 break; 6779 } 6780 } 6781 return multipleFamilies; 6782 } 6783 6784 /** 6785 * Attempts to atomically load a group of hfiles. This is critical for loading 6786 * rows with multiple column families atomically. 6787 * 6788 * @param familyPaths List of Pair<byte[] column family, String hfilePath> 6789 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a 6790 * file about to be bulk loaded 6791 * @param assignSeqId 6792 * @return Map from family to List of store file paths if successful, null if failed recoverably 6793 * @throws IOException if failed unrecoverably. 6794 */ 6795 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId, 6796 BulkLoadListener bulkLoadListener) throws IOException { 6797 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false, 6798 null, true); 6799 } 6800 6801 /** 6802 * Listener class to enable callers of 6803 * bulkLoadHFile() to perform any necessary 6804 * pre/post processing of a given bulkload call 6805 */ 6806 public interface BulkLoadListener { 6807 /** 6808 * Called before an HFile is actually loaded 6809 * @param family family being loaded to 6810 * @param srcPath path of HFile 6811 * @return final path to be used for actual loading 6812 * @throws IOException 6813 */ 6814 String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile) 6815 throws IOException; 6816 6817 /** 6818 * Called after a successful HFile load 6819 * @param family family being loaded to 6820 * @param srcPath path of HFile 6821 * @throws IOException 6822 */ 6823 void doneBulkLoad(byte[] family, String srcPath) throws IOException; 6824 6825 /** 6826 * Called after a failed HFile load 6827 * @param family family being loaded to 6828 * @param srcPath path of HFile 6829 * @throws IOException 6830 */ 6831 void failedBulkLoad(byte[] family, String srcPath) throws IOException; 6832 } 6833 6834 /** 6835 * Attempts to atomically load a group of hfiles. This is critical for loading 6836 * rows with multiple column families atomically. 6837 * 6838 * @param familyPaths List of Pair<byte[] column family, String hfilePath> 6839 * @param assignSeqId 6840 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a 6841 * file about to be bulk loaded 6842 * @param copyFile always copy hfiles if true 6843 * @param clusterIds ids from clusters that had already handled the given bulkload event. 6844 * @return Map from family to List of store file paths if successful, null if failed recoverably 6845 * @throws IOException if failed unrecoverably. 6846 */ 6847 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, 6848 boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile, 6849 List<String> clusterIds, boolean replicate) throws IOException { 6850 long seqId = -1; 6851 Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); 6852 Map<String, Long> storeFilesSizes = new HashMap<>(); 6853 Preconditions.checkNotNull(familyPaths); 6854 // we need writeLock for multi-family bulk load 6855 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths)); 6856 boolean isSuccessful = false; 6857 try { 6858 this.writeRequestsCount.increment(); 6859 6860 // There possibly was a split that happened between when the split keys 6861 // were gathered and before the HRegion's write lock was taken. We need 6862 // to validate the HFile region before attempting to bulk load all of them 6863 IOException ioException = null; 6864 List<Pair<byte[], String>> failures = new ArrayList<>(); 6865 for (Pair<byte[], String> p : familyPaths) { 6866 byte[] familyName = p.getFirst(); 6867 String path = p.getSecond(); 6868 6869 HStore store = getStore(familyName); 6870 if (store == null) { 6871 ioException = new org.apache.hadoop.hbase.DoNotRetryIOException( 6872 "No such column family " + Bytes.toStringBinary(familyName)); 6873 } else { 6874 try { 6875 store.assertBulkLoadHFileOk(new Path(path)); 6876 } catch (WrongRegionException wre) { 6877 // recoverable (file doesn't fit in region) 6878 failures.add(p); 6879 } catch (IOException ioe) { 6880 // unrecoverable (hdfs problem) 6881 ioException = ioe; 6882 } 6883 } 6884 6885 // validation failed because of some sort of IO problem. 6886 if (ioException != null) { 6887 LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this, 6888 ioException); 6889 throw ioException; 6890 } 6891 } 6892 // validation failed, bail out before doing anything permanent. 6893 if (failures.size() != 0) { 6894 StringBuilder list = new StringBuilder(); 6895 for (Pair<byte[], String> p : failures) { 6896 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ") 6897 .append(p.getSecond()); 6898 } 6899 // problem when validating 6900 LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family," 6901 + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this); 6902 return null; 6903 } 6904 6905 // We need to assign a sequential ID that's in between two memstores in order to preserve 6906 // the guarantee that all the edits lower than the highest sequential ID from all the 6907 // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is 6908 // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is 6909 // a sequence id that we can be sure is beyond the last hfile written). 6910 if (assignSeqId) { 6911 FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY); 6912 if (fs.isFlushSucceeded()) { 6913 seqId = ((FlushResultImpl)fs).flushSequenceId; 6914 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { 6915 seqId = ((FlushResultImpl)fs).flushSequenceId; 6916 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) { 6917 // CANNOT_FLUSH may mean that a flush is already on-going 6918 // we need to wait for that flush to complete 6919 waitForFlushes(); 6920 } else { 6921 throw new IOException("Could not bulk load with an assigned sequential ID because the "+ 6922 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason); 6923 } 6924 } 6925 6926 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath = 6927 new TreeMap<>(Bytes.BYTES_COMPARATOR); 6928 for (Pair<byte[], String> p : familyPaths) { 6929 byte[] familyName = p.getFirst(); 6930 String path = p.getSecond(); 6931 HStore store = getStore(familyName); 6932 if (!familyWithFinalPath.containsKey(familyName)) { 6933 familyWithFinalPath.put(familyName, new ArrayList<>()); 6934 } 6935 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName); 6936 try { 6937 String finalPath = path; 6938 if (bulkLoadListener != null) { 6939 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile); 6940 } 6941 Pair<Path, Path> pair = store.preBulkLoadHFile(finalPath, seqId); 6942 lst.add(pair); 6943 } catch (IOException ioe) { 6944 // A failure here can cause an atomicity violation that we currently 6945 // cannot recover from since it is likely a failed HDFS operation. 6946 6947 LOG.error("There was a partial failure due to IO when attempting to" + 6948 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe); 6949 if (bulkLoadListener != null) { 6950 try { 6951 bulkLoadListener.failedBulkLoad(familyName, path); 6952 } catch (Exception ex) { 6953 LOG.error("Error while calling failedBulkLoad for family " + 6954 Bytes.toString(familyName) + " with path " + path, ex); 6955 } 6956 } 6957 throw ioe; 6958 } 6959 } 6960 6961 if (this.getCoprocessorHost() != null) { 6962 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) { 6963 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue()); 6964 } 6965 } 6966 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) { 6967 byte[] familyName = entry.getKey(); 6968 for (Pair<Path, Path> p : entry.getValue()) { 6969 String path = p.getFirst().toString(); 6970 Path commitedStoreFile = p.getSecond(); 6971 HStore store = getStore(familyName); 6972 try { 6973 store.bulkLoadHFile(familyName, path, commitedStoreFile); 6974 // Note the size of the store file 6975 try { 6976 FileSystem fs = commitedStoreFile.getFileSystem(baseConf); 6977 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile) 6978 .getLen()); 6979 } catch (IOException e) { 6980 LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e); 6981 storeFilesSizes.put(commitedStoreFile.getName(), 0L); 6982 } 6983 6984 if(storeFiles.containsKey(familyName)) { 6985 storeFiles.get(familyName).add(commitedStoreFile); 6986 } else { 6987 List<Path> storeFileNames = new ArrayList<>(); 6988 storeFileNames.add(commitedStoreFile); 6989 storeFiles.put(familyName, storeFileNames); 6990 } 6991 if (bulkLoadListener != null) { 6992 bulkLoadListener.doneBulkLoad(familyName, path); 6993 } 6994 } catch (IOException ioe) { 6995 // A failure here can cause an atomicity violation that we currently 6996 // cannot recover from since it is likely a failed HDFS operation. 6997 6998 // TODO Need a better story for reverting partial failures due to HDFS. 6999 LOG.error("There was a partial failure due to IO when attempting to" + 7000 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe); 7001 if (bulkLoadListener != null) { 7002 try { 7003 bulkLoadListener.failedBulkLoad(familyName, path); 7004 } catch (Exception ex) { 7005 LOG.error("Error while calling failedBulkLoad for family " + 7006 Bytes.toString(familyName) + " with path " + path, ex); 7007 } 7008 } 7009 throw ioe; 7010 } 7011 } 7012 } 7013 7014 isSuccessful = true; 7015 //request compaction 7016 familyWithFinalPath.keySet().forEach(family -> { 7017 HStore store = getStore(family); 7018 try { 7019 if (this.rsServices != null && store.needsCompaction()) { 7020 this.rsServices.getCompactionRequestor().requestCompaction(this, store, 7021 "bulkload hfiles request compaction", Store.PRIORITY_USER + 1, 7022 CompactionLifeCycleTracker.DUMMY, null); 7023 } 7024 } catch (IOException e) { 7025 LOG.error("bulkload hfiles request compaction error ", e); 7026 } 7027 }); 7028 } finally { 7029 if (wal != null && !storeFiles.isEmpty()) { 7030 // Write a bulk load event for hfiles that are loaded 7031 try { 7032 WALProtos.BulkLoadDescriptor loadDescriptor = 7033 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(), 7034 UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()), 7035 storeFiles, storeFilesSizes, seqId, clusterIds, replicate); 7036 WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(), 7037 loadDescriptor, mvcc); 7038 } catch (IOException ioe) { 7039 if (this.rsServices != null) { 7040 // Have to abort region server because some hfiles has been loaded but we can't write 7041 // the event into WAL 7042 isSuccessful = false; 7043 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe); 7044 } 7045 } 7046 } 7047 7048 closeBulkRegionOperation(); 7049 } 7050 return isSuccessful ? storeFiles : null; 7051 } 7052 7053 @Override 7054 public boolean equals(Object o) { 7055 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(), 7056 ((HRegion) o).getRegionInfo().getRegionName()); 7057 } 7058 7059 @Override 7060 public int hashCode() { 7061 return Bytes.hashCode(getRegionInfo().getRegionName()); 7062 } 7063 7064 @Override 7065 public String toString() { 7066 return getRegionInfo().getRegionNameAsString(); 7067 } 7068 7069 /** 7070 * RegionScannerImpl is used to combine scanners from multiple Stores (aka column families). 7071 */ 7072 class RegionScannerImpl 7073 implements RegionScanner, Shipper, org.apache.hadoop.hbase.ipc.RpcCallback { 7074 // Package local for testability 7075 KeyValueHeap storeHeap = null; 7076 /** Heap of key-values that are not essential for the provided filters and are thus read 7077 * on demand, if on-demand column family loading is enabled.*/ 7078 KeyValueHeap joinedHeap = null; 7079 /** 7080 * If the joined heap data gathering is interrupted due to scan limits, this will 7081 * contain the row for which we are populating the values.*/ 7082 protected Cell joinedContinuationRow = null; 7083 private boolean filterClosed = false; 7084 7085 protected final byte[] stopRow; 7086 protected final boolean includeStopRow; 7087 protected final HRegion region; 7088 protected final CellComparator comparator; 7089 7090 private final long readPt; 7091 private final long maxResultSize; 7092 private final ScannerContext defaultScannerContext; 7093 private final FilterWrapper filter; 7094 7095 @Override 7096 public RegionInfo getRegionInfo() { 7097 return region.getRegionInfo(); 7098 } 7099 7100 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region) 7101 throws IOException { 7102 this(scan, additionalScanners, region, HConstants.NO_NONCE, HConstants.NO_NONCE); 7103 } 7104 7105 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region, 7106 long nonceGroup, long nonce) throws IOException { 7107 this.region = region; 7108 this.maxResultSize = scan.getMaxResultSize(); 7109 if (scan.hasFilter()) { 7110 this.filter = new FilterWrapper(scan.getFilter()); 7111 } else { 7112 this.filter = null; 7113 } 7114 this.comparator = region.getCellComparator(); 7115 /** 7116 * By default, calls to next/nextRaw must enforce the batch limit. Thus, construct a default 7117 * scanner context that can be used to enforce the batch limit in the event that a 7118 * ScannerContext is not specified during an invocation of next/nextRaw 7119 */ 7120 defaultScannerContext = ScannerContext.newBuilder() 7121 .setBatchLimit(scan.getBatch()).build(); 7122 this.stopRow = scan.getStopRow(); 7123 this.includeStopRow = scan.includeStopRow(); 7124 7125 // synchronize on scannerReadPoints so that nobody calculates 7126 // getSmallestReadPoint, before scannerReadPoints is updated. 7127 IsolationLevel isolationLevel = scan.getIsolationLevel(); 7128 long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan); 7129 synchronized (scannerReadPoints) { 7130 if (mvccReadPoint > 0) { 7131 this.readPt = mvccReadPoint; 7132 } else if (nonce == HConstants.NO_NONCE || rsServices == null 7133 || rsServices.getNonceManager() == null) { 7134 this.readPt = getReadPoint(isolationLevel); 7135 } else { 7136 this.readPt = rsServices.getNonceManager().getMvccFromOperationContext(nonceGroup, nonce); 7137 } 7138 scannerReadPoints.put(this, this.readPt); 7139 } 7140 initializeScanners(scan, additionalScanners); 7141 } 7142 7143 protected void initializeScanners(Scan scan, List<KeyValueScanner> additionalScanners) 7144 throws IOException { 7145 // Here we separate all scanners into two lists - scanner that provide data required 7146 // by the filter to operate (scanners list) and all others (joinedScanners list). 7147 List<KeyValueScanner> scanners = new ArrayList<>(scan.getFamilyMap().size()); 7148 List<KeyValueScanner> joinedScanners = new ArrayList<>(scan.getFamilyMap().size()); 7149 // Store all already instantiated scanners for exception handling 7150 List<KeyValueScanner> instantiatedScanners = new ArrayList<>(); 7151 // handle additionalScanners 7152 if (additionalScanners != null && !additionalScanners.isEmpty()) { 7153 scanners.addAll(additionalScanners); 7154 instantiatedScanners.addAll(additionalScanners); 7155 } 7156 7157 try { 7158 for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) { 7159 HStore store = stores.get(entry.getKey()); 7160 KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt); 7161 instantiatedScanners.add(scanner); 7162 if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand() 7163 || this.filter.isFamilyEssential(entry.getKey())) { 7164 scanners.add(scanner); 7165 } else { 7166 joinedScanners.add(scanner); 7167 } 7168 } 7169 initializeKVHeap(scanners, joinedScanners, region); 7170 } catch (Throwable t) { 7171 throw handleException(instantiatedScanners, t); 7172 } 7173 } 7174 7175 protected void initializeKVHeap(List<KeyValueScanner> scanners, 7176 List<KeyValueScanner> joinedScanners, HRegion region) 7177 throws IOException { 7178 this.storeHeap = new KeyValueHeap(scanners, comparator); 7179 if (!joinedScanners.isEmpty()) { 7180 this.joinedHeap = new KeyValueHeap(joinedScanners, comparator); 7181 } 7182 } 7183 7184 private IOException handleException(List<KeyValueScanner> instantiatedScanners, 7185 Throwable t) { 7186 // remove scaner read point before throw the exception 7187 scannerReadPoints.remove(this); 7188 if (storeHeap != null) { 7189 storeHeap.close(); 7190 storeHeap = null; 7191 if (joinedHeap != null) { 7192 joinedHeap.close(); 7193 joinedHeap = null; 7194 } 7195 } else { 7196 // close all already instantiated scanners before throwing the exception 7197 for (KeyValueScanner scanner : instantiatedScanners) { 7198 scanner.close(); 7199 } 7200 } 7201 return t instanceof IOException ? (IOException) t : new IOException(t); 7202 } 7203 7204 @Override 7205 public long getMaxResultSize() { 7206 return maxResultSize; 7207 } 7208 7209 @Override 7210 public long getMvccReadPoint() { 7211 return this.readPt; 7212 } 7213 7214 @Override 7215 public int getBatch() { 7216 return this.defaultScannerContext.getBatchLimit(); 7217 } 7218 7219 /** 7220 * Reset both the filter and the old filter. 7221 * 7222 * @throws IOException in case a filter raises an I/O exception. 7223 */ 7224 protected void resetFilters() throws IOException { 7225 if (filter != null) { 7226 filter.reset(); 7227 } 7228 } 7229 7230 @Override 7231 public boolean next(List<Cell> outResults) 7232 throws IOException { 7233 // apply the batching limit by default 7234 return next(outResults, defaultScannerContext); 7235 } 7236 7237 @Override 7238 public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext) 7239 throws IOException { 7240 if (this.filterClosed) { 7241 throw new UnknownScannerException("Scanner was closed (timed out?) " + 7242 "after we renewed it. Could be caused by a very slow scanner " + 7243 "or a lengthy garbage collection"); 7244 } 7245 startRegionOperation(Operation.SCAN); 7246 try { 7247 return nextRaw(outResults, scannerContext); 7248 } finally { 7249 closeRegionOperation(Operation.SCAN); 7250 } 7251 } 7252 7253 @Override 7254 public boolean nextRaw(List<Cell> outResults) throws IOException { 7255 // Use the RegionScanner's context by default 7256 return nextRaw(outResults, defaultScannerContext); 7257 } 7258 7259 @Override 7260 public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext) 7261 throws IOException { 7262 if (storeHeap == null) { 7263 // scanner is closed 7264 throw new UnknownScannerException("Scanner was closed"); 7265 } 7266 boolean moreValues = false; 7267 if (outResults.isEmpty()) { 7268 // Usually outResults is empty. This is true when next is called 7269 // to handle scan or get operation. 7270 moreValues = nextInternal(outResults, scannerContext); 7271 } else { 7272 List<Cell> tmpList = new ArrayList<>(); 7273 moreValues = nextInternal(tmpList, scannerContext); 7274 outResults.addAll(tmpList); 7275 } 7276 7277 if (!outResults.isEmpty()) { 7278 readRequestsCount.increment(); 7279 if (metricsRegion != null) { 7280 metricsRegion.updateReadRequestCount(); 7281 } 7282 } 7283 if (rsServices != null && rsServices.getMetrics() != null) { 7284 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable()); 7285 } 7286 7287 // If the size limit was reached it means a partial Result is being returned. Returning a 7288 // partial Result means that we should not reset the filters; filters should only be reset in 7289 // between rows 7290 if (!scannerContext.mayHaveMoreCellsInRow()) { 7291 resetFilters(); 7292 } 7293 7294 if (isFilterDoneInternal()) { 7295 moreValues = false; 7296 } 7297 return moreValues; 7298 } 7299 7300 /** 7301 * @return true if more cells exist after this batch, false if scanner is done 7302 */ 7303 private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext) 7304 throws IOException { 7305 assert joinedContinuationRow != null; 7306 boolean moreValues = populateResult(results, this.joinedHeap, scannerContext, 7307 joinedContinuationRow); 7308 7309 if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { 7310 // We are done with this row, reset the continuation. 7311 joinedContinuationRow = null; 7312 } 7313 // As the data is obtained from two independent heaps, we need to 7314 // ensure that result list is sorted, because Result relies on that. 7315 sort(results, comparator); 7316 return moreValues; 7317 } 7318 7319 /** 7320 * Fetches records with currentRow into results list, until next row, batchLimit (if not -1) is 7321 * reached, or remainingResultSize (if not -1) is reaced 7322 * @param heap KeyValueHeap to fetch data from.It must be positioned on correct row before call. 7323 * @param scannerContext 7324 * @param currentRowCell 7325 * @return state of last call to {@link KeyValueHeap#next()} 7326 */ 7327 private boolean populateResult(List<Cell> results, KeyValueHeap heap, 7328 ScannerContext scannerContext, Cell currentRowCell) throws IOException { 7329 Cell nextKv; 7330 boolean moreCellsInRow = false; 7331 boolean tmpKeepProgress = scannerContext.getKeepProgress(); 7332 // Scanning between column families and thus the scope is between cells 7333 LimitScope limitScope = LimitScope.BETWEEN_CELLS; 7334 do { 7335 // Check for thread interrupt status in case we have been signaled from 7336 // #interruptRegionOperation. 7337 checkInterrupt(); 7338 7339 // We want to maintain any progress that is made towards the limits while scanning across 7340 // different column families. To do this, we toggle the keep progress flag on during calls 7341 // to the StoreScanner to ensure that any progress made thus far is not wiped away. 7342 scannerContext.setKeepProgress(true); 7343 heap.next(results, scannerContext); 7344 scannerContext.setKeepProgress(tmpKeepProgress); 7345 7346 nextKv = heap.peek(); 7347 moreCellsInRow = moreCellsInRow(nextKv, currentRowCell); 7348 if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext); 7349 if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) { 7350 return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues(); 7351 } else if (scannerContext.checkSizeLimit(limitScope)) { 7352 ScannerContext.NextState state = 7353 moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED; 7354 return scannerContext.setScannerState(state).hasMoreValues(); 7355 } else if (scannerContext.checkTimeLimit(limitScope)) { 7356 ScannerContext.NextState state = 7357 moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED; 7358 return scannerContext.setScannerState(state).hasMoreValues(); 7359 } 7360 } while (moreCellsInRow); 7361 return nextKv != null; 7362 } 7363 7364 /** 7365 * Based on the nextKv in the heap, and the current row, decide whether or not there are more 7366 * cells to be read in the heap. If the row of the nextKv in the heap matches the current row 7367 * then there are more cells to be read in the row. 7368 * @param nextKv 7369 * @param currentRowCell 7370 * @return true When there are more cells in the row to be read 7371 */ 7372 private boolean moreCellsInRow(final Cell nextKv, Cell currentRowCell) { 7373 return nextKv != null && CellUtil.matchingRows(nextKv, currentRowCell); 7374 } 7375 7376 /* 7377 * @return True if a filter rules the scanner is over, done. 7378 */ 7379 @Override 7380 public synchronized boolean isFilterDone() throws IOException { 7381 return isFilterDoneInternal(); 7382 } 7383 7384 private boolean isFilterDoneInternal() throws IOException { 7385 return this.filter != null && this.filter.filterAllRemaining(); 7386 } 7387 7388 private boolean nextInternal(List<Cell> results, ScannerContext scannerContext) 7389 throws IOException { 7390 if (!results.isEmpty()) { 7391 throw new IllegalArgumentException("First parameter should be an empty list"); 7392 } 7393 if (scannerContext == null) { 7394 throw new IllegalArgumentException("Scanner context cannot be null"); 7395 } 7396 Optional<RpcCall> rpcCall = RpcServer.getCurrentCall(); 7397 7398 // Save the initial progress from the Scanner context in these local variables. The progress 7399 // may need to be reset a few times if rows are being filtered out so we save the initial 7400 // progress. 7401 int initialBatchProgress = scannerContext.getBatchProgress(); 7402 long initialSizeProgress = scannerContext.getDataSizeProgress(); 7403 long initialHeapSizeProgress = scannerContext.getHeapSizeProgress(); 7404 7405 // Used to check time limit 7406 LimitScope limitScope = LimitScope.BETWEEN_CELLS; 7407 7408 // The loop here is used only when at some point during the next we determine 7409 // that due to effects of filters or otherwise, we have an empty row in the result. 7410 // Then we loop and try again. Otherwise, we must get out on the first iteration via return, 7411 // "true" if there's more data to read, "false" if there isn't (storeHeap is at a stop row, 7412 // and joinedHeap has no more data to read for the last row (if set, joinedContinuationRow). 7413 while (true) { 7414 // Starting to scan a new row. Reset the scanner progress according to whether or not 7415 // progress should be kept. 7416 if (scannerContext.getKeepProgress()) { 7417 // Progress should be kept. Reset to initial values seen at start of method invocation. 7418 scannerContext.setProgress(initialBatchProgress, initialSizeProgress, 7419 initialHeapSizeProgress); 7420 } else { 7421 scannerContext.clearProgress(); 7422 } 7423 if (rpcCall.isPresent()) { 7424 // If a user specifies a too-restrictive or too-slow scanner, the 7425 // client might time out and disconnect while the server side 7426 // is still processing the request. We should abort aggressively 7427 // in that case. 7428 long afterTime = rpcCall.get().disconnectSince(); 7429 if (afterTime >= 0) { 7430 throw new CallerDisconnectedException( 7431 "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " + 7432 this + " after " + afterTime + " ms, since " + 7433 "caller disconnected"); 7434 } 7435 } 7436 7437 // Check for thread interrupt status in case we have been signaled from 7438 // #interruptRegionOperation. 7439 checkInterrupt(); 7440 7441 // Let's see what we have in the storeHeap. 7442 Cell current = this.storeHeap.peek(); 7443 7444 boolean shouldStop = shouldStop(current); 7445 // When has filter row is true it means that the all the cells for a particular row must be 7446 // read before a filtering decision can be made. This means that filters where hasFilterRow 7447 // run the risk of enLongAddering out of memory errors in the case that they are applied to a 7448 // table that has very large rows. 7449 boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow(); 7450 7451 // If filter#hasFilterRow is true, partial results are not allowed since allowing them 7452 // would prevent the filters from being evaluated. Thus, if it is true, change the 7453 // scope of any limits that could potentially create partial results to 7454 // LimitScope.BETWEEN_ROWS so that those limits are not reached mid-row 7455 if (hasFilterRow) { 7456 if (LOG.isTraceEnabled()) { 7457 LOG.trace("filter#hasFilterRow is true which prevents partial results from being " 7458 + " formed. Changing scope of limits that may create partials"); 7459 } 7460 scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS); 7461 scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS); 7462 limitScope = LimitScope.BETWEEN_ROWS; 7463 } 7464 7465 if (scannerContext.checkTimeLimit(LimitScope.BETWEEN_CELLS)) { 7466 if (hasFilterRow) { 7467 throw new IncompatibleFilterException( 7468 "Filter whose hasFilterRow() returns true is incompatible with scans that must " + 7469 " stop mid-row because of a limit. ScannerContext:" + scannerContext); 7470 } 7471 return true; 7472 } 7473 7474 // Check if we were getting data from the joinedHeap and hit the limit. 7475 // If not, then it's main path - getting results from storeHeap. 7476 if (joinedContinuationRow == null) { 7477 // First, check if we are at a stop row. If so, there are no more results. 7478 if (shouldStop) { 7479 if (hasFilterRow) { 7480 filter.filterRowCells(results); 7481 } 7482 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7483 } 7484 7485 // Check if rowkey filter wants to exclude this row. If so, loop to next. 7486 // Technically, if we hit limits before on this row, we don't need this call. 7487 if (filterRowKey(current)) { 7488 incrementCountOfRowsFilteredMetric(scannerContext); 7489 // early check, see HBASE-16296 7490 if (isFilterDoneInternal()) { 7491 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7492 } 7493 // Typically the count of rows scanned is incremented inside #populateResult. However, 7494 // here we are filtering a row based purely on its row key, preventing us from calling 7495 // #populateResult. Thus, perform the necessary increment here to rows scanned metric 7496 incrementCountOfRowsScannedMetric(scannerContext); 7497 boolean moreRows = nextRow(scannerContext, current); 7498 if (!moreRows) { 7499 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7500 } 7501 results.clear(); 7502 7503 // Read nothing as the rowkey was filtered, but still need to check time limit 7504 if (scannerContext.checkTimeLimit(limitScope)) { 7505 return true; 7506 } 7507 continue; 7508 } 7509 7510 // Ok, we are good, let's try to get some results from the main heap. 7511 populateResult(results, this.storeHeap, scannerContext, current); 7512 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { 7513 if (hasFilterRow) { 7514 throw new IncompatibleFilterException( 7515 "Filter whose hasFilterRow() returns true is incompatible with scans that must " 7516 + " stop mid-row because of a limit. ScannerContext:" + scannerContext); 7517 } 7518 return true; 7519 } 7520 7521 // Check for thread interrupt status in case we have been signaled from 7522 // #interruptRegionOperation. 7523 checkInterrupt(); 7524 7525 Cell nextKv = this.storeHeap.peek(); 7526 shouldStop = shouldStop(nextKv); 7527 // save that the row was empty before filters applied to it. 7528 final boolean isEmptyRow = results.isEmpty(); 7529 7530 // We have the part of the row necessary for filtering (all of it, usually). 7531 // First filter with the filterRow(List). 7532 FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED; 7533 if (hasFilterRow) { 7534 ret = filter.filterRowCellsWithRet(results); 7535 7536 // We don't know how the results have changed after being filtered. Must set progress 7537 // according to contents of results now. 7538 if (scannerContext.getKeepProgress()) { 7539 scannerContext.setProgress(initialBatchProgress, initialSizeProgress, 7540 initialHeapSizeProgress); 7541 } else { 7542 scannerContext.clearProgress(); 7543 } 7544 scannerContext.incrementBatchProgress(results.size()); 7545 for (Cell cell : results) { 7546 scannerContext.incrementSizeProgress(PrivateCellUtil.estimatedSerializedSizeOf(cell), 7547 cell.heapSize()); 7548 } 7549 } 7550 7551 if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) { 7552 incrementCountOfRowsFilteredMetric(scannerContext); 7553 results.clear(); 7554 boolean moreRows = nextRow(scannerContext, current); 7555 if (!moreRows) { 7556 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7557 } 7558 7559 // This row was totally filtered out, if this is NOT the last row, 7560 // we should continue on. Otherwise, nothing else to do. 7561 if (!shouldStop) { 7562 // Read nothing as the cells was filtered, but still need to check time limit 7563 if (scannerContext.checkTimeLimit(limitScope)) { 7564 return true; 7565 } 7566 continue; 7567 } 7568 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7569 } 7570 7571 // Ok, we are done with storeHeap for this row. 7572 // Now we may need to fetch additional, non-essential data into row. 7573 // These values are not needed for filter to work, so we postpone their 7574 // fetch to (possibly) reduce amount of data loads from disk. 7575 if (this.joinedHeap != null) { 7576 boolean mayHaveData = joinedHeapMayHaveData(current); 7577 if (mayHaveData) { 7578 joinedContinuationRow = current; 7579 populateFromJoinedHeap(results, scannerContext); 7580 7581 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { 7582 return true; 7583 } 7584 } 7585 } 7586 } else { 7587 // Populating from the joined heap was stopped by limits, populate some more. 7588 populateFromJoinedHeap(results, scannerContext); 7589 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) { 7590 return true; 7591 } 7592 } 7593 // We may have just called populateFromJoinedMap and hit the limits. If that is 7594 // the case, we need to call it again on the next next() invocation. 7595 if (joinedContinuationRow != null) { 7596 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); 7597 } 7598 7599 // Finally, we are done with both joinedHeap and storeHeap. 7600 // Double check to prevent empty rows from appearing in result. It could be 7601 // the case when SingleColumnValueExcludeFilter is used. 7602 if (results.isEmpty()) { 7603 incrementCountOfRowsFilteredMetric(scannerContext); 7604 boolean moreRows = nextRow(scannerContext, current); 7605 if (!moreRows) { 7606 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7607 } 7608 if (!shouldStop) continue; 7609 } 7610 7611 if (shouldStop) { 7612 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues(); 7613 } else { 7614 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues(); 7615 } 7616 } 7617 } 7618 7619 protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) { 7620 filteredReadRequestsCount.increment(); 7621 if (metricsRegion != null) { 7622 metricsRegion.updateFilteredRecords(); 7623 } 7624 7625 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return; 7626 7627 scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet(); 7628 } 7629 7630 protected void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) { 7631 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return; 7632 7633 scannerContext.getMetrics().countOfRowsScanned.incrementAndGet(); 7634 } 7635 7636 /** 7637 * @param currentRowCell 7638 * @return true when the joined heap may have data for the current row 7639 * @throws IOException 7640 */ 7641 private boolean joinedHeapMayHaveData(Cell currentRowCell) 7642 throws IOException { 7643 Cell nextJoinedKv = joinedHeap.peek(); 7644 boolean matchCurrentRow = 7645 nextJoinedKv != null && CellUtil.matchingRows(nextJoinedKv, currentRowCell); 7646 boolean matchAfterSeek = false; 7647 7648 // If the next value in the joined heap does not match the current row, try to seek to the 7649 // correct row 7650 if (!matchCurrentRow) { 7651 Cell firstOnCurrentRow = PrivateCellUtil.createFirstOnRow(currentRowCell); 7652 boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true); 7653 matchAfterSeek = 7654 seekSuccessful && joinedHeap.peek() != null 7655 && CellUtil.matchingRows(joinedHeap.peek(), currentRowCell); 7656 } 7657 7658 return matchCurrentRow || matchAfterSeek; 7659 } 7660 7661 /** 7662 * This function is to maintain backward compatibility for 0.94 filters. HBASE-6429 combines 7663 * both filterRow & filterRow({@code List<KeyValue> kvs}) functions. While 0.94 code or older, 7664 * it may not implement hasFilterRow as HBase-6429 expects because 0.94 hasFilterRow() only 7665 * returns true when filterRow({@code List<KeyValue> kvs}) is overridden not the filterRow(). 7666 * Therefore, the filterRow() will be skipped. 7667 */ 7668 private boolean filterRow() throws IOException { 7669 // when hasFilterRow returns true, filter.filterRow() will be called automatically inside 7670 // filterRowCells(List<Cell> kvs) so we skip that scenario here. 7671 return filter != null && (!filter.hasFilterRow()) 7672 && filter.filterRow(); 7673 } 7674 7675 private boolean filterRowKey(Cell current) throws IOException { 7676 return filter != null && filter.filterRowKey(current); 7677 } 7678 7679 protected boolean nextRow(ScannerContext scannerContext, Cell curRowCell) throws IOException { 7680 assert this.joinedContinuationRow == null: "Trying to go to next row during joinedHeap read."; 7681 Cell next; 7682 while ((next = this.storeHeap.peek()) != null && 7683 CellUtil.matchingRows(next, curRowCell)) { 7684 // Check for thread interrupt status in case we have been signaled from 7685 // #interruptRegionOperation. 7686 checkInterrupt(); 7687 this.storeHeap.next(MOCKED_LIST); 7688 } 7689 resetFilters(); 7690 7691 // Calling the hook in CP which allows it to do a fast forward 7692 return this.region.getCoprocessorHost() == null 7693 || this.region.getCoprocessorHost() 7694 .postScannerFilterRow(this, curRowCell); 7695 } 7696 7697 protected boolean shouldStop(Cell currentRowCell) { 7698 if (currentRowCell == null) { 7699 return true; 7700 } 7701 if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_END_ROW)) { 7702 return false; 7703 } 7704 int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length); 7705 return c > 0 || (c == 0 && !includeStopRow); 7706 } 7707 7708 @Override 7709 public synchronized void close() { 7710 if (storeHeap != null) { 7711 storeHeap.close(); 7712 storeHeap = null; 7713 } 7714 if (joinedHeap != null) { 7715 joinedHeap.close(); 7716 joinedHeap = null; 7717 } 7718 // no need to synchronize here. 7719 scannerReadPoints.remove(this); 7720 this.filterClosed = true; 7721 } 7722 7723 KeyValueHeap getStoreHeapForTesting() { 7724 return storeHeap; 7725 } 7726 7727 @Override 7728 public synchronized boolean reseek(byte[] row) throws IOException { 7729 if (row == null) { 7730 throw new IllegalArgumentException("Row cannot be null."); 7731 } 7732 boolean result = false; 7733 startRegionOperation(); 7734 Cell kv = PrivateCellUtil.createFirstOnRow(row, 0, (short) row.length); 7735 try { 7736 // use request seek to make use of the lazy seek option. See HBASE-5520 7737 result = this.storeHeap.requestSeek(kv, true, true); 7738 if (this.joinedHeap != null) { 7739 result = this.joinedHeap.requestSeek(kv, true, true) || result; 7740 } 7741 } finally { 7742 closeRegionOperation(); 7743 } 7744 return result; 7745 } 7746 7747 @Override 7748 public void shipped() throws IOException { 7749 if (storeHeap != null) { 7750 storeHeap.shipped(); 7751 } 7752 if (joinedHeap != null) { 7753 joinedHeap.shipped(); 7754 } 7755 } 7756 7757 @Override 7758 public void run() throws IOException { 7759 // This is the RPC callback method executed. We do the close in of the scanner in this 7760 // callback 7761 this.close(); 7762 } 7763 } 7764 7765 // Utility methods 7766 /** 7767 * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL} 7768 * configuration property. 7769 * @param tableDir qualified path of directory where region should be located, usually the table 7770 * directory. 7771 * @param wal The WAL is the outbound log for any updates to the HRegion The wal file is a logfile 7772 * from the previous execution that's custom-computed for this HRegion. The HRegionServer 7773 * computes and sorts the appropriate wal info for this HRegion. If there is a previous 7774 * file (implying that the HRegion has been written-to before), then read it from the 7775 * supplied path. 7776 * @param fs is the filesystem. 7777 * @param conf is global configuration settings. 7778 * @param regionInfo - RegionInfo that describes the region is new), then read them from the 7779 * supplied path. 7780 * @param htd the table descriptor 7781 * @return the new instance 7782 */ 7783 public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs, 7784 Configuration conf, RegionInfo regionInfo, final TableDescriptor htd, 7785 RegionServerServices rsServices) { 7786 try { 7787 @SuppressWarnings("unchecked") 7788 Class<? extends HRegion> regionClass = 7789 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class); 7790 7791 Constructor<? extends HRegion> c = 7792 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class, 7793 Configuration.class, RegionInfo.class, TableDescriptor.class, 7794 RegionServerServices.class); 7795 7796 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices); 7797 } catch (Throwable e) { 7798 // todo: what should I throw here? 7799 throw new IllegalStateException("Could not instantiate a region instance.", e); 7800 } 7801 } 7802 7803 /** 7804 * Convenience method creating new HRegions. Used by createTable. 7805 * @param info Info for region to create. 7806 * @param rootDir Root directory for HBase instance 7807 * @param wal shared WAL 7808 * @param initialize - true to initialize the region 7809 * @return new HRegion 7810 */ 7811 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7812 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal, 7813 final boolean initialize) throws IOException { 7814 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null); 7815 } 7816 7817 /** 7818 * Convenience method creating new HRegions. Used by createTable. 7819 * @param info Info for region to create. 7820 * @param rootDir Root directory for HBase instance 7821 * @param wal shared WAL 7822 * @param initialize - true to initialize the region 7823 * @param rsRpcServices An interface we can request flushes against. 7824 * @return new HRegion 7825 */ 7826 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7827 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal, 7828 final boolean initialize, RegionServerServices rsRpcServices) throws IOException { 7829 LOG.info("creating " + info + ", tableDescriptor=" 7830 + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir); 7831 createRegionDir(conf, info, rootDir); 7832 FileSystem fs = rootDir.getFileSystem(conf); 7833 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 7834 HRegion region = 7835 HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices); 7836 if (initialize) { 7837 region.initialize(null); 7838 } 7839 return region; 7840 } 7841 7842 /** 7843 * Create a region under the given table directory. 7844 */ 7845 public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs, 7846 Path tableDir, TableDescriptor tableDesc) throws IOException { 7847 LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc, 7848 tableDir); 7849 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo); 7850 HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null); 7851 return region; 7852 } 7853 7854 /** 7855 * Create the region directory in the filesystem. 7856 */ 7857 public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri, 7858 Path rootDir) 7859 throws IOException { 7860 FileSystem fs = rootDir.getFileSystem(configuration); 7861 Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable()); 7862 // If directory already exists, will log warning and keep going. Will try to create 7863 // .regioninfo. If one exists, will overwrite. 7864 return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri); 7865 } 7866 7867 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7868 final Configuration conf, 7869 final TableDescriptor hTableDescriptor, 7870 final WAL wal) 7871 throws IOException { 7872 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true); 7873 } 7874 7875 7876 /** 7877 * Open a Region. 7878 * @param info Info for region to be opened. 7879 * @param wal WAL for region to use. This method will call 7880 * WAL#setSequenceNumber(long) passing the result of the call to 7881 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7882 * up. HRegionStore does this every time it opens a new region. 7883 * @return new HRegion 7884 * 7885 * @throws IOException 7886 */ 7887 public static HRegion openHRegion(final RegionInfo info, 7888 final TableDescriptor htd, final WAL wal, 7889 final Configuration conf) 7890 throws IOException { 7891 return openHRegion(info, htd, wal, conf, null, null); 7892 } 7893 7894 /** 7895 * Open a Region. 7896 * @param info Info for region to be opened 7897 * @param htd the table descriptor 7898 * @param wal WAL for region to use. This method will call 7899 * WAL#setSequenceNumber(long) passing the result of the call to 7900 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7901 * up. HRegionStore does this every time it opens a new region. 7902 * @param conf The Configuration object to use. 7903 * @param rsServices An interface we can request flushes against. 7904 * @param reporter An interface we can report progress against. 7905 * @return new HRegion 7906 * 7907 * @throws IOException 7908 */ 7909 public static HRegion openHRegion(final RegionInfo info, 7910 final TableDescriptor htd, final WAL wal, final Configuration conf, 7911 final RegionServerServices rsServices, 7912 final CancelableProgressable reporter) 7913 throws IOException { 7914 return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter); 7915 } 7916 7917 /** 7918 * Open a Region. 7919 * @param rootDir Root directory for HBase instance 7920 * @param info Info for region to be opened. 7921 * @param htd the table descriptor 7922 * @param wal WAL for region to use. This method will call 7923 * WAL#setSequenceNumber(long) passing the result of the call to 7924 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7925 * up. HRegionStore does this every time it opens a new region. 7926 * @param conf The Configuration object to use. 7927 * @return new HRegion 7928 * @throws IOException 7929 */ 7930 public static HRegion openHRegion(Path rootDir, final RegionInfo info, 7931 final TableDescriptor htd, final WAL wal, final Configuration conf) 7932 throws IOException { 7933 return openHRegion(rootDir, info, htd, wal, conf, null, null); 7934 } 7935 7936 /** 7937 * Open a Region. 7938 * @param rootDir Root directory for HBase instance 7939 * @param info Info for region to be opened. 7940 * @param htd the table descriptor 7941 * @param wal WAL for region to use. This method will call 7942 * WAL#setSequenceNumber(long) passing the result of the call to 7943 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7944 * up. HRegionStore does this every time it opens a new region. 7945 * @param conf The Configuration object to use. 7946 * @param rsServices An interface we can request flushes against. 7947 * @param reporter An interface we can report progress against. 7948 * @return new HRegion 7949 * @throws IOException 7950 */ 7951 public static HRegion openHRegion(final Path rootDir, final RegionInfo info, 7952 final TableDescriptor htd, final WAL wal, final Configuration conf, 7953 final RegionServerServices rsServices, 7954 final CancelableProgressable reporter) 7955 throws IOException { 7956 FileSystem fs = null; 7957 if (rsServices != null) { 7958 fs = rsServices.getFileSystem(); 7959 } 7960 if (fs == null) { 7961 fs = rootDir.getFileSystem(conf); 7962 } 7963 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter); 7964 } 7965 7966 /** 7967 * Open a Region. 7968 * @param conf The Configuration object to use. 7969 * @param fs Filesystem to use 7970 * @param rootDir Root directory for HBase instance 7971 * @param info Info for region to be opened. 7972 * @param htd the table descriptor 7973 * @param wal WAL for region to use. This method will call 7974 * WAL#setSequenceNumber(long) passing the result of the call to 7975 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7976 * up. HRegionStore does this every time it opens a new region. 7977 * @return new HRegion 7978 */ 7979 public static HRegion openHRegion(final Configuration conf, final FileSystem fs, 7980 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal) 7981 throws IOException { 7982 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null); 7983 } 7984 7985 /** 7986 * Open a Region. 7987 * @param conf The Configuration object to use. 7988 * @param fs Filesystem to use 7989 * @param rootDir Root directory for HBase instance 7990 * @param info Info for region to be opened. 7991 * @param htd the table descriptor 7992 * @param wal WAL for region to use. This method will call 7993 * WAL#setSequenceNumber(long) passing the result of the call to 7994 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 7995 * up. HRegionStore does this every time it opens a new region. 7996 * @param rsServices An interface we can request flushes against. 7997 * @param reporter An interface we can report progress against. 7998 * @return new HRegion 7999 */ 8000 public static HRegion openHRegion(final Configuration conf, final FileSystem fs, 8001 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal, 8002 final RegionServerServices rsServices, final CancelableProgressable reporter) 8003 throws IOException { 8004 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 8005 return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter); 8006 } 8007 8008 /** 8009 * Open a Region. 8010 * @param conf The Configuration object to use. 8011 * @param fs Filesystem to use 8012 * @param info Info for region to be opened. 8013 * @param htd the table descriptor 8014 * @param wal WAL for region to use. This method will call 8015 * WAL#setSequenceNumber(long) passing the result of the call to 8016 * HRegion#getMinSequenceId() to ensure the wal id is properly kept 8017 * up. HRegionStore does this every time it opens a new region. 8018 * @param rsServices An interface we can request flushes against. 8019 * @param reporter An interface we can report progress against. 8020 * @return new HRegion 8021 */ 8022 public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs, 8023 final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal, 8024 final RegionServerServices rsServices, final CancelableProgressable reporter) 8025 throws IOException { 8026 Objects.requireNonNull(info, "RegionInfo cannot be null"); 8027 LOG.debug("Opening region: {}", info); 8028 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices); 8029 return r.openHRegion(reporter); 8030 } 8031 8032 public NavigableMap<byte[], Integer> getReplicationScope() { 8033 return this.replicationScope; 8034 } 8035 8036 /** 8037 * Useful when reopening a closed region (normally for unit tests) 8038 * @param other original object 8039 * @param reporter An interface we can report progress against. 8040 * @return new HRegion 8041 */ 8042 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter) 8043 throws IOException { 8044 HRegionFileSystem regionFs = other.getRegionFileSystem(); 8045 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(), 8046 other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null); 8047 return r.openHRegion(reporter); 8048 } 8049 8050 public static Region openHRegion(final Region other, final CancelableProgressable reporter) 8051 throws IOException { 8052 return openHRegion((HRegion)other, reporter); 8053 } 8054 8055 /** 8056 * Open HRegion. 8057 * Calls initialize and sets sequenceId. 8058 * @return Returns <code>this</code> 8059 */ 8060 protected HRegion openHRegion(final CancelableProgressable reporter) 8061 throws IOException { 8062 try { 8063 // Refuse to open the region if we are missing local compression support 8064 TableDescriptorChecker.checkCompression(htableDescriptor); 8065 // Refuse to open the region if encryption configuration is incorrect or 8066 // codec support is missing 8067 TableDescriptorChecker.checkEncryption(conf, htableDescriptor); 8068 // Refuse to open the region if a required class cannot be loaded 8069 TableDescriptorChecker.checkClassLoading(conf, htableDescriptor); 8070 this.openSeqNum = initialize(reporter); 8071 this.mvcc.advanceTo(openSeqNum); 8072 // The openSeqNum must be increased every time when a region is assigned, as we rely on it to 8073 // determine whether a region has been successfully reopened. So here we always write open 8074 // marker, even if the table is read only. 8075 if (wal != null && getRegionServerServices() != null && 8076 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) { 8077 writeRegionOpenMarker(wal, openSeqNum); 8078 } 8079 } catch (Throwable t) { 8080 // By coprocessor path wrong region will open failed, 8081 // MetricsRegionWrapperImpl is already init and not close, 8082 // add region close when open failed 8083 try { 8084 this.close(); 8085 } catch (Throwable e) { 8086 LOG.warn("Open region: {} failed. Try close region but got exception ", this.getRegionInfo(), 8087 e); 8088 } 8089 throw t; 8090 } 8091 return this; 8092 } 8093 8094 /** 8095 * Open a Region on a read-only file-system (like hdfs snapshots) 8096 * @param conf The Configuration object to use. 8097 * @param fs Filesystem to use 8098 * @param info Info for region to be opened. 8099 * @param htd the table descriptor 8100 * @return new HRegion 8101 */ 8102 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs, 8103 final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException { 8104 if (info == null) { 8105 throw new NullPointerException("Passed region info is null"); 8106 } 8107 if (LOG.isDebugEnabled()) { 8108 LOG.debug("Opening region (readOnly filesystem): " + info); 8109 } 8110 if (info.getReplicaId() <= 0) { 8111 info = RegionInfoBuilder.newBuilder(info).setReplicaId(1).build(); 8112 } 8113 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null); 8114 r.writestate.setReadOnly(true); 8115 return r.openHRegion(null); 8116 } 8117 8118 public static void warmupHRegion(final RegionInfo info, 8119 final TableDescriptor htd, final WAL wal, final Configuration conf, 8120 final RegionServerServices rsServices, 8121 final CancelableProgressable reporter) 8122 throws IOException { 8123 8124 Objects.requireNonNull(info, "RegionInfo cannot be null"); 8125 LOG.debug("Warmup {}", info); 8126 Path rootDir = CommonFSUtils.getRootDir(conf); 8127 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 8128 FileSystem fs = null; 8129 if (rsServices != null) { 8130 fs = rsServices.getFileSystem(); 8131 } 8132 if (fs == null) { 8133 fs = rootDir.getFileSystem(conf); 8134 } 8135 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null); 8136 r.initializeWarmup(reporter); 8137 } 8138 8139 /** 8140 * Computes the Path of the HRegion 8141 * 8142 * @param tabledir qualified path for table 8143 * @param name ENCODED region name 8144 * @return Path of HRegion directory 8145 * @deprecated For tests only; to be removed. 8146 */ 8147 @Deprecated 8148 public static Path getRegionDir(final Path tabledir, final String name) { 8149 return new Path(tabledir, name); 8150 } 8151 8152 /** 8153 * Determines if the specified row is within the row range specified by the 8154 * specified RegionInfo 8155 * 8156 * @param info RegionInfo that specifies the row range 8157 * @param row row to be checked 8158 * @return true if the row is within the range specified by the RegionInfo 8159 */ 8160 public static boolean rowIsInRange(RegionInfo info, final byte [] row) { 8161 return ((info.getStartKey().length == 0) || 8162 (Bytes.compareTo(info.getStartKey(), row) <= 0)) && 8163 ((info.getEndKey().length == 0) || 8164 (Bytes.compareTo(info.getEndKey(), row) > 0)); 8165 } 8166 8167 public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset, 8168 final short length) { 8169 return ((info.getStartKey().length == 0) || 8170 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length, 8171 row, offset, length) <= 0)) && 8172 ((info.getEndKey().length == 0) || 8173 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0)); 8174 } 8175 8176 @Override 8177 public Result get(final Get get) throws IOException { 8178 prepareGet(get); 8179 List<Cell> results = get(get, true); 8180 boolean stale = this.getRegionInfo().getReplicaId() != 0; 8181 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale); 8182 } 8183 8184 void prepareGet(final Get get) throws IOException { 8185 checkRow(get.getRow(), "Get"); 8186 // Verify families are all valid 8187 if (get.hasFamilies()) { 8188 for (byte[] family : get.familySet()) { 8189 checkFamily(family); 8190 } 8191 } else { // Adding all families to scanner 8192 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) { 8193 get.addFamily(family); 8194 } 8195 } 8196 } 8197 8198 @Override 8199 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException { 8200 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE); 8201 } 8202 8203 public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) 8204 throws IOException { 8205 List<Cell> results = new ArrayList<>(); 8206 long before = EnvironmentEdgeManager.currentTime(); 8207 8208 // pre-get CP hook 8209 if (withCoprocessor && (coprocessorHost != null)) { 8210 if (coprocessorHost.preGet(get, results)) { 8211 metricsUpdateForGet(results, before); 8212 return results; 8213 } 8214 } 8215 Scan scan = new Scan(get); 8216 if (scan.getLoadColumnFamiliesOnDemandValue() == null) { 8217 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault()); 8218 } 8219 RegionScanner scanner = null; 8220 try { 8221 scanner = getScanner(scan, null, nonceGroup, nonce); 8222 scanner.next(results); 8223 } finally { 8224 if (scanner != null) 8225 scanner.close(); 8226 } 8227 8228 // post-get CP hook 8229 if (withCoprocessor && (coprocessorHost != null)) { 8230 coprocessorHost.postGet(get, results); 8231 } 8232 8233 metricsUpdateForGet(results, before); 8234 8235 return results; 8236 } 8237 8238 void metricsUpdateForGet(List<Cell> results, long before) { 8239 if (this.metricsRegion != null) { 8240 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before); 8241 } 8242 } 8243 8244 @Override 8245 public Result mutateRow(RowMutations rm) throws IOException { 8246 final List<Mutation> m = rm.getMutations(); 8247 OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, 8248 HConstants.NO_NONCE, HConstants.NO_NONCE); 8249 8250 List<Result> results = new ArrayList<>(); 8251 for (OperationStatus status : statuses) { 8252 if (status.getResult() != null) { 8253 results.add(status.getResult()); 8254 } 8255 } 8256 8257 if (results.isEmpty()) { 8258 return null; 8259 } 8260 8261 // Merge the results of the Increment/Append operations 8262 List<Cell> cells = new ArrayList<>(); 8263 for (Result result : results) { 8264 if (result.rawCells() != null) { 8265 cells.addAll(Arrays.asList(result.rawCells())); 8266 } 8267 } 8268 return Result.create(cells); 8269 } 8270 8271 /** 8272 * Perform atomic (all or none) mutations within the region. 8273 * @param mutations The list of mutations to perform. 8274 * <code>mutations</code> can contain operations for multiple rows. 8275 * Caller has to ensure that all rows are contained in this region. 8276 * @param rowsToLock Rows to lock 8277 * @param nonceGroup Optional nonce group of the operation (client Id) 8278 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence") 8279 * If multiple rows are locked care should be taken that 8280 * <code>rowsToLock</code> is sorted in order to avoid deadlocks. 8281 * @throws IOException 8282 */ 8283 @Override 8284 public void mutateRowsWithLocks(Collection<Mutation> mutations, 8285 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException { 8286 batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]), 8287 true, nonceGroup, nonce) { 8288 @Override 8289 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch( 8290 List<RowLock> acquiredRowLocks) throws IOException { 8291 RowLock prevRowLock = null; 8292 for (byte[] row : rowsToLock) { 8293 try { 8294 RowLock rowLock = region.getRowLockInternal(row, false, prevRowLock); // write lock 8295 if (rowLock != prevRowLock) { 8296 acquiredRowLocks.add(rowLock); 8297 prevRowLock = rowLock; 8298 } 8299 } catch (IOException ioe) { 8300 LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this, 8301 ioe); 8302 throw ioe; 8303 } 8304 } 8305 return createMiniBatch(size(), size()); 8306 } 8307 }); 8308 } 8309 8310 /** 8311 * @return statistics about the current load of the region 8312 */ 8313 public ClientProtos.RegionLoadStats getLoadStatistics() { 8314 if (!regionStatsEnabled) { 8315 return null; 8316 } 8317 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder(); 8318 stats.setMemStoreLoad((int) (Math.min(100, 8319 (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize))); 8320 if (rsServices.getHeapMemoryManager() != null) { 8321 // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM, 8322 // so we could just do the calculation below and we'll get a 0. 8323 // treating it as a special case analogous to no HMM instead so that it can be 8324 // programatically treated different from using <1% of heap. 8325 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent(); 8326 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) { 8327 stats.setHeapOccupancy((int)(occupancy * 100)); 8328 } 8329 } 8330 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100 8331 : rsServices.getCompactionPressure() * 100)); 8332 return stats.build(); 8333 } 8334 8335 @Override 8336 public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException { 8337 processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE, HConstants.NO_NONCE); 8338 } 8339 8340 @Override 8341 public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce) 8342 throws IOException { 8343 processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce); 8344 } 8345 8346 @Override 8347 public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout, 8348 long nonceGroup, long nonce) throws IOException { 8349 for (byte[] row : processor.getRowsToLock()) { 8350 checkRow(row, "processRowsWithLocks"); 8351 } 8352 if (!processor.readOnly()) { 8353 checkReadOnly(); 8354 } 8355 checkResources(); 8356 startRegionOperation(); 8357 WALEdit walEdit = new WALEdit(); 8358 8359 // STEP 1. Run pre-process hook 8360 preProcess(processor, walEdit); 8361 // Short circuit the read only case 8362 if (processor.readOnly()) { 8363 try { 8364 long now = EnvironmentEdgeManager.currentTime(); 8365 doProcessRowWithTimeout(processor, now, this, null, null, timeout); 8366 processor.postProcess(this, walEdit, true); 8367 } finally { 8368 closeRegionOperation(); 8369 } 8370 return; 8371 } 8372 8373 boolean locked = false; 8374 List<RowLock> acquiredRowLocks = null; 8375 List<Mutation> mutations = new ArrayList<>(); 8376 Collection<byte[]> rowsToLock = processor.getRowsToLock(); 8377 // This is assigned by mvcc either explicity in the below or in the guts of the WAL append 8378 // when it assigns the edit a sequencedid (A.K.A the mvcc write number). 8379 WriteEntry writeEntry = null; 8380 MemStoreSizing memstoreAccounting = new NonThreadSafeMemStoreSizing(); 8381 8382 // Check for thread interrupt status in case we have been signaled from 8383 // #interruptRegionOperation. 8384 checkInterrupt(); 8385 8386 try { 8387 boolean success = false; 8388 try { 8389 // STEP 2. Acquire the row lock(s) 8390 acquiredRowLocks = new ArrayList<>(rowsToLock.size()); 8391 RowLock prevRowLock = null; 8392 for (byte[] row : rowsToLock) { 8393 // Attempt to lock all involved rows, throw if any lock times out 8394 // use a writer lock for mixed reads and writes 8395 RowLock rowLock = getRowLockInternal(row, false, prevRowLock); 8396 if (rowLock != prevRowLock) { 8397 acquiredRowLocks.add(rowLock); 8398 prevRowLock = rowLock; 8399 } 8400 } 8401 8402 // Check for thread interrupt status in case we have been signaled from 8403 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for 8404 // the WAL append. 8405 checkInterrupt(); 8406 8407 // STEP 3. Region lock 8408 lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size()); 8409 locked = true; 8410 8411 // From this point until memstore update this operation should not be interrupted. 8412 disableInterrupts(); 8413 8414 long now = EnvironmentEdgeManager.currentTime(); 8415 // STEP 4. Let the processor scan the rows, generate mutations and add waledits 8416 doProcessRowWithTimeout(processor, now, this, mutations, walEdit, timeout); 8417 if (!mutations.isEmpty()) { 8418 writeRequestsCount.add(mutations.size()); 8419 // STEP 5. Call the preBatchMutate hook 8420 processor.preBatchMutate(this, walEdit); 8421 8422 // STEP 6. Append and sync if walEdit has data to write out. 8423 if (!walEdit.isEmpty()) { 8424 writeEntry = doWALAppend(walEdit, getEffectiveDurability(processor.useDurability()), 8425 processor.getClusterIds(), now, nonceGroup, nonce); 8426 } else { 8427 // We are here if WAL is being skipped. 8428 writeEntry = this.mvcc.begin(); 8429 } 8430 8431 // STEP 7. Apply to memstore 8432 long sequenceId = writeEntry.getWriteNumber(); 8433 for (Mutation m : mutations) { 8434 // Handle any tag based cell features. 8435 // TODO: Do we need to call rewriteCellTags down in applyToMemStore()? Why not before 8436 // so tags go into WAL? 8437 rewriteCellTags(m.getFamilyCellMap(), m); 8438 for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { 8439 Cell cell = cellScanner.current(); 8440 if (walEdit.isEmpty()) { 8441 // If walEdit is empty, we put nothing in WAL. WAL stamps Cells with sequence id. 8442 // If no WAL, need to stamp it here. 8443 PrivateCellUtil.setSequenceId(cell, sequenceId); 8444 } 8445 applyToMemStore(getStore(cell), cell, memstoreAccounting); 8446 } 8447 } 8448 8449 // STEP 8. call postBatchMutate hook 8450 processor.postBatchMutate(this); 8451 8452 // STEP 9. Complete mvcc. 8453 mvcc.completeAndWait(writeEntry); 8454 writeEntry = null; 8455 8456 // STEP 10. Release region lock 8457 if (locked) { 8458 this.updatesLock.readLock().unlock(); 8459 locked = false; 8460 } 8461 8462 // STEP 11. Release row lock(s) 8463 releaseRowLocks(acquiredRowLocks); 8464 8465 if (rsServices != null && rsServices.getMetrics() != null) { 8466 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor. 8467 getTableName(), mutations.size()); 8468 } 8469 } 8470 success = true; 8471 } finally { 8472 // Call complete rather than completeAndWait because we probably had error if walKey != null 8473 if (writeEntry != null) mvcc.complete(writeEntry); 8474 if (locked) { 8475 this.updatesLock.readLock().unlock(); 8476 } 8477 // release locks if some were acquired but another timed out 8478 releaseRowLocks(acquiredRowLocks); 8479 8480 enableInterrupts(); 8481 } 8482 8483 // 12. Run post-process hook 8484 processor.postProcess(this, walEdit, success); 8485 } finally { 8486 closeRegionOperation(); 8487 if (!mutations.isEmpty()) { 8488 this.incMemStoreSize(memstoreAccounting.getMemStoreSize()); 8489 requestFlushIfNeeded(); 8490 } 8491 } 8492 } 8493 8494 private void preProcess(final RowProcessor<?,?> processor, final WALEdit walEdit) 8495 throws IOException { 8496 try { 8497 processor.preProcess(this, walEdit); 8498 } catch (IOException e) { 8499 closeRegionOperation(); 8500 throw e; 8501 } 8502 } 8503 8504 private void doProcessRowWithTimeout(final RowProcessor<?,?> processor, 8505 final long now, 8506 final HRegion region, 8507 final List<Mutation> mutations, 8508 final WALEdit walEdit, 8509 final long timeout) throws IOException { 8510 // Short circuit the no time bound case. 8511 if (timeout < 0) { 8512 try { 8513 processor.process(now, region, mutations, walEdit); 8514 } catch (IOException e) { 8515 String row = processor.getRowsToLock().isEmpty() ? "" : 8516 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "..."; 8517 LOG.warn("RowProcessor: {}, in region {}, throws Exception {}", 8518 processor.getClass().getName(), getRegionInfo().getRegionNameAsString(), row, e); 8519 throw e; 8520 } 8521 return; 8522 } 8523 8524 // Case with time bound 8525 FutureTask<Void> task = new FutureTask<>(new Callable<Void>() { 8526 @Override 8527 public Void call() throws IOException { 8528 try { 8529 processor.process(now, region, mutations, walEdit); 8530 return null; 8531 } catch (IOException e) { 8532 String row = processor.getRowsToLock().isEmpty() ? "" : 8533 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "..."; 8534 LOG.warn("RowProcessor: {}, in region {}, throws Exception {}", 8535 processor.getClass().getName(), getRegionInfo().getRegionNameAsString(), row, e); 8536 throw e; 8537 } 8538 } 8539 }); 8540 rowProcessorExecutor.execute(task); 8541 try { 8542 task.get(timeout, TimeUnit.MILLISECONDS); 8543 } catch (InterruptedException ie) { 8544 throw throwOnInterrupt(ie); 8545 } catch (TimeoutException te) { 8546 String row = processor.getRowsToLock().isEmpty() ? "" : 8547 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "..."; 8548 LOG.error("RowProcessor timeout: {} ms, in region {}, {}", timeout, 8549 getRegionInfo().getRegionNameAsString(), row); 8550 throw new IOException(te); 8551 } catch (Exception e) { 8552 throw new IOException(e); 8553 } 8554 } 8555 8556 @Override 8557 public Result append(Append append) throws IOException { 8558 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE); 8559 } 8560 8561 public Result append(Append append, long nonceGroup, long nonce) throws IOException { 8562 checkReadOnly(); 8563 checkResources(); 8564 startRegionOperation(Operation.APPEND); 8565 try { 8566 // All edits for the given row (across all column families) must happen atomically. 8567 return doBatchMutate(append, true, nonceGroup, nonce).getResult(); 8568 } finally { 8569 closeRegionOperation(Operation.APPEND); 8570 } 8571 } 8572 8573 @Override 8574 public Result increment(Increment increment) throws IOException { 8575 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE); 8576 } 8577 8578 public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException { 8579 checkReadOnly(); 8580 checkResources(); 8581 startRegionOperation(Operation.INCREMENT); 8582 try { 8583 // All edits for the given row (across all column families) must happen atomically. 8584 return doBatchMutate(increment, true, nonceGroup, nonce).getResult(); 8585 } finally { 8586 closeRegionOperation(Operation.INCREMENT); 8587 } 8588 } 8589 8590 private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds, 8591 long now, long nonceGroup, long nonce) throws IOException { 8592 return doWALAppend(walEdit, durability, clusterIds, now, nonceGroup, nonce, 8593 SequenceId.NO_SEQUENCE_ID); 8594 } 8595 8596 /** 8597 * @return writeEntry associated with this append 8598 */ 8599 private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds, 8600 long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException { 8601 Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(), 8602 "WALEdit is null or empty!"); 8603 Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != SequenceId.NO_SEQUENCE_ID, 8604 "Invalid replay sequence Id for replay WALEdit!"); 8605 // Using default cluster id, as this can only happen in the originating cluster. 8606 // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey 8607 // here instead of WALKeyImpl directly to support legacy coprocessors. 8608 WALKeyImpl walKey = walEdit.isReplay()? 8609 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(), 8610 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds, 8611 nonceGroup, nonce, mvcc) : 8612 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(), 8613 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds, 8614 nonceGroup, nonce, mvcc, this.getReplicationScope()); 8615 if (walEdit.isReplay()) { 8616 walKey.setOrigLogSeqNum(origLogSeqNum); 8617 } 8618 //don't call the coproc hook for writes to the WAL caused by 8619 //system lifecycle events like flushes or compactions 8620 if (this.coprocessorHost != null && !walEdit.isMetaEdit()) { 8621 this.coprocessorHost.preWALAppend(walKey, walEdit); 8622 } 8623 WriteEntry writeEntry = null; 8624 try { 8625 long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit); 8626 // Call sync on our edit. 8627 if (txid != 0) { 8628 sync(txid, durability); 8629 } 8630 writeEntry = walKey.getWriteEntry(); 8631 } catch (IOException ioe) { 8632 if (walKey != null && walKey.getWriteEntry() != null) { 8633 mvcc.complete(walKey.getWriteEntry()); 8634 } 8635 throw ioe; 8636 } 8637 return writeEntry; 8638 } 8639 8640 /** 8641 * @return Sorted list of <code>cells</code> using <code>comparator</code> 8642 */ 8643 private static List<Cell> sort(List<Cell> cells, final CellComparator comparator) { 8644 cells.sort(comparator); 8645 return cells; 8646 } 8647 8648 // 8649 // New HBASE-880 Helpers 8650 // 8651 8652 void checkFamily(final byte [] family) 8653 throws NoSuchColumnFamilyException { 8654 if (!this.htableDescriptor.hasColumnFamily(family)) { 8655 throw new NoSuchColumnFamilyException("Column family " + 8656 Bytes.toString(family) + " does not exist in region " + this 8657 + " in table " + this.htableDescriptor); 8658 } 8659 } 8660 8661 public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false); 8662 8663 // woefully out of date - currently missing: 8664 // 1 x HashMap - coprocessorServiceHandlers 8665 // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL, 8666 // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount, 8667 // writeRequestsCount 8668 // 1 x HRegion$WriteState - writestate 8669 // 1 x RegionCoprocessorHost - coprocessorHost 8670 // 1 x RegionSplitPolicy - splitPolicy 8671 // 1 x MetricsRegion - metricsRegion 8672 // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper 8673 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + 8674 ClassSize.OBJECT + // closeLock 8675 (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing 8676 (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL, 8677 // compactionsFailed 8678 (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders 8679 WriteState.HEAP_SIZE + // writestate 8680 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores 8681 (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock 8682 MultiVersionConcurrencyControl.FIXED_SIZE // mvcc 8683 + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes 8684 + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress 8685 + ClassSize.STORE_SERVICES // store services 8686 + StoreHotnessProtector.FIXED_SIZE 8687 ; 8688 8689 @Override 8690 public long heapSize() { 8691 // this does not take into account row locks, recent flushes, mvcc entries, and more 8692 return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum(); 8693 } 8694 8695 /** 8696 * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to 8697 * be available for handling Region#execService(com.google.protobuf.RpcController, 8698 * org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall) calls. 8699 * 8700 * <p> 8701 * Only a single instance may be registered per region for a given {@link Service} subclass (the 8702 * instances are keyed on {@link com.google.protobuf.Descriptors.ServiceDescriptor#getFullName()}. 8703 * After the first registration, subsequent calls with the same service name will fail with 8704 * a return value of {@code false}. 8705 * </p> 8706 * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint 8707 * @return {@code true} if the registration was successful, {@code false} 8708 * otherwise 8709 */ 8710 public boolean registerService(com.google.protobuf.Service instance) { 8711 /* 8712 * No stacking of instances is allowed for a single service name 8713 */ 8714 com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType(); 8715 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc); 8716 if (coprocessorServiceHandlers.containsKey(serviceName)) { 8717 LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}", 8718 serviceName, instance, this); 8719 return false; 8720 } 8721 8722 coprocessorServiceHandlers.put(serviceName, instance); 8723 if (LOG.isDebugEnabled()) { 8724 LOG.debug("Registered coprocessor service: region=" + 8725 Bytes.toStringBinary(getRegionInfo().getRegionName()) + 8726 " service=" + serviceName); 8727 } 8728 return true; 8729 } 8730 8731 /** 8732 * Executes a single protocol buffer coprocessor endpoint {@link Service} method using 8733 * the registered protocol handlers. {@link Service} implementations must be registered via the 8734 * {@link #registerService(com.google.protobuf.Service)} 8735 * method before they are available. 8736 * 8737 * @param controller an {@code RpcContoller} implementation to pass to the invoked service 8738 * @param call a {@code CoprocessorServiceCall} instance identifying the service, method, 8739 * and parameters for the method invocation 8740 * @return a protocol buffer {@code Message} instance containing the method's result 8741 * @throws IOException if no registered service handler is found or an error 8742 * occurs during the invocation 8743 * @see #registerService(com.google.protobuf.Service) 8744 */ 8745 public com.google.protobuf.Message execService(com.google.protobuf.RpcController controller, 8746 CoprocessorServiceCall call) throws IOException { 8747 String serviceName = call.getServiceName(); 8748 com.google.protobuf.Service service = coprocessorServiceHandlers.get(serviceName); 8749 if (service == null) { 8750 throw new UnknownProtocolException(null, "No registered coprocessor service found for " + 8751 serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName())); 8752 } 8753 com.google.protobuf.Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType(); 8754 8755 String methodName = call.getMethodName(); 8756 com.google.protobuf.Descriptors.MethodDescriptor methodDesc = 8757 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc); 8758 8759 com.google.protobuf.Message.Builder builder = 8760 service.getRequestPrototype(methodDesc).newBuilderForType(); 8761 8762 org.apache.hadoop.hbase.protobuf.ProtobufUtil.mergeFrom(builder, 8763 call.getRequest().toByteArray()); 8764 com.google.protobuf.Message request = 8765 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest()); 8766 8767 if (coprocessorHost != null) { 8768 request = coprocessorHost.preEndpointInvocation(service, methodName, request); 8769 } 8770 8771 final com.google.protobuf.Message.Builder responseBuilder = 8772 service.getResponsePrototype(methodDesc).newBuilderForType(); 8773 service.callMethod(methodDesc, controller, request, 8774 new com.google.protobuf.RpcCallback<com.google.protobuf.Message>() { 8775 @Override 8776 public void run(com.google.protobuf.Message message) { 8777 if (message != null) { 8778 responseBuilder.mergeFrom(message); 8779 } 8780 } 8781 }); 8782 8783 if (coprocessorHost != null) { 8784 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder); 8785 } 8786 IOException exception = 8787 org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller); 8788 if (exception != null) { 8789 throw exception; 8790 } 8791 8792 return responseBuilder.build(); 8793 } 8794 8795 public Optional<byte[]> checkSplit() { 8796 return checkSplit(false); 8797 } 8798 8799 /** 8800 * Return the split point. An empty result indicates the region isn't splittable. 8801 */ 8802 public Optional<byte[]> checkSplit(boolean force) { 8803 // Can't split META 8804 if (this.getRegionInfo().isMetaRegion() || 8805 TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) { 8806 return Optional.empty(); 8807 } 8808 8809 // Can't split a region that is closing. 8810 if (this.isClosing()) { 8811 return Optional.empty(); 8812 } 8813 8814 if (!force && !splitPolicy.shouldSplit()) { 8815 return Optional.empty(); 8816 } 8817 8818 byte[] ret = splitPolicy.getSplitPoint(); 8819 8820 if (ret != null) { 8821 try { 8822 checkRow(ret, "calculated split"); 8823 } catch (IOException e) { 8824 LOG.error("Ignoring invalid split for region {}", this, e); 8825 return Optional.empty(); 8826 } 8827 return Optional.of(ret); 8828 } else { 8829 return Optional.empty(); 8830 } 8831 } 8832 8833 /** 8834 * @return The priority that this region should have in the compaction queue 8835 */ 8836 public int getCompactPriority() { 8837 return stores.values().stream().mapToInt(HStore::getCompactPriority).min() 8838 .orElse(Store.NO_PRIORITY); 8839 } 8840 8841 /** @return the coprocessor host */ 8842 public RegionCoprocessorHost getCoprocessorHost() { 8843 return coprocessorHost; 8844 } 8845 8846 /** @param coprocessorHost the new coprocessor host */ 8847 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) { 8848 this.coprocessorHost = coprocessorHost; 8849 } 8850 8851 @Override 8852 public void startRegionOperation() throws IOException { 8853 startRegionOperation(Operation.ANY); 8854 } 8855 8856 @Override 8857 public void startRegionOperation(Operation op) throws IOException { 8858 boolean isInterruptableOp = false; 8859 switch (op) { 8860 case GET: // interruptible read operations 8861 case SCAN: 8862 isInterruptableOp = true; 8863 checkReadsEnabled(); 8864 break; 8865 case INCREMENT: // interruptible write operations 8866 case APPEND: 8867 case PUT: 8868 case DELETE: 8869 case BATCH_MUTATE: 8870 case CHECK_AND_MUTATE: 8871 isInterruptableOp = true; 8872 break; 8873 default: // all others 8874 break; 8875 } 8876 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION 8877 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) { 8878 // split, merge or compact region doesn't need to check the closing/closed state or lock the 8879 // region 8880 return; 8881 } 8882 if (this.closing.get()) { 8883 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); 8884 } 8885 lock(lock.readLock()); 8886 // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from 8887 // an RPC handler 8888 Thread thisThread = Thread.currentThread(); 8889 if (isInterruptableOp) { 8890 regionLockHolders.put(thisThread, true); 8891 } 8892 if (this.closed.get()) { 8893 lock.readLock().unlock(); 8894 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); 8895 } 8896 // The unit for snapshot is a region. So, all stores for this region must be 8897 // prepared for snapshot operation before proceeding. 8898 if (op == Operation.SNAPSHOT) { 8899 stores.values().forEach(HStore::preSnapshotOperation); 8900 } 8901 try { 8902 if (coprocessorHost != null) { 8903 coprocessorHost.postStartRegionOperation(op); 8904 } 8905 } catch (Exception e) { 8906 if (isInterruptableOp) { 8907 // would be harmless to remove what we didn't add but we know by 'isInterruptableOp' 8908 // if we added this thread to regionLockHolders 8909 regionLockHolders.remove(thisThread); 8910 } 8911 lock.readLock().unlock(); 8912 throw new IOException(e); 8913 } 8914 } 8915 8916 @Override 8917 public void closeRegionOperation() throws IOException { 8918 closeRegionOperation(Operation.ANY); 8919 } 8920 8921 @Override 8922 public void closeRegionOperation(Operation operation) throws IOException { 8923 if (operation == Operation.SNAPSHOT) { 8924 stores.values().forEach(HStore::postSnapshotOperation); 8925 } 8926 Thread thisThread = Thread.currentThread(); 8927 regionLockHolders.remove(thisThread); 8928 lock.readLock().unlock(); 8929 if (coprocessorHost != null) { 8930 coprocessorHost.postCloseRegionOperation(operation); 8931 } 8932 } 8933 8934 /** 8935 * This method needs to be called before any public call that reads or 8936 * modifies stores in bulk. It has to be called just before a try. 8937 * #closeBulkRegionOperation needs to be called in the try's finally block 8938 * Acquires a writelock and checks if the region is closing or closed. 8939 * @throws NotServingRegionException when the region is closing or closed 8940 * @throws RegionTooBusyException if failed to get the lock in time 8941 * @throws InterruptedIOException if interrupted while waiting for a lock 8942 */ 8943 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException { 8944 if (this.closing.get()) { 8945 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); 8946 } 8947 if (writeLockNeeded) lock(lock.writeLock()); 8948 else lock(lock.readLock()); 8949 if (this.closed.get()) { 8950 if (writeLockNeeded) lock.writeLock().unlock(); 8951 else lock.readLock().unlock(); 8952 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); 8953 } 8954 regionLockHolders.put(Thread.currentThread(), true); 8955 } 8956 8957 /** 8958 * Closes the lock. This needs to be called in the finally block corresponding 8959 * to the try block of #startRegionOperation 8960 */ 8961 private void closeBulkRegionOperation(){ 8962 regionLockHolders.remove(Thread.currentThread()); 8963 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock(); 8964 else lock.readLock().unlock(); 8965 } 8966 8967 /** 8968 * Update LongAdders for number of puts without wal and the size of possible data loss. 8969 * These information are exposed by the region server metrics. 8970 */ 8971 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) { 8972 numMutationsWithoutWAL.increment(); 8973 if (numMutationsWithoutWAL.sum() <= 1) { 8974 LOG.info("writing data to region " + this + 8975 " with WAL disabled. Data may be lost in the event of a crash."); 8976 } 8977 8978 long mutationSize = 0; 8979 for (List<Cell> cells: familyMap.values()) { 8980 // Optimization: 'foreach' loop is not used. See: 8981 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 8982 assert cells instanceof RandomAccess; 8983 int listSize = cells.size(); 8984 for (int i=0; i < listSize; i++) { 8985 Cell cell = cells.get(i); 8986 mutationSize += cell.getSerializedSize(); 8987 } 8988 } 8989 8990 dataInMemoryWithoutWAL.add(mutationSize); 8991 } 8992 8993 private void lock(final Lock lock) throws IOException { 8994 lock(lock, 1); 8995 } 8996 8997 /** 8998 * Try to acquire a lock. Throw RegionTooBusyException 8999 * if failed to get the lock in time. Throw InterruptedIOException 9000 * if interrupted while waiting for the lock. 9001 */ 9002 private void lock(final Lock lock, final int multiplier) throws IOException { 9003 try { 9004 final long waitTime = Math.min(maxBusyWaitDuration, 9005 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier)); 9006 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) { 9007 // Don't print millis. Message is used as a key over in 9008 // RetriesExhaustedWithDetailsException processing. 9009 final String regionName = 9010 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString(); 9011 final String serverName = this.getRegionServerServices() == null ? 9012 "unknown" : (this.getRegionServerServices().getServerName() == null ? 9013 "unknown" : this.getRegionServerServices().getServerName().toString()); 9014 RegionTooBusyException rtbe = new RegionTooBusyException( 9015 "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName); 9016 LOG.warn("Region is too busy to allow lock acquisition.", rtbe); 9017 throw rtbe; 9018 } 9019 } catch (InterruptedException ie) { 9020 if (LOG.isDebugEnabled()) { 9021 LOG.debug("Interrupted while waiting for a lock in region {}", this); 9022 } 9023 throw throwOnInterrupt(ie); 9024 } 9025 } 9026 9027 /** 9028 * Calls sync with the given transaction ID 9029 * @param txid should sync up to which transaction 9030 * @throws IOException If anything goes wrong with DFS 9031 */ 9032 private void sync(long txid, Durability durability) throws IOException { 9033 if (this.getRegionInfo().isMetaRegion()) { 9034 this.wal.sync(txid); 9035 } else { 9036 switch(durability) { 9037 case USE_DEFAULT: 9038 // do what table defaults to 9039 if (shouldSyncWAL()) { 9040 this.wal.sync(txid); 9041 } 9042 break; 9043 case SKIP_WAL: 9044 // nothing do to 9045 break; 9046 case ASYNC_WAL: 9047 // nothing do to 9048 break; 9049 case SYNC_WAL: 9050 this.wal.sync(txid, false); 9051 break; 9052 case FSYNC_WAL: 9053 this.wal.sync(txid, true); 9054 break; 9055 default: 9056 throw new RuntimeException("Unknown durability " + durability); 9057 } 9058 } 9059 } 9060 9061 /** 9062 * Check whether we should sync the wal from the table's durability settings 9063 */ 9064 private boolean shouldSyncWAL() { 9065 return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal(); 9066 } 9067 9068 /** 9069 * A mocked list implementation - discards all updates. 9070 */ 9071 private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() { 9072 9073 @Override 9074 public void add(int index, Cell element) { 9075 // do nothing 9076 } 9077 9078 @Override 9079 public boolean addAll(int index, Collection<? extends Cell> c) { 9080 return false; // this list is never changed as a result of an update 9081 } 9082 9083 @Override 9084 public KeyValue get(int index) { 9085 throw new UnsupportedOperationException(); 9086 } 9087 9088 @Override 9089 public int size() { 9090 return 0; 9091 } 9092 }; 9093 9094 /** @return the latest sequence number that was read from storage when this region was opened */ 9095 public long getOpenSeqNum() { 9096 return this.openSeqNum; 9097 } 9098 9099 @Override 9100 public Map<byte[], Long> getMaxStoreSeqId() { 9101 return this.maxSeqIdInStores; 9102 } 9103 9104 public long getOldestSeqIdOfStore(byte[] familyName) { 9105 return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName); 9106 } 9107 9108 @Override 9109 public CompactionState getCompactionState() { 9110 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0; 9111 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR) 9112 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE)); 9113 } 9114 9115 public void reportCompactionRequestStart(boolean isMajor){ 9116 (isMajor ? majorInProgress : minorInProgress).incrementAndGet(); 9117 } 9118 9119 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) { 9120 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet(); 9121 9122 // metrics 9123 compactionsFinished.increment(); 9124 compactionNumFilesCompacted.add(numFiles); 9125 compactionNumBytesCompacted.add(filesSizeCompacted); 9126 9127 assert newValue >= 0; 9128 } 9129 9130 public void reportCompactionRequestFailure() { 9131 compactionsFailed.increment(); 9132 } 9133 9134 public void incrementCompactionsQueuedCount() { 9135 compactionsQueued.increment(); 9136 } 9137 9138 public void decrementCompactionsQueuedCount() { 9139 compactionsQueued.decrement(); 9140 } 9141 9142 public void incrementFlushesQueuedCount() { 9143 flushesQueued.increment(); 9144 } 9145 9146 public long getReadPoint() { 9147 return getReadPoint(IsolationLevel.READ_COMMITTED); 9148 } 9149 9150 /** 9151 * If a handler thread is eligible for interrupt, make it ineligible. Should be paired 9152 * with {{@link #enableInterrupts()}. 9153 */ 9154 protected void disableInterrupts() { 9155 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> false); 9156 } 9157 9158 /** 9159 * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()}, 9160 * make it eligible again. No-op if interrupts are already enabled. 9161 */ 9162 protected void enableInterrupts() { 9163 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> true); 9164 } 9165 9166 /** 9167 * Interrupt any region options that have acquired the region lock via 9168 * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)}, 9169 * or {@link #startBulkRegionOperation(boolean)}. 9170 */ 9171 private void interruptRegionOperations() { 9172 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) { 9173 // An entry in this map will have a boolean value indicating if it is currently 9174 // eligible for interrupt; if so, we should interrupt it. 9175 if (entry.getValue().booleanValue()) { 9176 entry.getKey().interrupt(); 9177 } 9178 } 9179 } 9180 9181 /** 9182 * Check thread interrupt status and throw an exception if interrupted. 9183 * @throws NotServingRegionException if region is closing 9184 * @throws InterruptedIOException if interrupted but region is not closing 9185 */ 9186 // Package scope for tests 9187 void checkInterrupt() throws NotServingRegionException, InterruptedIOException { 9188 if (Thread.interrupted()) { 9189 if (this.closing.get()) { 9190 throw new NotServingRegionException( 9191 getRegionInfo().getRegionNameAsString() + " is closing"); 9192 } 9193 throw new InterruptedIOException(); 9194 } 9195 } 9196 9197 /** 9198 * Throw the correct exception upon interrupt 9199 * @param t cause 9200 */ 9201 // Package scope for tests 9202 IOException throwOnInterrupt(Throwable t) { 9203 if (this.closing.get()) { 9204 return (NotServingRegionException) new NotServingRegionException( 9205 getRegionInfo().getRegionNameAsString() + " is closing") 9206 .initCause(t); 9207 } 9208 return (InterruptedIOException) new InterruptedIOException().initCause(t); 9209 } 9210 9211 /** 9212 * {@inheritDoc} 9213 */ 9214 @Override 9215 public void onConfigurationChange(Configuration conf) { 9216 this.storeHotnessProtector.update(conf); 9217 } 9218 9219 /** 9220 * {@inheritDoc} 9221 */ 9222 @Override 9223 public void registerChildren(ConfigurationManager manager) { 9224 configurationManager = manager; 9225 stores.values().forEach(manager::registerObserver); 9226 } 9227 9228 /** 9229 * {@inheritDoc} 9230 */ 9231 @Override 9232 public void deregisterChildren(ConfigurationManager manager) { 9233 stores.values().forEach(configurationManager::deregisterObserver); 9234 } 9235 9236 @Override 9237 public CellComparator getCellComparator() { 9238 return cellComparator; 9239 } 9240 9241 public long getMemStoreFlushSize() { 9242 return this.memstoreFlushSize; 9243 } 9244 9245 9246 //// method for debugging tests 9247 void throwException(String title, String regionName) { 9248 StringBuilder buf = new StringBuilder(); 9249 buf.append(title + ", "); 9250 buf.append(getRegionInfo().toString()); 9251 buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " "); 9252 buf.append("stores: "); 9253 for (HStore s : stores.values()) { 9254 buf.append(s.getColumnFamilyDescriptor().getNameAsString()); 9255 buf.append(" size: "); 9256 buf.append(s.getMemStoreSize().getDataSize()); 9257 buf.append(" "); 9258 } 9259 buf.append("end-of-stores"); 9260 buf.append(", memstore size "); 9261 buf.append(getMemStoreDataSize()); 9262 if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) { 9263 throw new RuntimeException(buf.toString()); 9264 } 9265 } 9266 9267 @Override 9268 public void requestCompaction(String why, int priority, boolean major, 9269 CompactionLifeCycleTracker tracker) throws IOException { 9270 if (major) { 9271 stores.values().forEach(HStore::triggerMajorCompaction); 9272 } 9273 rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker, 9274 RpcServer.getRequestUser().orElse(null)); 9275 } 9276 9277 @Override 9278 public void requestCompaction(byte[] family, String why, int priority, boolean major, 9279 CompactionLifeCycleTracker tracker) throws IOException { 9280 HStore store = stores.get(family); 9281 if (store == null) { 9282 throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family) + 9283 " does not exist in region " + getRegionInfo().getRegionNameAsString()); 9284 } 9285 if (major) { 9286 store.triggerMajorCompaction(); 9287 } 9288 rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker, 9289 RpcServer.getRequestUser().orElse(null)); 9290 } 9291 9292 private void requestFlushIfNeeded() throws RegionTooBusyException { 9293 if(isFlushSize(this.memStoreSizing.getMemStoreSize())) { 9294 requestFlush(); 9295 } 9296 } 9297 9298 private void requestFlush() { 9299 if (this.rsServices == null) { 9300 return; 9301 } 9302 requestFlush0(FlushLifeCycleTracker.DUMMY); 9303 } 9304 9305 private void requestFlush0(FlushLifeCycleTracker tracker) { 9306 boolean shouldFlush = false; 9307 synchronized (writestate) { 9308 if (!this.writestate.isFlushRequested()) { 9309 shouldFlush = true; 9310 writestate.flushRequested = true; 9311 } 9312 } 9313 if (shouldFlush) { 9314 // Make request outside of synchronize block; HBASE-818. 9315 this.rsServices.getFlushRequester().requestFlush(this, tracker); 9316 if (LOG.isDebugEnabled()) { 9317 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName()); 9318 } 9319 } else { 9320 tracker.notExecuted("Flush already requested on " + this); 9321 } 9322 } 9323 9324 @Override 9325 public void requestFlush(FlushLifeCycleTracker tracker) throws IOException { 9326 requestFlush0(tracker); 9327 } 9328 9329 /** 9330 * This method modifies the region's configuration in order to inject replication-related 9331 * features 9332 * @param conf region configurations 9333 */ 9334 static void decorateRegionConfiguration(Configuration conf) { 9335 if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) { 9336 String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,""); 9337 String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName(); 9338 if (!plugins.contains(replicationCoprocessorClass)) { 9339 conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, 9340 (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass); 9341 } 9342 } 9343 } 9344 9345 public void setReadRequestsCount(long readRequestsCount) { 9346 this.readRequestsCount.add(readRequestsCount); 9347 } 9348 9349 public void setWriteRequestsCount(long writeRequestsCount) { 9350 this.writeRequestsCount.add(writeRequestsCount); 9351 } 9352}