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