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