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