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