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