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