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