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