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