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 * Do preparation for pending compaction. 2253 */ 2254 protected void doRegionCompactionPrep() throws IOException { 2255 } 2256 2257 /** 2258 * Synchronously compact all stores in the region. 2259 * <p> 2260 * This operation could block for a long time, so don't call it from a time-sensitive thread. 2261 * <p> 2262 * Note that no locks are taken to prevent possible conflicts between compaction and splitting 2263 * activities. The regionserver does not normally compact and split in parallel. However by 2264 * calling this method you may introduce unexpected and unhandled concurrency. Don't do this 2265 * unless you know what you are doing. 2266 * @param majorCompaction True to force a major compaction regardless of thresholds 2267 */ 2268 public void compact(boolean majorCompaction) throws IOException { 2269 if (majorCompaction) { 2270 stores.values().forEach(HStore::triggerMajorCompaction); 2271 } 2272 for (HStore s : stores.values()) { 2273 Optional<CompactionContext> compaction = s.requestCompaction(); 2274 if (compaction.isPresent()) { 2275 ThroughputController controller = null; 2276 if (rsServices != null) { 2277 controller = CompactionThroughputControllerFactory.create(rsServices, conf); 2278 } 2279 if (controller == null) { 2280 controller = NoLimitThroughputController.INSTANCE; 2281 } 2282 compact(compaction.get(), s, controller, null); 2283 } 2284 } 2285 } 2286 2287 /** 2288 * This is a helper function that compact all the stores synchronously. 2289 * <p> 2290 * It is used by utilities and testing 2291 */ 2292 public void compactStores() throws IOException { 2293 for (HStore s : stores.values()) { 2294 Optional<CompactionContext> compaction = s.requestCompaction(); 2295 if (compaction.isPresent()) { 2296 compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null); 2297 } 2298 } 2299 } 2300 2301 /** 2302 * This is a helper function that compact the given store. 2303 * <p> 2304 * It is used by utilities and testing 2305 */ 2306 void compactStore(byte[] family, ThroughputController throughputController) throws IOException { 2307 HStore s = getStore(family); 2308 Optional<CompactionContext> compaction = s.requestCompaction(); 2309 if (compaction.isPresent()) { 2310 compact(compaction.get(), s, throughputController, null); 2311 } 2312 } 2313 2314 /** 2315 * Called by compaction thread and after region is opened to compact the HStores if necessary. 2316 * <p> 2317 * This operation could block for a long time, so don't call it from a time-sensitive thread. Note 2318 * that no locking is necessary at this level because compaction only conflicts with a region 2319 * split, and that cannot happen because the region server does them sequentially and not in 2320 * parallel. 2321 * @param compaction Compaction details, obtained by requestCompaction() 2322 * @return whether the compaction completed 2323 */ 2324 public boolean compact(CompactionContext compaction, HStore store, 2325 ThroughputController throughputController) throws IOException { 2326 return compact(compaction, store, throughputController, null); 2327 } 2328 2329 private boolean shouldForbidMajorCompaction() { 2330 if (rsServices != null && rsServices.getReplicationSourceService() != null) { 2331 return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider() 2332 .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get()); 2333 } 2334 return false; 2335 } 2336 2337 /** 2338 * <p> 2339 * We are trying to remove / relax the region read lock for compaction. Let's see what are the 2340 * potential race conditions among the operations (user scan, region split, region close and 2341 * region bulk load). 2342 * </p> 2343 * 2344 * <pre> 2345 * user scan ---> region read lock 2346 * region split --> region close first --> region write lock 2347 * region close --> region write lock 2348 * region bulk load --> region write lock 2349 * </pre> 2350 * <p> 2351 * read lock is compatible with read lock. ---> no problem with user scan/read region bulk load 2352 * does not cause problem for compaction (no consistency problem, store lock will help the store 2353 * file accounting). They can run almost concurrently at the region level. 2354 * </p> 2355 * <p> 2356 * The only remaining race condition is between the region close and compaction. So we will 2357 * evaluate, below, how region close intervenes with compaction if compaction does not acquire 2358 * region read lock. 2359 * </p> 2360 * <p> 2361 * Here are the steps for compaction: 2362 * <ol> 2363 * <li>obtain list of StoreFile's</li> 2364 * <li>create StoreFileScanner's based on list from #1</li> 2365 * <li>perform compaction and save resulting files under tmp dir</li> 2366 * <li>swap in compacted files</li> 2367 * </ol> 2368 * </p> 2369 * <p> 2370 * #1 is guarded by store lock. This patch does not change this --> no worse or better For #2, we 2371 * obtain smallest read point (for region) across all the Scanners (for both default compactor and 2372 * stripe compactor). The read points are for user scans. Region keeps the read points for all 2373 * currently open user scanners. Compaction needs to know the smallest read point so that during 2374 * re-write of the hfiles, it can remove the mvcc points for the cells if their mvccs are older 2375 * than the smallest since they are not needed anymore. This will not conflict with compaction. 2376 * </p> 2377 * <p> 2378 * For #3, it can be performed in parallel to other operations. 2379 * </p> 2380 * <p> 2381 * For #4 bulk load and compaction don't conflict with each other on the region level (for 2382 * multi-family atomicy). 2383 * </p> 2384 * <p> 2385 * Region close and compaction are guarded pretty well by the 'writestate'. In HRegion#doClose(), 2386 * we have : 2387 * 2388 * <pre> 2389 * synchronized (writestate) { 2390 * // Disable compacting and flushing by background threads for this 2391 * // region. 2392 * canFlush = !writestate.readOnly; 2393 * writestate.writesEnabled = false; 2394 * LOG.debug("Closing " + this + ": disabling compactions & flushes"); 2395 * waitForFlushesAndCompactions(); 2396 * } 2397 * </pre> 2398 * 2399 * {@code waitForFlushesAndCompactions()} would wait for {@code writestate.compacting} to come 2400 * down to 0. and in {@code HRegion.compact()} 2401 * 2402 * <pre> 2403 * try { 2404 * synchronized (writestate) { 2405 * if (writestate.writesEnabled) { 2406 * wasStateSet = true; 2407 * ++writestate.compacting; 2408 * } else { 2409 * String msg = "NOT compacting region " + this + ". Writes disabled."; 2410 * LOG.info(msg); 2411 * status.abort(msg); 2412 * return false; 2413 * } 2414 * } 2415 * } 2416 * </pre> 2417 * 2418 * Also in {@code compactor.performCompaction()}: check periodically to see if a system stop is 2419 * requested 2420 * 2421 * <pre> 2422 * if (closeChecker != null && closeChecker.isTimeLimit(store, now)) { 2423 * progress.cancel(); 2424 * return false; 2425 * } 2426 * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) { 2427 * progress.cancel(); 2428 * return false; 2429 * } 2430 * </pre> 2431 * </p> 2432 */ 2433 public boolean compact(CompactionContext compaction, HStore store, 2434 ThroughputController throughputController, User user) throws IOException { 2435 assert compaction != null && compaction.hasSelection(); 2436 assert !compaction.getRequest().getFiles().isEmpty(); 2437 if (this.closing.get() || this.closed.get()) { 2438 LOG.debug("Skipping compaction on " + this + " because closing/closed"); 2439 store.cancelRequestedCompaction(compaction); 2440 return false; 2441 } 2442 2443 if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) { 2444 LOG.warn("Skipping major compaction on " + this 2445 + " because this cluster is transiting sync replication state" 2446 + " from STANDBY to DOWNGRADE_ACTIVE"); 2447 store.cancelRequestedCompaction(compaction); 2448 return false; 2449 } 2450 2451 MonitoredTask status = null; 2452 boolean requestNeedsCancellation = true; 2453 try { 2454 byte[] cf = Bytes.toBytes(store.getColumnFamilyName()); 2455 if (stores.get(cf) != store) { 2456 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this 2457 + " has been re-instantiated, cancel this compaction request. " 2458 + " It may be caused by the roll back of split transaction"); 2459 return false; 2460 } 2461 2462 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this); 2463 if (this.closed.get()) { 2464 String msg = "Skipping compaction on " + this + " because closed"; 2465 LOG.debug(msg); 2466 status.abort(msg); 2467 return false; 2468 } 2469 boolean wasStateSet = false; 2470 try { 2471 synchronized (writestate) { 2472 if (writestate.writesEnabled) { 2473 wasStateSet = true; 2474 writestate.compacting.incrementAndGet(); 2475 } else { 2476 String msg = "NOT compacting region " + this + ". Writes disabled."; 2477 LOG.info(msg); 2478 status.abort(msg); 2479 return false; 2480 } 2481 } 2482 LOG.info("Starting compaction of {} in {}{}", store, this, 2483 (compaction.getRequest().isOffPeak() ? " as an off-peak compaction" : "")); 2484 doRegionCompactionPrep(); 2485 try { 2486 status.setStatus("Compacting store " + store); 2487 // We no longer need to cancel the request on the way out of this 2488 // method because Store#compact will clean up unconditionally 2489 requestNeedsCancellation = false; 2490 store.compact(compaction, throughputController, user); 2491 } catch (InterruptedIOException iioe) { 2492 String msg = "region " + this + " compaction interrupted"; 2493 LOG.info(msg, iioe); 2494 status.abort(msg); 2495 return false; 2496 } 2497 } finally { 2498 if (wasStateSet) { 2499 synchronized (writestate) { 2500 writestate.compacting.decrementAndGet(); 2501 if (writestate.compacting.get() <= 0) { 2502 writestate.notifyAll(); 2503 } 2504 } 2505 } 2506 } 2507 status.markComplete("Compaction complete"); 2508 return true; 2509 } finally { 2510 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction); 2511 if (status != null) { 2512 LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 2513 status.prettyPrintJournal()); 2514 status.cleanup(); 2515 } 2516 } 2517 } 2518 2519 /** 2520 * Flush the cache. 2521 * <p> 2522 * When this method is called the cache will be flushed unless: 2523 * <ol> 2524 * <li>the cache is empty</li> 2525 * <li>the region is closed.</li> 2526 * <li>a flush is already in progress</li> 2527 * <li>writes are disabled</li> 2528 * </ol> 2529 * <p> 2530 * This method may block for some time, so it should not be called from a time-sensitive thread. 2531 * @param flushAllStores whether we want to force a flush of all stores 2532 * @return FlushResult indicating whether the flush was successful or not and if the region needs 2533 * compacting 2534 * @throws IOException general io exceptions because a snapshot was not properly persisted. 2535 */ 2536 // TODO HBASE-18905. We might have to expose a requestFlush API for CPs 2537 public FlushResult flush(boolean flushAllStores) throws IOException { 2538 return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY); 2539 } 2540 2541 public interface FlushResult { 2542 enum Result { 2543 FLUSHED_NO_COMPACTION_NEEDED, 2544 FLUSHED_COMPACTION_NEEDED, 2545 // Special case where a flush didn't run because there's nothing in the memstores. Used when 2546 // bulk loading to know when we can still load even if a flush didn't happen. 2547 CANNOT_FLUSH_MEMSTORE_EMPTY, 2548 CANNOT_FLUSH 2549 } 2550 2551 /** Returns the detailed result code */ 2552 Result getResult(); 2553 2554 /** Returns true if the memstores were flushed, else false */ 2555 boolean isFlushSucceeded(); 2556 2557 /** Returns True if the flush requested a compaction, else false */ 2558 boolean isCompactionNeeded(); 2559 } 2560 2561 public FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker, 2562 FlushLifeCycleTracker tracker) throws IOException { 2563 List<byte[]> families = null; 2564 if (flushAllStores) { 2565 families = new ArrayList<>(); 2566 families.addAll(this.getTableDescriptor().getColumnFamilyNames()); 2567 } 2568 return this.flushcache(families, writeFlushRequestWalMarker, tracker); 2569 } 2570 2571 /** 2572 * Flush the cache. When this method is called the cache will be flushed unless: 2573 * <ol> 2574 * <li>the cache is empty</li> 2575 * <li>the region is closed.</li> 2576 * <li>a flush is already in progress</li> 2577 * <li>writes are disabled</li> 2578 * </ol> 2579 * <p> 2580 * This method may block for some time, so it should not be called from a time-sensitive thread. 2581 * @param families stores of region to flush. 2582 * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL 2583 * @param tracker used to track the life cycle of this flush 2584 * @return whether the flush is success and whether the region needs compacting 2585 * @throws IOException general io exceptions 2586 * @throws DroppedSnapshotException Thrown when replay of wal is required because a Snapshot was 2587 * not properly persisted. The region is put in closing mode, and 2588 * the caller MUST abort after this. 2589 */ 2590 public FlushResultImpl flushcache(List<byte[]> families, boolean writeFlushRequestWalMarker, 2591 FlushLifeCycleTracker tracker) throws IOException { 2592 // fail-fast instead of waiting on the lock 2593 if (this.closing.get()) { 2594 String msg = "Skipping flush on " + this + " because closing"; 2595 LOG.debug(msg); 2596 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2597 } 2598 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this); 2599 status.setStatus("Acquiring readlock on region"); 2600 // block waiting for the lock for flushing cache 2601 lock.readLock().lock(); 2602 boolean flushed = true; 2603 try { 2604 if (this.closed.get()) { 2605 String msg = "Skipping flush on " + this + " because closed"; 2606 LOG.debug(msg); 2607 status.abort(msg); 2608 flushed = false; 2609 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2610 } 2611 if (coprocessorHost != null) { 2612 status.setStatus("Running coprocessor pre-flush hooks"); 2613 coprocessorHost.preFlush(tracker); 2614 } 2615 // TODO: this should be managed within memstore with the snapshot, updated only after flush 2616 // successful 2617 if (numMutationsWithoutWAL.sum() > 0) { 2618 numMutationsWithoutWAL.reset(); 2619 dataInMemoryWithoutWAL.reset(); 2620 } 2621 synchronized (writestate) { 2622 if (!writestate.flushing && writestate.writesEnabled) { 2623 this.writestate.flushing = true; 2624 } else { 2625 String msg = "NOT flushing " + this + " as " 2626 + (writestate.flushing ? "already flushing" : "writes are not enabled"); 2627 LOG.debug(msg); 2628 status.abort(msg); 2629 flushed = false; 2630 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false); 2631 } 2632 } 2633 2634 try { 2635 // The reason that we do not always use flushPolicy is, when the flush is 2636 // caused by logRoller, we should select stores which must be flushed 2637 // rather than could be flushed. 2638 Collection<HStore> specificStoresToFlush = null; 2639 if (families != null) { 2640 specificStoresToFlush = getSpecificStores(families); 2641 } else { 2642 specificStoresToFlush = flushPolicy.selectStoresToFlush(); 2643 } 2644 FlushResultImpl fs = 2645 internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker); 2646 2647 if (coprocessorHost != null) { 2648 status.setStatus("Running post-flush coprocessor hooks"); 2649 coprocessorHost.postFlush(tracker); 2650 } 2651 2652 if (fs.isFlushSucceeded()) { 2653 flushesQueued.reset(); 2654 } 2655 2656 status.markComplete("Flush successful " + fs.toString()); 2657 return fs; 2658 } finally { 2659 synchronized (writestate) { 2660 writestate.flushing = false; 2661 this.writestate.flushRequested = false; 2662 writestate.notifyAll(); 2663 } 2664 } 2665 } finally { 2666 lock.readLock().unlock(); 2667 if (flushed) { 2668 // Don't log this journal stuff if no flush -- confusing. 2669 LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(), 2670 status.prettyPrintJournal()); 2671 } 2672 status.cleanup(); 2673 } 2674 } 2675 2676 /** 2677 * get stores which matches the specified families 2678 * @return the stores need to be flushed. 2679 */ 2680 private Collection<HStore> getSpecificStores(List<byte[]> families) { 2681 Collection<HStore> specificStoresToFlush = new ArrayList<>(); 2682 for (byte[] family : families) { 2683 specificStoresToFlush.add(stores.get(family)); 2684 } 2685 return specificStoresToFlush; 2686 } 2687 2688 /** 2689 * Should the store be flushed because it is old enough. 2690 * <p> 2691 * Every FlushPolicy should call this to determine whether a store is old enough to flush (except 2692 * that you always flush all stores). Otherwise the method will always returns true which will 2693 * make a lot of flush requests. 2694 */ 2695 boolean shouldFlushStore(HStore store) { 2696 long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), 2697 store.getColumnFamilyDescriptor().getName()) - 1; 2698 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) { 2699 if (LOG.isDebugEnabled()) { 2700 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " 2701 + getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest 2702 + " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint()); 2703 } 2704 return true; 2705 } 2706 if (this.flushCheckInterval <= 0) { 2707 return false; 2708 } 2709 long now = EnvironmentEdgeManager.currentTime(); 2710 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) { 2711 if (LOG.isDebugEnabled()) { 2712 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " 2713 + getRegionInfo().getEncodedName() + " because time of oldest edit=" 2714 + store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now); 2715 } 2716 return true; 2717 } 2718 return false; 2719 } 2720 2721 /** 2722 * Should the memstore be flushed now 2723 */ 2724 boolean shouldFlush(final StringBuilder whyFlush) { 2725 whyFlush.setLength(0); 2726 // This is a rough measure. 2727 if ( 2728 this.maxFlushedSeqId > 0 2729 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint()) 2730 ) { 2731 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush"); 2732 return true; 2733 } 2734 long modifiedFlushCheckInterval = flushCheckInterval; 2735 if ( 2736 getRegionInfo().getTable().isSystemTable() 2737 && getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID 2738 ) { 2739 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL; 2740 } 2741 if (modifiedFlushCheckInterval <= 0) { // disabled 2742 return false; 2743 } 2744 long now = EnvironmentEdgeManager.currentTime(); 2745 // if we flushed in the recent past, we don't need to do again now 2746 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) { 2747 return false; 2748 } 2749 // since we didn't flush in the recent past, flush now if certain conditions 2750 // are met. Return true on first such memstore hit. 2751 for (HStore s : stores.values()) { 2752 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) { 2753 // we have an old enough edit in the memstore, flush 2754 whyFlush.append(s.toString() + " has an old edit so flush to free WALs"); 2755 return true; 2756 } 2757 } 2758 return false; 2759 } 2760 2761 /** 2762 * Flushing all stores. 2763 * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker) 2764 */ 2765 private FlushResult internalFlushcache(MonitoredTask status) throws IOException { 2766 return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY); 2767 } 2768 2769 /** 2770 * Flushing given stores. 2771 * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker) 2772 */ 2773 private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status, 2774 boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException { 2775 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status, 2776 writeFlushWalMarker, tracker); 2777 } 2778 2779 /** 2780 * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the 2781 * memstore, all of which have also been written to the wal. We need to write those updates in the 2782 * memstore out to disk, while being able to process reads/writes as much as possible during the 2783 * flush operation. 2784 * <p> 2785 * This method may block for some time. Every time you call it, we up the regions sequence id even 2786 * if we don't flush; i.e. the returned region id will be at least one larger than the last edit 2787 * applied to this region. The returned id does not refer to an actual edit. The returned id can 2788 * be used for say installing a bulk loaded file just ahead of the last hfile that was the result 2789 * of this flush, etc. 2790 * @param wal Null if we're NOT to go via wal. 2791 * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file. 2792 * @param storesToFlush The list of stores to flush. 2793 * @return object describing the flush's state 2794 * @throws IOException general io exceptions 2795 * @throws DroppedSnapshotException Thrown when replay of WAL is required. 2796 */ 2797 protected FlushResultImpl internalFlushcache(WAL wal, long myseqid, 2798 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, 2799 FlushLifeCycleTracker tracker) throws IOException { 2800 PrepareFlushResult result = 2801 internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker); 2802 if (result.result == null) { 2803 return internalFlushCacheAndCommit(wal, status, result, storesToFlush); 2804 } else { 2805 return result.result; // early exit due to failure from prepare stage 2806 } 2807 } 2808 2809 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "DLS_DEAD_LOCAL_STORE", 2810 justification = "FindBugs seems confused about trxId") 2811 protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid, 2812 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker, 2813 FlushLifeCycleTracker tracker) throws IOException { 2814 if (this.rsServices != null && this.rsServices.isAborted()) { 2815 // Don't flush when server aborting, it's unsafe 2816 throw new IOException("Aborting flush because server is aborted..."); 2817 } 2818 final long startTime = EnvironmentEdgeManager.currentTime(); 2819 // If nothing to flush, return, but return with a valid unused sequenceId. 2820 // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a 2821 // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs 2822 // to no other that it can use to associate with the bulk load. Hence this little dance below 2823 // to go get one. 2824 if (this.memStoreSizing.getDataSize() <= 0) { 2825 // Take an update lock so no edits can come into memory just yet. 2826 this.updatesLock.writeLock().lock(); 2827 WriteEntry writeEntry = null; 2828 try { 2829 if (this.memStoreSizing.getDataSize() <= 0) { 2830 // Presume that if there are still no edits in the memstore, then there are no edits for 2831 // this region out in the WAL subsystem so no need to do any trickery clearing out 2832 // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for 2833 // sure just beyond the last appended region edit and not associated with any edit 2834 // (useful as marker when bulk loading, etc.). 2835 if (wal != null) { 2836 writeEntry = mvcc.begin(); 2837 long flushOpSeqId = writeEntry.getWriteNumber(); 2838 FlushResultImpl flushResult = new FlushResultImpl( 2839 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId, "Nothing to flush", 2840 writeCanNotFlushMarkerToWAL(writeEntry, wal, writeFlushWalMarker)); 2841 mvcc.completeAndWait(writeEntry); 2842 // Set to null so we don't complete it again down in finally block. 2843 writeEntry = null; 2844 return new PrepareFlushResult(flushResult, myseqid); 2845 } else { 2846 return new PrepareFlushResult(new FlushResultImpl( 2847 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid); 2848 } 2849 } 2850 } finally { 2851 if (writeEntry != null) { 2852 // If writeEntry is non-null, this operation failed; the mvcc transaction failed... 2853 // but complete it anyways so it doesn't block the mvcc queue. 2854 mvcc.complete(writeEntry); 2855 } 2856 this.updatesLock.writeLock().unlock(); 2857 } 2858 } 2859 logFatLineOnFlush(storesToFlush, myseqid); 2860 // Stop updates while we snapshot the memstore of all of these regions' stores. We only have 2861 // to do this for a moment. It is quick. We also set the memstore size to zero here before we 2862 // allow updates again so its value will represent the size of the updates received 2863 // during flush 2864 2865 // We have to take an update lock during snapshot, or else a write could end up in both snapshot 2866 // and memstore (makes it difficult to do atomic rows then) 2867 status.setStatus("Obtaining lock to block concurrent updates"); 2868 // block waiting for the lock for internal flush 2869 this.updatesLock.writeLock().lock(); 2870 status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName()); 2871 MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing(); 2872 2873 Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>(); 2874 for (HStore store : storesToFlush) { 2875 flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(), 2876 store.preFlushSeqIDEstimation()); 2877 } 2878 2879 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2880 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2881 TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR); 2882 // The sequence id of this flush operation which is used to log FlushMarker and pass to 2883 // createFlushContext to use as the store file's sequence id. It can be in advance of edits 2884 // still in the memstore, edits that are in other column families yet to be flushed. 2885 long flushOpSeqId = HConstants.NO_SEQNUM; 2886 // The max flushed sequence id after this flush operation completes. All edits in memstore 2887 // will be in advance of this sequence id. 2888 long flushedSeqId = HConstants.NO_SEQNUM; 2889 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes(); 2890 try { 2891 if (wal != null) { 2892 Long earliestUnflushedSequenceIdForTheRegion = 2893 wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq); 2894 if (earliestUnflushedSequenceIdForTheRegion == null) { 2895 // This should never happen. This is how startCacheFlush signals flush cannot proceed. 2896 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing."; 2897 status.setStatus(msg); 2898 return new PrepareFlushResult( 2899 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false), myseqid); 2900 } 2901 flushOpSeqId = getNextSequenceId(wal); 2902 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit 2903 flushedSeqId = earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM 2904 ? flushOpSeqId 2905 : earliestUnflushedSequenceIdForTheRegion.longValue() - 1; 2906 } else { 2907 // use the provided sequence Id as WAL is not being used for this flush. 2908 flushedSeqId = flushOpSeqId = myseqid; 2909 } 2910 2911 for (HStore s : storesToFlush) { 2912 storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), 2913 s.createFlushContext(flushOpSeqId, tracker)); 2914 // for writing stores to WAL 2915 committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); 2916 } 2917 2918 // write the snapshot start to WAL 2919 if (wal != null && !writestate.readOnly) { 2920 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH, 2921 getRegionInfo(), flushOpSeqId, committedFiles); 2922 // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH 2923 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, 2924 mvcc, regionReplicationSink.orElse(null)); 2925 } 2926 2927 // Prepare flush (take a snapshot) 2928 storeFlushCtxs.forEach((name, flush) -> { 2929 MemStoreSize snapshotSize = flush.prepare(); 2930 totalSizeOfFlushableStores.incMemStoreSize(snapshotSize); 2931 storeFlushableSize.put(name, snapshotSize); 2932 }); 2933 } catch (IOException ex) { 2934 doAbortFlushToWAL(wal, flushOpSeqId, committedFiles); 2935 throw ex; 2936 } finally { 2937 this.updatesLock.writeLock().unlock(); 2938 } 2939 String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " 2940 + "flushsize=" + totalSizeOfFlushableStores; 2941 status.setStatus(s); 2942 doSyncOfUnflushedWALChanges(wal, getRegionInfo()); 2943 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime, 2944 flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores); 2945 } 2946 2947 /** 2948 * Utility method broken out of internalPrepareFlushCache so that method is smaller. 2949 */ 2950 private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) { 2951 if (!LOG.isInfoEnabled()) { 2952 return; 2953 } 2954 // Log a fat line detailing what is being flushed. 2955 StringBuilder perCfExtras = null; 2956 if (!isAllFamilies(storesToFlush)) { 2957 perCfExtras = new StringBuilder(); 2958 for (HStore store : storesToFlush) { 2959 MemStoreSize mss = store.getFlushableSize(); 2960 perCfExtras.append("; ").append(store.getColumnFamilyName()); 2961 perCfExtras.append("={dataSize=").append(StringUtils.byteDesc(mss.getDataSize())); 2962 perCfExtras.append(", heapSize=").append(StringUtils.byteDesc(mss.getHeapSize())); 2963 perCfExtras.append(", offHeapSize=").append(StringUtils.byteDesc(mss.getOffHeapSize())); 2964 perCfExtras.append("}"); 2965 } 2966 } 2967 MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); 2968 LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " + storesToFlush.size() + "/" 2969 + stores.size() + " column families," + " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) 2970 + " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) 2971 + ((perCfExtras != null && perCfExtras.length() > 0) ? perCfExtras.toString() : "") 2972 + ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId)); 2973 } 2974 2975 private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId, 2976 final Map<byte[], List<Path>> committedFiles) { 2977 if (wal == null) return; 2978 try { 2979 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, 2980 getRegionInfo(), flushOpSeqId, committedFiles); 2981 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, mvcc, 2982 null); 2983 } catch (Throwable t) { 2984 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in " 2985 + " region {}", StringUtils.stringifyException(t), this); 2986 // ignore this since we will be aborting the RS with DSE. 2987 } 2988 // we have called wal.startCacheFlush(), now we have to abort it 2989 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes()); 2990 } 2991 2992 /** 2993 * Sync unflushed WAL changes. See HBASE-8208 for details 2994 */ 2995 private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri) 2996 throws IOException { 2997 if (wal == null) { 2998 return; 2999 } 3000 try { 3001 wal.sync(); // ensure that flush marker is sync'ed 3002 } catch (IOException ioe) { 3003 wal.abortCacheFlush(hri.getEncodedNameAsBytes()); 3004 throw ioe; 3005 } 3006 } 3007 3008 /** Returns True if passed Set is all families in the region. */ 3009 private boolean isAllFamilies(Collection<HStore> families) { 3010 return families == null || this.stores.size() == families.size(); 3011 } 3012 3013 /** 3014 * This method is only used when we flush but the memstore is empty,if writeFlushWalMarker is 3015 * true,we write the {@link FlushAction#CANNOT_FLUSH} flush marker to WAL when the memstore is 3016 * empty. Ignores exceptions from WAL. Returns whether the write succeeded. 3017 * @return whether WAL write was successful 3018 */ 3019 private boolean writeCanNotFlushMarkerToWAL(WriteEntry flushOpSeqIdMVCCEntry, WAL wal, 3020 boolean writeFlushWalMarker) { 3021 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH, getRegionInfo(), 3022 -1, new TreeMap<>(Bytes.BYTES_COMPARATOR)); 3023 RegionReplicationSink sink = regionReplicationSink.orElse(null); 3024 3025 if (sink != null && !writeFlushWalMarker) { 3026 /** 3027 * Here for replication to secondary region replica could use {@link FlushAction#CANNOT_FLUSH} 3028 * to recover when writeFlushWalMarker is false, we create {@link WALEdit} for 3029 * {@link FlushDescriptor} and attach the {@link RegionReplicationSink#add} to the 3030 * flushOpSeqIdMVCCEntry,see HBASE-26960 for more details. 3031 */ 3032 this.attachRegionReplicationToFlushOpSeqIdMVCCEntry(flushOpSeqIdMVCCEntry, desc, sink); 3033 return false; 3034 } 3035 3036 if (writeFlushWalMarker && wal != null && !writestate.readOnly) { 3037 try { 3038 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc, 3039 sink); 3040 return true; 3041 } catch (IOException e) { 3042 LOG.warn(getRegionInfo().getEncodedName() + " : " 3043 + "Received exception while trying to write the flush request to wal", e); 3044 } 3045 } 3046 return false; 3047 } 3048 3049 /** 3050 * Create {@link WALEdit} for {@link FlushDescriptor} and attach {@link RegionReplicationSink#add} 3051 * to the flushOpSeqIdMVCCEntry. 3052 */ 3053 private void attachRegionReplicationToFlushOpSeqIdMVCCEntry(WriteEntry flushOpSeqIdMVCCEntry, 3054 FlushDescriptor desc, RegionReplicationSink sink) { 3055 assert !flushOpSeqIdMVCCEntry.getCompletionAction().isPresent(); 3056 WALEdit flushMarkerWALEdit = WALEdit.createFlushWALEdit(getRegionInfo(), desc); 3057 WALKeyImpl walKey = 3058 WALUtil.createWALKey(getRegionInfo(), mvcc, this.getReplicationScope(), null); 3059 walKey.setWriteEntry(flushOpSeqIdMVCCEntry); 3060 /** 3061 * Here the {@link ServerCall} is null for {@link RegionReplicationSink#add} because the 3062 * flushMarkerWALEdit is created by ourselves, not from rpc. 3063 */ 3064 flushOpSeqIdMVCCEntry.attachCompletionAction(() -> sink.add(walKey, flushMarkerWALEdit, null)); 3065 } 3066 3067 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY", 3068 justification = "Intentional; notify is about completed flush") 3069 FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status, 3070 PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException { 3071 // prepare flush context is carried via PrepareFlushResult 3072 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs; 3073 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles; 3074 long startTime = prepareResult.startTime; 3075 long flushOpSeqId = prepareResult.flushOpSeqId; 3076 long flushedSeqId = prepareResult.flushedSeqId; 3077 3078 String s = "Flushing stores of " + this; 3079 status.setStatus(s); 3080 if (LOG.isTraceEnabled()) LOG.trace(s); 3081 3082 // Any failure from here on out will be catastrophic requiring server 3083 // restart so wal content can be replayed and put back into the memstore. 3084 // Otherwise, the snapshot content while backed up in the wal, it will not 3085 // be part of the current running servers state. 3086 boolean compactionRequested = false; 3087 long flushedOutputFileSize = 0; 3088 try { 3089 // A. Flush memstore to all the HStores. 3090 // Keep running vector of all store files that includes both old and the 3091 // just-made new flush store file. The new flushed file is still in the 3092 // tmp directory. 3093 3094 for (StoreFlushContext flush : storeFlushCtxs.values()) { 3095 flush.flushCache(status); 3096 } 3097 3098 // Switch snapshot (in memstore) -> new hfile (thus causing 3099 // all the store scanners to reset/reseek). 3100 for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) { 3101 StoreFlushContext sfc = flushEntry.getValue(); 3102 boolean needsCompaction = sfc.commit(status); 3103 if (needsCompaction) { 3104 compactionRequested = true; 3105 } 3106 byte[] storeName = flushEntry.getKey(); 3107 List<Path> storeCommittedFiles = sfc.getCommittedFiles(); 3108 committedFiles.put(storeName, storeCommittedFiles); 3109 // Flush committed no files, indicating flush is empty or flush was canceled 3110 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) { 3111 MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName); 3112 prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize); 3113 } 3114 flushedOutputFileSize += sfc.getOutputFileSize(); 3115 } 3116 storeFlushCtxs.clear(); 3117 3118 // Set down the memstore size by amount of flush. 3119 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); 3120 this.decrMemStoreSize(mss); 3121 3122 // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled. 3123 // During startup, quota manager may not be initialized yet. 3124 if (rsServices != null) { 3125 RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager(); 3126 if (quotaManager != null) { 3127 quotaManager.getRegionSizeStore().incrementRegionSize(this.getRegionInfo(), 3128 flushedOutputFileSize); 3129 } 3130 } 3131 3132 if (wal != null) { 3133 // write flush marker to WAL. If fail, we should throw DroppedSnapshotException 3134 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH, 3135 getRegionInfo(), flushOpSeqId, committedFiles); 3136 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc, 3137 regionReplicationSink.orElse(null)); 3138 } 3139 } catch (Throwable t) { 3140 // An exception here means that the snapshot was not persisted. 3141 // The wal needs to be replayed so its content is restored to memstore. 3142 // Currently, only a server restart will do this. 3143 // We used to only catch IOEs but its possible that we'd get other 3144 // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch 3145 // all and sundry. 3146 if (wal != null) { 3147 try { 3148 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH, 3149 getRegionInfo(), flushOpSeqId, committedFiles); 3150 WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc, 3151 null); 3152 } catch (Throwable ex) { 3153 LOG.warn( 3154 getRegionInfo().getEncodedName() + " : " + "failed writing ABORT_FLUSH marker to WAL", 3155 ex); 3156 // ignore this since we will be aborting the RS with DSE. 3157 } 3158 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes()); 3159 } 3160 DroppedSnapshotException dse = new DroppedSnapshotException( 3161 "region: " + Bytes.toStringBinary(getRegionInfo().getRegionName()), t); 3162 status.abort("Flush failed: " + StringUtils.stringifyException(t)); 3163 3164 // Callers for flushcache() should catch DroppedSnapshotException and abort the region server. 3165 // However, since we may have the region read lock, we cannot call close(true) here since 3166 // we cannot promote to a write lock. Instead we are setting closing so that all other region 3167 // operations except for close will be rejected. 3168 this.closing.set(true); 3169 3170 if (rsServices != null) { 3171 // This is a safeguard against the case where the caller fails to explicitly handle aborting 3172 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse); 3173 } 3174 3175 throw dse; 3176 } 3177 3178 // If we get to here, the HStores have been written. 3179 if (wal != null) { 3180 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId); 3181 } 3182 3183 // Record latest flush time 3184 for (HStore store : storesToFlush) { 3185 this.lastStoreFlushTimeMap.put(store, startTime); 3186 } 3187 3188 this.maxFlushedSeqId = flushedSeqId; 3189 this.lastFlushOpSeqId = flushOpSeqId; 3190 3191 // C. Finally notify anyone waiting on memstore to clear: 3192 // e.g. checkResources(). 3193 synchronized (this) { 3194 notifyAll(); // FindBugs NN_NAKED_NOTIFY 3195 } 3196 3197 long time = EnvironmentEdgeManager.currentTime() - startTime; 3198 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize(); 3199 long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize(); 3200 String msg = "Finished flush of" + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" 3201 + mss.getDataSize() + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/" 3202 + mss.getHeapSize() + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/" 3203 + memstoresize + " for " + this.getRegionInfo().getEncodedName() + " in " + time 3204 + "ms, sequenceid=" + flushOpSeqId + ", compaction requested=" + compactionRequested 3205 + ((wal == null) ? "; wal=null" : ""); 3206 LOG.info(msg); 3207 status.setStatus(msg); 3208 3209 if (rsServices != null && rsServices.getMetrics() != null) { 3210 rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(), 3211 time, mss.getDataSize(), flushedOutputFileSize); 3212 } 3213 3214 return new FlushResultImpl(compactionRequested 3215 ? FlushResult.Result.FLUSHED_COMPACTION_NEEDED 3216 : FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId); 3217 } 3218 3219 /** 3220 * Method to safely get the next sequence number. 3221 * @return Next sequence number unassociated with any actual edit. 3222 */ 3223 protected long getNextSequenceId(final WAL wal) throws IOException { 3224 WriteEntry we = mvcc.begin(); 3225 mvcc.completeAndWait(we); 3226 return we.getWriteNumber(); 3227 } 3228 3229 ////////////////////////////////////////////////////////////////////////////// 3230 // get() methods for client use. 3231 ////////////////////////////////////////////////////////////////////////////// 3232 3233 @Override 3234 public RegionScannerImpl getScanner(Scan scan) throws IOException { 3235 return getScanner(scan, null); 3236 } 3237 3238 @Override 3239 public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners) 3240 throws IOException { 3241 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE); 3242 } 3243 3244 private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners, 3245 long nonceGroup, long nonce) throws IOException { 3246 return TraceUtil.trace(() -> { 3247 startRegionOperation(Operation.SCAN); 3248 try { 3249 // Verify families are all valid 3250 if (!scan.hasFamilies()) { 3251 // Adding all families to scanner 3252 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) { 3253 scan.addFamily(family); 3254 } 3255 } else { 3256 for (byte[] family : scan.getFamilyMap().keySet()) { 3257 checkFamily(family); 3258 } 3259 } 3260 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce); 3261 } finally { 3262 closeRegionOperation(Operation.SCAN); 3263 } 3264 }, () -> createRegionSpan("Region.getScanner")); 3265 } 3266 3267 protected RegionScannerImpl instantiateRegionScanner(Scan scan, 3268 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException { 3269 if (scan.isReversed()) { 3270 if (scan.getFilter() != null) { 3271 scan.getFilter().setReversed(true); 3272 } 3273 return new ReversedRegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce); 3274 } 3275 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce); 3276 } 3277 3278 /** 3279 * Prepare a delete for a row mutation processor 3280 * @param delete The passed delete is modified by this method. WARNING! 3281 */ 3282 private void prepareDelete(Delete delete) throws IOException { 3283 // Check to see if this is a deleteRow insert 3284 if (delete.getFamilyCellMap().isEmpty()) { 3285 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) { 3286 // Don't eat the timestamp 3287 delete.addFamily(family, delete.getTimestamp()); 3288 } 3289 } else { 3290 for (byte[] family : delete.getFamilyCellMap().keySet()) { 3291 if (family == null) { 3292 throw new NoSuchColumnFamilyException("Empty family is invalid"); 3293 } 3294 checkFamily(family, delete.getDurability()); 3295 } 3296 } 3297 } 3298 3299 @Override 3300 public void delete(Delete delete) throws IOException { 3301 TraceUtil.trace(() -> { 3302 checkReadOnly(); 3303 checkResources(); 3304 startRegionOperation(Operation.DELETE); 3305 try { 3306 // All edits for the given row (across all column families) must happen atomically. 3307 return mutate(delete); 3308 } finally { 3309 closeRegionOperation(Operation.DELETE); 3310 } 3311 }, () -> createRegionSpan("Region.delete")); 3312 } 3313 3314 /** 3315 * Set up correct timestamps in the KVs in Delete object. 3316 * <p/> 3317 * Caller should have the row and region locks. 3318 */ 3319 private void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<ExtendedCell>> familyMap, 3320 byte[] byteNow) throws IOException { 3321 for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) { 3322 3323 byte[] family = e.getKey(); 3324 List<ExtendedCell> cells = e.getValue(); 3325 assert cells instanceof RandomAccess; 3326 3327 Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR); 3328 int listSize = cells.size(); 3329 for (int i = 0; i < listSize; i++) { 3330 ExtendedCell cell = cells.get(i); 3331 // Check if time is LATEST, change to time of most recent addition if so 3332 // This is expensive. 3333 if ( 3334 cell.getTimestamp() == HConstants.LATEST_TIMESTAMP && PrivateCellUtil.isDeleteType(cell) 3335 ) { 3336 byte[] qual = CellUtil.cloneQualifier(cell); 3337 3338 Integer count = kvCount.get(qual); 3339 if (count == null) { 3340 kvCount.put(qual, 1); 3341 } else { 3342 kvCount.put(qual, count + 1); 3343 } 3344 count = kvCount.get(qual); 3345 3346 Get get = new Get(CellUtil.cloneRow(cell)); 3347 get.readVersions(count); 3348 get.addColumn(family, qual); 3349 if (coprocessorHost != null) { 3350 if ( 3351 !coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell, byteNow, get) 3352 ) { 3353 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow); 3354 } 3355 } else { 3356 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow); 3357 } 3358 } else { 3359 PrivateCellUtil.updateLatestStamp(cell, byteNow); 3360 } 3361 } 3362 } 3363 } 3364 3365 private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow) 3366 throws IOException { 3367 try (RegionScanner scanner = getScanner(new Scan(get))) { 3368 // NOTE: Please don't use HRegion.get() instead, 3369 // because it will copy cells to heap. See HBASE-26036 3370 List<ExtendedCell> result = new ArrayList<>(); 3371 scanner.next(result); 3372 3373 if (result.size() < count) { 3374 // Nothing to delete 3375 PrivateCellUtil.updateLatestStamp(cell, byteNow); 3376 return; 3377 } 3378 if (result.size() > count) { 3379 throw new RuntimeException("Unexpected size: " + result.size()); 3380 } 3381 Cell getCell = result.get(count - 1); 3382 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp()); 3383 } 3384 } 3385 3386 @Override 3387 public void put(Put put) throws IOException { 3388 TraceUtil.trace(() -> { 3389 checkReadOnly(); 3390 3391 // Do a rough check that we have resources to accept a write. The check is 3392 // 'rough' in that between the resource check and the call to obtain a 3393 // read lock, resources may run out. For now, the thought is that this 3394 // will be extremely rare; we'll deal with it when it happens. 3395 checkResources(); 3396 startRegionOperation(Operation.PUT); 3397 try { 3398 // All edits for the given row (across all column families) must happen atomically. 3399 return mutate(put); 3400 } finally { 3401 closeRegionOperation(Operation.PUT); 3402 } 3403 }, () -> createRegionSpan("Region.put")); 3404 } 3405 3406 /** 3407 * Class that tracks the progress of a batch operations, accumulating status codes and tracking 3408 * the index at which processing is proceeding. These batch operations may get split into 3409 * mini-batches for processing. 3410 */ 3411 private abstract static class BatchOperation<T> { 3412 protected final T[] operations; 3413 protected final OperationStatus[] retCodeDetails; 3414 protected final WALEdit[] walEditsFromCoprocessors; 3415 // reference family cell maps directly so coprocessors can mutate them if desired 3416 protected final Map<byte[], List<ExtendedCell>>[] familyCellMaps; 3417 // For Increment/Append operations 3418 protected final Result[] results; 3419 3420 protected final HRegion region; 3421 protected int nextIndexToProcess = 0; 3422 protected final ObservedExceptionsInBatch observedExceptions; 3423 // Durability of the batch (highest durability of all operations) 3424 protected Durability durability; 3425 protected boolean atomic = false; 3426 3427 public BatchOperation(final HRegion region, T[] operations) { 3428 this.operations = operations; 3429 this.retCodeDetails = new OperationStatus[operations.length]; 3430 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN); 3431 this.walEditsFromCoprocessors = new WALEdit[operations.length]; 3432 familyCellMaps = new Map[operations.length]; 3433 this.results = new Result[operations.length]; 3434 3435 this.region = region; 3436 observedExceptions = new ObservedExceptionsInBatch(); 3437 durability = Durability.USE_DEFAULT; 3438 } 3439 3440 /** 3441 * Visitor interface for batch operations 3442 */ 3443 @FunctionalInterface 3444 interface Visitor { 3445 /** 3446 * @param index operation index 3447 * @return If true continue visiting remaining entries, break otherwise 3448 */ 3449 boolean visit(int index) throws IOException; 3450 } 3451 3452 /** 3453 * Helper method for visiting pending/ all batch operations 3454 */ 3455 public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor) 3456 throws IOException { 3457 assert lastIndexExclusive <= this.size(); 3458 for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) { 3459 if (!pendingOnly || isOperationPending(i)) { 3460 if (!visitor.visit(i)) { 3461 break; 3462 } 3463 } 3464 } 3465 } 3466 3467 public abstract Mutation getMutation(int index); 3468 3469 public abstract long getNonceGroup(int index); 3470 3471 public abstract long getNonce(int index); 3472 3473 /** 3474 * This method is potentially expensive and useful mostly for non-replay CP path. 3475 */ 3476 public abstract Mutation[] getMutationsForCoprocs(); 3477 3478 public abstract boolean isInReplay(); 3479 3480 public abstract long getOrigLogSeqNum(); 3481 3482 public abstract void startRegionOperation() throws IOException; 3483 3484 public abstract void closeRegionOperation() throws IOException; 3485 3486 /** 3487 * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs 3488 * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This 3489 * is intended to operate on entire batch and will be called from outside of class to check and 3490 * prepare batch. This can be implemented by calling helper method 3491 * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations. 3492 */ 3493 public abstract void checkAndPrepare() throws IOException; 3494 3495 /** 3496 * Implement any Put request specific check and prepare logic here. Please refer to 3497 * {@link #checkAndPrepareMutation(Mutation, long)} for how its used. 3498 */ 3499 protected abstract void checkAndPreparePut(final Put p) throws IOException; 3500 3501 /** 3502 * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell 3503 * count, tags and timestamp for all cells of all operations in a mini-batch. 3504 */ 3505 public abstract void prepareMiniBatchOperations( 3506 MiniBatchOperationInProgress<Mutation> miniBatchOp, long timestamp, 3507 final List<RowLock> acquiredRowLocks) throws IOException; 3508 3509 /** 3510 * Write mini-batch operations to MemStore 3511 */ 3512 public abstract WriteEntry writeMiniBatchOperationsToMemStore( 3513 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry, 3514 long now) throws IOException; 3515 3516 protected void writeMiniBatchOperationsToMemStore( 3517 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber) 3518 throws IOException { 3519 MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing(); 3520 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 3521 // We need to update the sequence id for following reasons. 3522 // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id. 3523 // 2) If no WAL, FSWALEntry won't be used 3524 // we use durability of the original mutation for the mutation passed by CP. 3525 if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) { 3526 region.updateSequenceId(familyCellMaps[index].values(), writeNumber); 3527 } 3528 applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting); 3529 return true; 3530 }); 3531 // update memStore size 3532 region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(), 3533 memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount()); 3534 } 3535 3536 public boolean isDone() { 3537 return nextIndexToProcess == operations.length; 3538 } 3539 3540 public int size() { 3541 return operations.length; 3542 } 3543 3544 public boolean isOperationPending(int index) { 3545 return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN; 3546 } 3547 3548 public List<UUID> getClusterIds() { 3549 assert size() != 0; 3550 return getMutation(0).getClusterIds(); 3551 } 3552 3553 boolean isAtomic() { 3554 return atomic; 3555 } 3556 3557 /** 3558 * Helper method that checks and prepares only one mutation. This can be used to implement 3559 * {@link #checkAndPrepare()} for entire Batch. NOTE: As CP 3560 * prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations, this method 3561 * should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks are run for 3562 * the mutation 3563 */ 3564 protected void checkAndPrepareMutation(Mutation mutation, final long timestamp) 3565 throws IOException { 3566 region.checkRow(mutation.getRow(), "batchMutate"); 3567 if (mutation instanceof Put) { 3568 // Check the families in the put. If bad, skip this one. 3569 checkAndPreparePut((Put) mutation); 3570 region.checkTimestamps(mutation.getFamilyCellMap(), timestamp); 3571 } else if (mutation instanceof Delete) { 3572 region.prepareDelete((Delete) mutation); 3573 } else if (mutation instanceof Increment || mutation instanceof Append) { 3574 region.checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability()); 3575 } 3576 } 3577 3578 protected void checkAndPrepareMutation(int index, long timestamp) throws IOException { 3579 Mutation mutation = getMutation(index); 3580 try { 3581 this.checkAndPrepareMutation(mutation, timestamp); 3582 3583 if (mutation instanceof Put || mutation instanceof Delete) { 3584 // store the family map reference to allow for mutations 3585 // we know that in mutation, only ExtendedCells are allow so here we do a fake cast, to 3586 // simplify later logic 3587 familyCellMaps[index] = ClientInternalHelper.getExtendedFamilyCellMap(mutation); 3588 } 3589 3590 // store durability for the batch (highest durability of all operations in the batch) 3591 Durability tmpDur = region.getEffectiveDurability(mutation.getDurability()); 3592 if (tmpDur.ordinal() > durability.ordinal()) { 3593 durability = tmpDur; 3594 } 3595 } catch (NoSuchColumnFamilyException nscfe) { 3596 final String msg = "No such column family in batch mutation in region " + this; 3597 if (observedExceptions.hasSeenNoSuchFamily()) { 3598 LOG.warn(msg + nscfe.getMessage()); 3599 } else { 3600 LOG.warn(msg, nscfe); 3601 observedExceptions.sawNoSuchFamily(); 3602 } 3603 retCodeDetails[index] = 3604 new OperationStatus(OperationStatusCode.BAD_FAMILY, nscfe.getMessage()); 3605 if (isAtomic()) { // fail, atomic means all or none 3606 throw nscfe; 3607 } 3608 } catch (FailedSanityCheckException fsce) { 3609 final String msg = "Batch Mutation did not pass sanity check in region " + this; 3610 if (observedExceptions.hasSeenFailedSanityCheck()) { 3611 LOG.warn(msg + fsce.getMessage()); 3612 } else { 3613 LOG.warn(msg, fsce); 3614 observedExceptions.sawFailedSanityCheck(); 3615 } 3616 retCodeDetails[index] = 3617 new OperationStatus(OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage()); 3618 if (isAtomic()) { 3619 throw fsce; 3620 } 3621 } catch (WrongRegionException we) { 3622 final String msg = "Batch mutation had a row that does not belong to this region " + this; 3623 if (observedExceptions.hasSeenWrongRegion()) { 3624 LOG.warn(msg + we.getMessage()); 3625 } else { 3626 LOG.warn(msg, we); 3627 observedExceptions.sawWrongRegion(); 3628 } 3629 retCodeDetails[index] = 3630 new OperationStatus(OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage()); 3631 if (isAtomic()) { 3632 throw we; 3633 } 3634 } 3635 } 3636 3637 /** 3638 * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which a row 3639 * lock can be acquired. All mutations with locked rows are considered to be In-progress 3640 * operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch is window over 3641 * {@link BatchOperation} and contains contiguous pending operations. 3642 * @param acquiredRowLocks keeps track of rowLocks acquired. 3643 */ 3644 public MiniBatchOperationInProgress<Mutation> 3645 lockRowsAndBuildMiniBatch(List<RowLock> acquiredRowLocks) throws IOException { 3646 int readyToWriteCount = 0; 3647 int lastIndexExclusive = 0; 3648 RowLock prevRowLock = null; 3649 for (; lastIndexExclusive < size(); lastIndexExclusive++) { 3650 // It reaches the miniBatchSize, stop here and process the miniBatch 3651 // This only applies to non-atomic batch operations. 3652 if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) { 3653 break; 3654 } 3655 3656 if (!isOperationPending(lastIndexExclusive)) { 3657 continue; 3658 } 3659 3660 // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting 3661 // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation 3662 // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't 3663 // pass the isOperationPending check 3664 Map<byte[], List<Cell>> curFamilyCellMap = 3665 getMutation(lastIndexExclusive).getFamilyCellMap(); 3666 try { 3667 // start the protector before acquiring row lock considering performance, and will finish 3668 // it when encountering exception 3669 region.storeHotnessProtector.start(curFamilyCellMap); 3670 } catch (RegionTooBusyException rtbe) { 3671 region.storeHotnessProtector.finish(curFamilyCellMap); 3672 if (isAtomic()) { 3673 throw rtbe; 3674 } 3675 retCodeDetails[lastIndexExclusive] = 3676 new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage()); 3677 continue; 3678 } 3679 3680 Mutation mutation = getMutation(lastIndexExclusive); 3681 // If we haven't got any rows in our batch, we should block to get the next one. 3682 RowLock rowLock = null; 3683 boolean throwException = false; 3684 try { 3685 // if atomic then get exclusive lock, else shared lock 3686 rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock); 3687 } catch (TimeoutIOException | InterruptedIOException e) { 3688 // NOTE: We will retry when other exceptions, but we should stop if we receive 3689 // TimeoutIOException or InterruptedIOException as operation has timed out or 3690 // interrupted respectively. 3691 throwException = true; 3692 throw e; 3693 } catch (IOException ioe) { 3694 LOG.warn("Failed getting lock, row={}, in region {}", 3695 Bytes.toStringBinary(mutation.getRow()), this, ioe); 3696 if (isAtomic()) { // fail, atomic means all or none 3697 throwException = true; 3698 throw ioe; 3699 } 3700 } catch (Throwable throwable) { 3701 throwException = true; 3702 throw throwable; 3703 } finally { 3704 if (throwException) { 3705 region.storeHotnessProtector.finish(curFamilyCellMap); 3706 } 3707 } 3708 if (rowLock == null) { 3709 // We failed to grab another lock 3710 if (isAtomic()) { 3711 region.storeHotnessProtector.finish(curFamilyCellMap); 3712 throw new IOException("Can't apply all operations atomically!"); 3713 } 3714 break; // Stop acquiring more rows for this batch 3715 } else { 3716 if (rowLock != prevRowLock) { 3717 // It is a different row now, add this to the acquiredRowLocks and 3718 // set prevRowLock to the new returned rowLock 3719 acquiredRowLocks.add(rowLock); 3720 prevRowLock = rowLock; 3721 } 3722 } 3723 3724 readyToWriteCount++; 3725 } 3726 return createMiniBatch(lastIndexExclusive, readyToWriteCount); 3727 } 3728 3729 protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive, 3730 final int readyToWriteCount) { 3731 return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails, 3732 walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount); 3733 } 3734 3735 protected WALEdit createWALEdit(final MiniBatchOperationInProgress<Mutation> miniBatchOp) { 3736 return new WALEdit(miniBatchOp.getCellCount(), isInReplay()); 3737 } 3738 3739 /** 3740 * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are 3741 * present, they are merged to result WALEdit. 3742 */ 3743 public List<Pair<NonceKey, WALEdit>> 3744 buildWALEdits(final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException { 3745 List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>(); 3746 3747 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() { 3748 private Pair<NonceKey, WALEdit> curWALEditForNonce; 3749 3750 @Override 3751 public boolean visit(int index) throws IOException { 3752 Mutation m = getMutation(index); 3753 // we use durability of the original mutation for the mutation passed by CP. 3754 if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) { 3755 region.recordMutationWithoutWal(m.getFamilyCellMap()); 3756 /** 3757 * Here is for HBASE-26993,in order to make the new framework for region replication 3758 * could work for SKIP_WAL, we save the {@link Mutation} which 3759 * {@link Mutation#getDurability} is {@link Durability#SKIP_WAL} in miniBatchOp. 3760 */ 3761 cacheSkipWALMutationForRegionReplication(miniBatchOp, walEdits, familyCellMaps[index]); 3762 return true; 3763 } 3764 3765 // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each. 3766 // Given how nonce keys are originally written, these should be contiguous. 3767 // They don't have to be, it will still work, just write more WALEdits than needed. 3768 long nonceGroup = getNonceGroup(index); 3769 long nonce = getNonce(index); 3770 if ( 3771 curWALEditForNonce == null 3772 || curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup 3773 || curWALEditForNonce.getFirst().getNonce() != nonce 3774 ) { 3775 curWALEditForNonce = 3776 new Pair<>(new NonceKey(nonceGroup, nonce), createWALEdit(miniBatchOp)); 3777 walEdits.add(curWALEditForNonce); 3778 } 3779 WALEdit walEdit = curWALEditForNonce.getSecond(); 3780 3781 // Add WAL edits from CPs. 3782 WALEdit fromCP = walEditsFromCoprocessors[index]; 3783 List<ExtendedCell> cellsFromCP = fromCP == null 3784 ? Collections.emptyList() 3785 : WALEditInternalHelper.getExtendedCells(fromCP); 3786 addNonSkipWALMutationsToWALEdit(miniBatchOp, walEdit, cellsFromCP, familyCellMaps[index]); 3787 return true; 3788 } 3789 }); 3790 return walEdits; 3791 } 3792 3793 protected void addNonSkipWALMutationsToWALEdit( 3794 final MiniBatchOperationInProgress<Mutation> miniBatchOp, WALEdit walEdit, 3795 List<ExtendedCell> cellsFromCP, Map<byte[], List<ExtendedCell>> familyCellMap) { 3796 doAddCellsToWALEdit(walEdit, cellsFromCP, familyCellMap); 3797 } 3798 3799 protected static void doAddCellsToWALEdit(WALEdit walEdit, List<ExtendedCell> cellsFromCP, 3800 Map<byte[], List<ExtendedCell>> familyCellMap) { 3801 WALEditInternalHelper.addExtendedCell(walEdit, cellsFromCP); 3802 WALEditInternalHelper.addMap(walEdit, familyCellMap); 3803 } 3804 3805 protected abstract void cacheSkipWALMutationForRegionReplication( 3806 final MiniBatchOperationInProgress<Mutation> miniBatchOp, 3807 List<Pair<NonceKey, WALEdit>> walEdits, Map<byte[], List<ExtendedCell>> familyCellMap); 3808 3809 /** 3810 * This method completes mini-batch operations by calling postBatchMutate() CP hook (if 3811 * required) and completing mvcc. 3812 */ 3813 public void completeMiniBatchOperations( 3814 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 3815 throws IOException { 3816 if (writeEntry != null) { 3817 region.mvcc.completeAndWait(writeEntry); 3818 } 3819 } 3820 3821 public void doPostOpCleanupForMiniBatch( 3822 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit, 3823 boolean success) throws IOException { 3824 doFinishHotnessProtector(miniBatchOp); 3825 } 3826 3827 private void 3828 doFinishHotnessProtector(final MiniBatchOperationInProgress<Mutation> miniBatchOp) { 3829 // check and return if the protector is not enabled 3830 if (!region.storeHotnessProtector.isEnable()) { 3831 return; 3832 } 3833 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception. 3834 // This case was handled. 3835 if (miniBatchOp == null) { 3836 return; 3837 } 3838 3839 final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive(); 3840 3841 for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) { 3842 switch (retCodeDetails[i].getOperationStatusCode()) { 3843 case SUCCESS: 3844 case FAILURE: 3845 region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap()); 3846 break; 3847 default: 3848 // do nothing 3849 // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the 3850 // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start 3851 break; 3852 } 3853 } 3854 } 3855 3856 /** 3857 * Atomically apply the given map of family->edits to the memstore. This handles the consistency 3858 * control on its own, but the caller should already have locked updatesLock.readLock(). This 3859 * also does <b>not</b> check the families for validity. 3860 * @param familyMap Map of Cells by family 3861 */ 3862 protected void applyFamilyMapToMemStore(Map<byte[], List<ExtendedCell>> familyMap, 3863 MemStoreSizing memstoreAccounting) { 3864 for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) { 3865 byte[] family = e.getKey(); 3866 List<ExtendedCell> cells = e.getValue(); 3867 assert cells instanceof RandomAccess; 3868 region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting); 3869 } 3870 } 3871 } 3872 3873 /** 3874 * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most of 3875 * the logic is same. 3876 */ 3877 private static class MutationBatchOperation extends BatchOperation<Mutation> { 3878 3879 // For nonce operations 3880 private long nonceGroup; 3881 private long nonce; 3882 protected boolean canProceed; 3883 private boolean regionReplicateEnable; 3884 3885 public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic, 3886 long nonceGroup, long nonce) { 3887 super(region, operations); 3888 this.atomic = atomic; 3889 this.nonceGroup = nonceGroup; 3890 this.nonce = nonce; 3891 this.regionReplicateEnable = region.regionReplicationSink.isPresent(); 3892 } 3893 3894 @Override 3895 public Mutation getMutation(int index) { 3896 return this.operations[index]; 3897 } 3898 3899 @Override 3900 public long getNonceGroup(int index) { 3901 return nonceGroup; 3902 } 3903 3904 @Override 3905 public long getNonce(int index) { 3906 return nonce; 3907 } 3908 3909 @Override 3910 public Mutation[] getMutationsForCoprocs() { 3911 return this.operations; 3912 } 3913 3914 @Override 3915 public boolean isInReplay() { 3916 return false; 3917 } 3918 3919 @Override 3920 public long getOrigLogSeqNum() { 3921 return SequenceId.NO_SEQUENCE_ID; 3922 } 3923 3924 @Override 3925 public void startRegionOperation() throws IOException { 3926 region.startRegionOperation(Operation.BATCH_MUTATE); 3927 } 3928 3929 @Override 3930 public void closeRegionOperation() throws IOException { 3931 region.closeRegionOperation(Operation.BATCH_MUTATE); 3932 } 3933 3934 @Override 3935 public void checkAndPreparePut(Put p) throws IOException { 3936 region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability()); 3937 } 3938 3939 @Override 3940 public void checkAndPrepare() throws IOException { 3941 // index 0: puts, index 1: deletes, index 2: increments, index 3: append 3942 final int[] metrics = { 0, 0, 0, 0 }; 3943 3944 visitBatchOperations(true, this.size(), new Visitor() { 3945 private long now = EnvironmentEdgeManager.currentTime(); 3946 private WALEdit walEdit; 3947 3948 @Override 3949 public boolean visit(int index) throws IOException { 3950 // Run coprocessor pre hook outside of locks to avoid deadlock 3951 if (region.coprocessorHost != null) { 3952 if (walEdit == null) { 3953 walEdit = new WALEdit(); 3954 } 3955 callPreMutateCPHook(index, walEdit, metrics); 3956 if (!walEdit.isEmpty()) { 3957 walEditsFromCoprocessors[index] = walEdit; 3958 walEdit = null; 3959 } 3960 } 3961 if (isOperationPending(index)) { 3962 // TODO: Currently validation is done with current time before acquiring locks and 3963 // updates are done with different timestamps after acquiring locks. This behavior is 3964 // inherited from the code prior to this change. Can this be changed? 3965 checkAndPrepareMutation(index, now); 3966 } 3967 return true; 3968 } 3969 }); 3970 3971 // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in 3972 // normal processing. 3973 // Update metrics in same way as it is done when we go the normal processing route (we now 3974 // update general metrics though a Coprocessor did the work). 3975 if (region.metricsRegion != null) { 3976 if (metrics[0] > 0) { 3977 // There were some Puts in the batch. 3978 region.metricsRegion.updatePut(); 3979 } 3980 if (metrics[1] > 0) { 3981 // There were some Deletes in the batch. 3982 region.metricsRegion.updateDelete(); 3983 } 3984 if (metrics[2] > 0) { 3985 // There were some Increment in the batch. 3986 region.metricsRegion.updateIncrement(); 3987 } 3988 if (metrics[3] > 0) { 3989 // There were some Append in the batch. 3990 region.metricsRegion.updateAppend(); 3991 } 3992 } 3993 } 3994 3995 @Override 3996 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp, 3997 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException { 3998 // For nonce operations 3999 canProceed = startNonceOperation(); 4000 4001 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 4002 Mutation mutation = getMutation(index); 4003 if (mutation instanceof Put) { 4004 HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp)); 4005 miniBatchOp.incrementNumOfPuts(); 4006 } else if (mutation instanceof Delete) { 4007 region.prepareDeleteTimestamps(mutation, familyCellMaps[index], Bytes.toBytes(timestamp)); 4008 miniBatchOp.incrementNumOfDeletes(); 4009 } else if (mutation instanceof Increment || mutation instanceof Append) { 4010 boolean returnResults; 4011 if (mutation instanceof Increment) { 4012 returnResults = ((Increment) mutation).isReturnResults(); 4013 } else { 4014 returnResults = ((Append) mutation).isReturnResults(); 4015 } 4016 4017 // For nonce operations 4018 if (!canProceed) { 4019 Result result; 4020 if (returnResults) { 4021 // convert duplicate increment/append to get 4022 List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce); 4023 result = Result.create(results); 4024 } else { 4025 result = Result.EMPTY_RESULT; 4026 } 4027 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 4028 return true; 4029 } 4030 4031 Result result = null; 4032 if (region.coprocessorHost != null) { 4033 if (mutation instanceof Increment) { 4034 result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation); 4035 } else { 4036 result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation); 4037 } 4038 } 4039 if (result != null) { 4040 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, 4041 returnResults ? result : Result.EMPTY_RESULT); 4042 return true; 4043 } 4044 4045 List<ExtendedCell> results = returnResults ? new ArrayList<>(mutation.size()) : null; 4046 familyCellMaps[index] = reckonDeltas(mutation, results, timestamp); 4047 this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT; 4048 4049 if (mutation instanceof Increment) { 4050 miniBatchOp.incrementNumOfIncrements(); 4051 } else { 4052 miniBatchOp.incrementNumOfAppends(); 4053 } 4054 } 4055 region.rewriteCellTags(familyCellMaps[index], mutation); 4056 4057 // update cell count 4058 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) { 4059 for (List<Cell> cells : mutation.getFamilyCellMap().values()) { 4060 miniBatchOp.addCellCount(cells.size()); 4061 } 4062 } 4063 4064 WALEdit fromCP = walEditsFromCoprocessors[index]; 4065 if (fromCP != null) { 4066 miniBatchOp.addCellCount(fromCP.size()); 4067 } 4068 return true; 4069 }); 4070 4071 if (region.coprocessorHost != null) { 4072 // calling the pre CP hook for batch mutation 4073 region.coprocessorHost.preBatchMutate(miniBatchOp); 4074 checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp); 4075 } 4076 } 4077 4078 /** 4079 * Starts the nonce operation for a mutation, if needed. 4080 * @return whether to proceed this mutation. 4081 */ 4082 private boolean startNonceOperation() throws IOException { 4083 if ( 4084 region.rsServices == null || region.rsServices.getNonceManager() == null 4085 || nonce == HConstants.NO_NONCE 4086 ) { 4087 return true; 4088 } 4089 boolean canProceed; 4090 try { 4091 canProceed = 4092 region.rsServices.getNonceManager().startOperation(nonceGroup, nonce, region.rsServices); 4093 } catch (InterruptedException ex) { 4094 throw new InterruptedIOException("Nonce start operation interrupted"); 4095 } 4096 return canProceed; 4097 } 4098 4099 /** 4100 * Ends nonce operation for a mutation, if needed. 4101 * @param success Whether the operation for this nonce has succeeded. 4102 */ 4103 private void endNonceOperation(boolean success) { 4104 if ( 4105 region.rsServices != null && region.rsServices.getNonceManager() != null 4106 && nonce != HConstants.NO_NONCE 4107 ) { 4108 region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success); 4109 } 4110 } 4111 4112 private static Get toGet(final Mutation mutation) throws IOException { 4113 assert mutation instanceof Increment || mutation instanceof Append; 4114 Get get = new Get(mutation.getRow()); 4115 CellScanner cellScanner = mutation.cellScanner(); 4116 while (cellScanner.advance()) { 4117 Cell cell = cellScanner.current(); 4118 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell)); 4119 } 4120 if (mutation instanceof Increment) { 4121 // Increment 4122 Increment increment = (Increment) mutation; 4123 get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax()); 4124 } else { 4125 // Append 4126 Append append = (Append) mutation; 4127 get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax()); 4128 } 4129 for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) { 4130 get.setAttribute(entry.getKey(), entry.getValue()); 4131 } 4132 return get; 4133 } 4134 4135 private Map<byte[], List<ExtendedCell>> reckonDeltas(Mutation mutation, 4136 List<ExtendedCell> results, long now) throws IOException { 4137 assert mutation instanceof Increment || mutation instanceof Append; 4138 Map<byte[], List<ExtendedCell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR); 4139 // Process a Store/family at a time. 4140 for (Map.Entry<byte[], List<ExtendedCell>> entry : ClientInternalHelper 4141 .getExtendedFamilyCellMap(mutation).entrySet()) { 4142 final byte[] columnFamilyName = entry.getKey(); 4143 List<ExtendedCell> deltas = entry.getValue(); 4144 // Reckon for the Store what to apply to WAL and MemStore. 4145 List<ExtendedCell> toApply = 4146 reckonDeltasByStore(region.stores.get(columnFamilyName), mutation, now, deltas, results); 4147 if (!toApply.isEmpty()) { 4148 for (ExtendedCell cell : toApply) { 4149 HStore store = region.getStore(cell); 4150 if (store == null) { 4151 region.checkFamily(CellUtil.cloneFamily(cell)); 4152 } else { 4153 ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(), 4154 key -> new ArrayList<>()).add(cell); 4155 } 4156 } 4157 } 4158 } 4159 return ret; 4160 } 4161 4162 /** 4163 * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed column 4164 * family/Store. Does Get of current value and then adds passed in deltas for this Store 4165 * returning the result. 4166 * @param mutation The encompassing Mutation object 4167 * @param deltas Changes to apply to this Store; either increment amount or data to append 4168 * @param results In here we accumulate all the Cells we are to return to the client. If null, 4169 * client doesn't want results returned. 4170 * @return Resulting Cells after <code>deltas</code> have been applied to current values. Side 4171 * effect is our filling out of the <code>results</code> List. 4172 */ 4173 private List<ExtendedCell> reckonDeltasByStore(HStore store, Mutation mutation, long now, 4174 List<ExtendedCell> deltas, List<ExtendedCell> results) throws IOException { 4175 assert mutation instanceof Increment || mutation instanceof Append; 4176 byte[] columnFamily = store.getColumnFamilyDescriptor().getName(); 4177 List<Pair<ExtendedCell, ExtendedCell>> cellPairs = new ArrayList<>(deltas.size()); 4178 4179 // Sort the cells so that they match the order that they appear in the Get results. 4180 // Otherwise, we won't be able to find the existing values if the cells are not specified 4181 // in order by the client since cells are in an array list. 4182 deltas.sort(store.getComparator()); 4183 4184 // Get previous values for all columns in this family. 4185 Get get = new Get(mutation.getRow()); 4186 for (ExtendedCell cell : deltas) { 4187 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell)); 4188 } 4189 TimeRange tr; 4190 if (mutation instanceof Increment) { 4191 tr = ((Increment) mutation).getTimeRange(); 4192 } else { 4193 tr = ((Append) mutation).getTimeRange(); 4194 } 4195 4196 if (tr != null) { 4197 get.setTimeRange(tr.getMin(), tr.getMax()); 4198 } 4199 4200 try (RegionScanner scanner = region.getScanner(new Scan(get))) { 4201 // NOTE: Please don't use HRegion.get() instead, 4202 // because it will copy cells to heap. See HBASE-26036 4203 List<ExtendedCell> currentValues = new ArrayList<>(); 4204 scanner.next(currentValues); 4205 // Iterate the input columns and update existing values if they were found, otherwise 4206 // add new column initialized to the delta amount 4207 int currentValuesIndex = 0; 4208 for (int i = 0; i < deltas.size(); i++) { 4209 ExtendedCell delta = deltas.get(i); 4210 ExtendedCell currentValue = null; 4211 if ( 4212 currentValuesIndex < currentValues.size() 4213 && CellUtil.matchingQualifier(currentValues.get(currentValuesIndex), delta) 4214 ) { 4215 currentValue = currentValues.get(currentValuesIndex); 4216 if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) { 4217 currentValuesIndex++; 4218 } 4219 } 4220 // Switch on whether this an increment or an append building the new Cell to apply. 4221 ExtendedCell newCell; 4222 if (mutation instanceof Increment) { 4223 long deltaAmount = getLongValue(delta); 4224 final long newValue = 4225 currentValue == null ? deltaAmount : getLongValue(currentValue) + deltaAmount; 4226 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, 4227 (oldCell) -> Bytes.toBytes(newValue)); 4228 } else { 4229 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation, 4230 (oldCell) -> ByteBuffer 4231 .wrap(new byte[delta.getValueLength() + oldCell.getValueLength()]) 4232 .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength()) 4233 .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength()) 4234 .array()); 4235 } 4236 if (region.maxCellSize > 0) { 4237 int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell); 4238 if (newCellSize > region.maxCellSize) { 4239 String msg = "Cell with size " + newCellSize + " exceeds limit of " 4240 + region.maxCellSize + " bytes in region " + this; 4241 LOG.debug(msg); 4242 throw new DoNotRetryIOException(msg); 4243 } 4244 } 4245 cellPairs.add(new Pair<>(currentValue, newCell)); 4246 // Add to results to get returned to the Client. If null, cilent does not want results. 4247 if (results != null) { 4248 results.add(newCell); 4249 } 4250 } 4251 // Give coprocessors a chance to update the new cells before apply to WAL or memstore 4252 if (region.coprocessorHost != null) { 4253 // Here the operation must be increment or append. 4254 cellPairs = mutation instanceof Increment 4255 ? region.coprocessorHost.postIncrementBeforeWAL(mutation, (List) cellPairs) 4256 : region.coprocessorHost.postAppendBeforeWAL(mutation, (List) cellPairs); 4257 } 4258 } 4259 return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList()); 4260 } 4261 4262 private static ExtendedCell reckonDelta(final ExtendedCell delta, 4263 final ExtendedCell currentCell, final byte[] columnFamily, final long now, Mutation mutation, 4264 Function<ExtendedCell, byte[]> supplier) throws IOException { 4265 // Forward any tags found on the delta. 4266 List<Tag> tags = TagUtil.carryForwardTags(delta); 4267 if (currentCell != null) { 4268 tags = TagUtil.carryForwardTags(tags, currentCell); 4269 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL()); 4270 byte[] newValue = supplier.apply(currentCell); 4271 return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY) 4272 .setRow(mutation.getRow(), 0, mutation.getRow().length) 4273 .setFamily(columnFamily, 0, columnFamily.length) 4274 // copy the qualifier if the cell is located in shared memory. 4275 .setQualifier(CellUtil.cloneQualifier(delta)) 4276 .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now)) 4277 .setType(KeyValue.Type.Put.getCode()).setValue(newValue, 0, newValue.length) 4278 .setTags(TagUtil.fromList(tags)).build(); 4279 } else { 4280 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL()); 4281 PrivateCellUtil.updateLatestStamp(delta, now); 4282 ExtendedCell deltaCell = (ExtendedCell) delta; 4283 return CollectionUtils.isEmpty(tags) 4284 ? deltaCell 4285 : PrivateCellUtil.createCell(deltaCell, tags); 4286 } 4287 } 4288 4289 /** Returns Get the long out of the passed in Cell */ 4290 private static long getLongValue(final Cell cell) throws DoNotRetryIOException { 4291 int len = cell.getValueLength(); 4292 if (len != Bytes.SIZEOF_LONG) { 4293 // throw DoNotRetryIOException instead of IllegalArgumentException 4294 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide"); 4295 } 4296 return PrivateCellUtil.getValueAsLong(cell); 4297 } 4298 4299 @Override 4300 public List<Pair<NonceKey, WALEdit>> 4301 buildWALEdits(final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException { 4302 List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp); 4303 // for MutationBatchOperation, more than one nonce is not allowed 4304 if (walEdits.size() > 1) { 4305 throw new IOException("Found multiple nonce keys per batch!"); 4306 } 4307 return walEdits; 4308 } 4309 4310 /** 4311 * Here is for HBASE-26993,in order to make the new framework for region replication could work 4312 * for SKIP_WAL, we save the {@link Mutation} which {@link Mutation#getDurability} is 4313 * {@link Durability#SKIP_WAL} in miniBatchOp. 4314 */ 4315 @Override 4316 protected void cacheSkipWALMutationForRegionReplication( 4317 MiniBatchOperationInProgress<Mutation> miniBatchOp, 4318 List<Pair<NonceKey, WALEdit>> nonceKeyAndWALEdits, 4319 Map<byte[], List<ExtendedCell>> familyCellMap) { 4320 if (!this.regionReplicateEnable) { 4321 return; 4322 } 4323 4324 WALEdit walEditForReplicateIfExistsSkipWAL = 4325 miniBatchOp.getWalEditForReplicateIfExistsSkipWAL(); 4326 /** 4327 * When there is a SKIP_WAL {@link Mutation},we create a new {@link WALEdit} for replicating 4328 * to region replica,first we fill the existing {@link WALEdit} to it and then add the 4329 * {@link Mutation} which is SKIP_WAL to it. 4330 */ 4331 if (walEditForReplicateIfExistsSkipWAL == null) { 4332 walEditForReplicateIfExistsSkipWAL = 4333 this.createWALEditForReplicateSkipWAL(miniBatchOp, nonceKeyAndWALEdits); 4334 miniBatchOp.setWalEditForReplicateIfExistsSkipWAL(walEditForReplicateIfExistsSkipWAL); 4335 } 4336 WALEditInternalHelper.addMap(walEditForReplicateIfExistsSkipWAL, familyCellMap); 4337 4338 } 4339 4340 private WALEdit createWALEditForReplicateSkipWAL( 4341 MiniBatchOperationInProgress<Mutation> miniBatchOp, 4342 List<Pair<NonceKey, WALEdit>> nonceKeyAndWALEdits) { 4343 if (nonceKeyAndWALEdits.isEmpty()) { 4344 return this.createWALEdit(miniBatchOp); 4345 } 4346 // for MutationBatchOperation, more than one nonce is not allowed 4347 assert nonceKeyAndWALEdits.size() == 1; 4348 WALEdit currentWALEdit = nonceKeyAndWALEdits.get(0).getSecond(); 4349 return new WALEdit(currentWALEdit); 4350 } 4351 4352 @Override 4353 protected void addNonSkipWALMutationsToWALEdit( 4354 final MiniBatchOperationInProgress<Mutation> miniBatchOp, WALEdit walEdit, 4355 List<ExtendedCell> cellsFromCP, Map<byte[], List<ExtendedCell>> familyCellMap) { 4356 super.addNonSkipWALMutationsToWALEdit(miniBatchOp, walEdit, cellsFromCP, familyCellMap); 4357 WALEdit walEditForReplicateIfExistsSkipWAL = 4358 miniBatchOp.getWalEditForReplicateIfExistsSkipWAL(); 4359 if (walEditForReplicateIfExistsSkipWAL == null) { 4360 return; 4361 } 4362 /** 4363 * When walEditForReplicateIfExistsSkipWAL is not null,it means there exists SKIP_WAL 4364 * {@link Mutation} and we create a new {@link WALEdit} in 4365 * {@link MutationBatchOperation#cacheSkipWALMutationForReplicateRegionReplica} for 4366 * replicating to region replica, so here we also add non SKIP_WAL{@link Mutation}s to 4367 * walEditForReplicateIfExistsSkipWAL. 4368 */ 4369 doAddCellsToWALEdit(walEditForReplicateIfExistsSkipWAL, cellsFromCP, familyCellMap); 4370 } 4371 4372 @Override 4373 public WriteEntry writeMiniBatchOperationsToMemStore( 4374 final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry, 4375 long now) throws IOException { 4376 boolean newWriteEntry = false; 4377 if (writeEntry == null) { 4378 writeEntry = region.mvcc.begin(); 4379 newWriteEntry = true; 4380 } 4381 super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber()); 4382 if (newWriteEntry) { 4383 /** 4384 * Here is for HBASE-26993 case 2,all {@link Mutation}s are {@link Durability#SKIP_WAL}. In 4385 * order to make the new framework for region replication could work for SKIP_WAL,because 4386 * there is no {@link RegionReplicationSink#add} attached in {@link HRegion#doWALAppend},so 4387 * here we get {@link WALEdit} from 4388 * {@link MiniBatchOperationInProgress#getWalEditForReplicateIfExistsSkipWAL} and attach 4389 * {@link RegionReplicationSink#add} to the new mvcc writeEntry. 4390 */ 4391 attachRegionReplicationToMVCCEntry(miniBatchOp, writeEntry, now); 4392 } 4393 return writeEntry; 4394 } 4395 4396 private WALKeyImpl createWALKey(long now) { 4397 // for MutationBatchOperation,isReplay is false. 4398 return this.region.createWALKeyForWALAppend(false, this, now, this.nonceGroup, this.nonce); 4399 } 4400 4401 /** 4402 * Create {@link WALKeyImpl} and get {@link WALEdit} from miniBatchOp and attach 4403 * {@link RegionReplicationSink#add} to the mvccWriteEntry. 4404 */ 4405 private void attachRegionReplicationToMVCCEntry( 4406 final MiniBatchOperationInProgress<Mutation> miniBatchOp, WriteEntry mvccWriteEntry, long now) 4407 throws IOException { 4408 if (!this.regionReplicateEnable) { 4409 return; 4410 } 4411 assert !mvccWriteEntry.getCompletionAction().isPresent(); 4412 4413 final WALKeyImpl walKey = this.createWALKey(now); 4414 walKey.setWriteEntry(mvccWriteEntry); 4415 region.doAttachReplicateRegionReplicaAction(walKey, 4416 miniBatchOp.getWalEditForReplicateIfExistsSkipWAL(), mvccWriteEntry); 4417 } 4418 4419 @Override 4420 public void completeMiniBatchOperations( 4421 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 4422 throws IOException { 4423 // TODO: can it be done after completing mvcc? 4424 // calling the post CP hook for batch mutation 4425 if (region.coprocessorHost != null) { 4426 region.coprocessorHost.postBatchMutate(miniBatchOp); 4427 } 4428 super.completeMiniBatchOperations(miniBatchOp, writeEntry); 4429 4430 if (nonce != HConstants.NO_NONCE) { 4431 if (region.rsServices != null && region.rsServices.getNonceManager() != null) { 4432 region.rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce, 4433 writeEntry.getWriteNumber()); 4434 } 4435 } 4436 } 4437 4438 @Override 4439 public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp, 4440 final WALEdit walEdit, boolean success) throws IOException { 4441 4442 super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success); 4443 if (miniBatchOp != null) { 4444 // synced so that the coprocessor contract is adhered to. 4445 if (region.coprocessorHost != null) { 4446 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> { 4447 // only for successful puts/deletes/increments/appends 4448 if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) { 4449 Mutation m = getMutation(i); 4450 if (m instanceof Put) { 4451 region.coprocessorHost.postPut((Put) m, walEdit); 4452 } else if (m instanceof Delete) { 4453 region.coprocessorHost.postDelete((Delete) m, walEdit); 4454 } else if (m instanceof Increment) { 4455 Result result = 4456 region.getCoprocessorHost().postIncrement((Increment) m, results[i], walEdit); 4457 if (result != results[i]) { 4458 retCodeDetails[i] = 4459 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result); 4460 } 4461 } else if (m instanceof Append) { 4462 Result result = 4463 region.getCoprocessorHost().postAppend((Append) m, results[i], walEdit); 4464 if (result != results[i]) { 4465 retCodeDetails[i] = 4466 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result); 4467 } 4468 } 4469 } 4470 return true; 4471 }); 4472 } 4473 4474 // For nonce operations 4475 if (canProceed && nonce != HConstants.NO_NONCE) { 4476 boolean[] areAllIncrementsAndAppendsSuccessful = new boolean[] { true }; 4477 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> { 4478 Mutation mutation = getMutation(i); 4479 if (mutation instanceof Increment || mutation instanceof Append) { 4480 if (retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) { 4481 areAllIncrementsAndAppendsSuccessful[0] = false; 4482 return false; 4483 } 4484 } 4485 return true; 4486 }); 4487 endNonceOperation(areAllIncrementsAndAppendsSuccessful[0]); 4488 } 4489 4490 // See if the column families were consistent through the whole thing. 4491 // if they were then keep them. If they were not then pass a null. 4492 // null will be treated as unknown. 4493 // Total time taken might be involving Puts, Deletes, Increments and Appends. 4494 // Split the time for puts and deletes based on the total number of Puts, Deletes, 4495 // Increments and Appends. 4496 if (region.metricsRegion != null) { 4497 if (miniBatchOp.getNumOfPuts() > 0) { 4498 // There were some Puts in the batch. 4499 region.metricsRegion.updatePut(); 4500 } 4501 if (miniBatchOp.getNumOfDeletes() > 0) { 4502 // There were some Deletes in the batch. 4503 region.metricsRegion.updateDelete(); 4504 } 4505 if (miniBatchOp.getNumOfIncrements() > 0) { 4506 // There were some Increments in the batch. 4507 region.metricsRegion.updateIncrement(); 4508 } 4509 if (miniBatchOp.getNumOfAppends() > 0) { 4510 // There were some Appends in the batch. 4511 region.metricsRegion.updateAppend(); 4512 } 4513 } 4514 } 4515 4516 if (region.coprocessorHost != null) { 4517 // call the coprocessor hook to do any finalization steps after the put is done 4518 region.coprocessorHost.postBatchMutateIndispensably( 4519 miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success); 4520 } 4521 } 4522 4523 /** 4524 * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch 4525 * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2: 4526 * count of increments and 3: count of appends 4527 */ 4528 private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics) 4529 throws IOException { 4530 Mutation m = getMutation(index); 4531 if (m instanceof Put) { 4532 if (region.coprocessorHost.prePut((Put) m, walEdit)) { 4533 // pre hook says skip this Put 4534 // mark as success and skip in doMiniBatchMutation 4535 metrics[0]++; 4536 retCodeDetails[index] = OperationStatus.SUCCESS; 4537 } 4538 } else if (m instanceof Delete) { 4539 Delete curDel = (Delete) m; 4540 if (curDel.getFamilyCellMap().isEmpty()) { 4541 // handle deleting a row case 4542 // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook. 4543 // Can this be avoided? 4544 region.prepareDelete(curDel); 4545 } 4546 if (region.coprocessorHost.preDelete(curDel, walEdit)) { 4547 // pre hook says skip this Delete 4548 // mark as success and skip in doMiniBatchMutation 4549 metrics[1]++; 4550 retCodeDetails[index] = OperationStatus.SUCCESS; 4551 } 4552 } else if (m instanceof Increment) { 4553 Increment increment = (Increment) m; 4554 Result result = region.coprocessorHost.preIncrement(increment, walEdit); 4555 if (result != null) { 4556 // pre hook says skip this Increment 4557 // mark as success and skip in doMiniBatchMutation 4558 metrics[2]++; 4559 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 4560 } 4561 } else if (m instanceof Append) { 4562 Append append = (Append) m; 4563 Result result = region.coprocessorHost.preAppend(append, walEdit); 4564 if (result != null) { 4565 // pre hook says skip this Append 4566 // mark as success and skip in doMiniBatchMutation 4567 metrics[3]++; 4568 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result); 4569 } 4570 } else { 4571 String msg = "Put/Delete/Increment/Append mutations only supported in a batch"; 4572 retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg); 4573 if (isAtomic()) { // fail, atomic means all or none 4574 throw new IOException(msg); 4575 } 4576 } 4577 } 4578 4579 // TODO Support Increment/Append operations 4580 private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp, 4581 final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException { 4582 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> { 4583 // we pass (i - firstIndex) below since the call expects a relative index 4584 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess); 4585 if (cpMutations == null) { 4586 return true; 4587 } 4588 // Else Coprocessor added more Mutations corresponding to the Mutation at this index. 4589 Mutation mutation = getMutation(i); 4590 for (Mutation cpMutation : cpMutations) { 4591 this.checkAndPrepareMutation(cpMutation, timestamp); 4592 4593 // Acquire row locks. If not, the whole batch will fail. 4594 acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null)); 4595 4596 // Returned mutations from coprocessor correspond to the Mutation at index i. We can 4597 // directly add the cells from those mutations to the familyMaps of this mutation. 4598 Map<byte[], List<ExtendedCell>> cpFamilyMap = 4599 ClientInternalHelper.getExtendedFamilyCellMap(cpMutation); 4600 region.rewriteCellTags(cpFamilyMap, mutation); 4601 // will get added to the memStore later 4602 mergeFamilyMaps(familyCellMaps[i], cpFamilyMap); 4603 4604 // The durability of returned mutation is replaced by the corresponding mutation. 4605 // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the 4606 // cells of returned mutation. 4607 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) { 4608 for (List<ExtendedCell> cells : cpFamilyMap.values()) { 4609 miniBatchOp.addCellCount(cells.size()); 4610 } 4611 } 4612 } 4613 return true; 4614 }); 4615 } 4616 4617 private void mergeFamilyMaps(Map<byte[], List<ExtendedCell>> familyMap, 4618 Map<byte[], List<ExtendedCell>> toBeMerged) { 4619 for (Map.Entry<byte[], List<ExtendedCell>> entry : toBeMerged.entrySet()) { 4620 List<ExtendedCell> cells = familyMap.get(entry.getKey()); 4621 if (cells == null) { 4622 familyMap.put(entry.getKey(), entry.getValue()); 4623 } else { 4624 cells.addAll(entry.getValue()); 4625 } 4626 } 4627 } 4628 } 4629 4630 /** 4631 * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most 4632 * of the logic is same. 4633 * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we will not use this operation to apply 4634 * edits at secondary replica side. 4635 */ 4636 @Deprecated 4637 private static final class ReplayBatchOperation extends BatchOperation<MutationReplay> { 4638 4639 private long origLogSeqNum = 0; 4640 4641 public ReplayBatchOperation(final HRegion region, MutationReplay[] operations, 4642 long origLogSeqNum) { 4643 super(region, operations); 4644 this.origLogSeqNum = origLogSeqNum; 4645 } 4646 4647 @Override 4648 public Mutation getMutation(int index) { 4649 return this.operations[index].mutation; 4650 } 4651 4652 @Override 4653 public long getNonceGroup(int index) { 4654 return this.operations[index].nonceGroup; 4655 } 4656 4657 @Override 4658 public long getNonce(int index) { 4659 return this.operations[index].nonce; 4660 } 4661 4662 @Override 4663 public Mutation[] getMutationsForCoprocs() { 4664 return null; 4665 } 4666 4667 @Override 4668 public boolean isInReplay() { 4669 return true; 4670 } 4671 4672 @Override 4673 public long getOrigLogSeqNum() { 4674 return this.origLogSeqNum; 4675 } 4676 4677 @Override 4678 public void startRegionOperation() throws IOException { 4679 region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE); 4680 } 4681 4682 @Override 4683 public void closeRegionOperation() throws IOException { 4684 region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE); 4685 } 4686 4687 /** 4688 * During replay, there could exist column families which are removed between region server 4689 * failure and replay 4690 */ 4691 @Override 4692 protected void checkAndPreparePut(Put p) throws IOException { 4693 Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap(); 4694 List<byte[]> nonExistentList = null; 4695 for (byte[] family : familyCellMap.keySet()) { 4696 if (!region.htableDescriptor.hasColumnFamily(family)) { 4697 if (nonExistentList == null) { 4698 nonExistentList = new ArrayList<>(); 4699 } 4700 nonExistentList.add(family); 4701 } 4702 } 4703 if (nonExistentList != null) { 4704 for (byte[] family : nonExistentList) { 4705 // Perhaps schema was changed between crash and replay 4706 LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this); 4707 familyCellMap.remove(family); 4708 } 4709 } 4710 } 4711 4712 @Override 4713 public void checkAndPrepare() throws IOException { 4714 long now = EnvironmentEdgeManager.currentTime(); 4715 visitBatchOperations(true, this.size(), (int index) -> { 4716 checkAndPrepareMutation(index, now); 4717 return true; 4718 }); 4719 } 4720 4721 @Override 4722 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp, 4723 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException { 4724 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> { 4725 // update cell count 4726 for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) { 4727 miniBatchOp.addCellCount(cells.size()); 4728 } 4729 return true; 4730 }); 4731 } 4732 4733 @Override 4734 public WriteEntry writeMiniBatchOperationsToMemStore( 4735 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry, 4736 long now) throws IOException { 4737 super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum()); 4738 return writeEntry; 4739 } 4740 4741 @Override 4742 public void completeMiniBatchOperations( 4743 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry) 4744 throws IOException { 4745 super.completeMiniBatchOperations(miniBatchOp, writeEntry); 4746 region.mvcc.advanceTo(getOrigLogSeqNum()); 4747 } 4748 4749 @Override 4750 protected void cacheSkipWALMutationForRegionReplication( 4751 MiniBatchOperationInProgress<Mutation> miniBatchOp, List<Pair<NonceKey, WALEdit>> walEdits, 4752 Map<byte[], List<ExtendedCell>> familyCellMap) { 4753 // There is no action to do if current region is secondary replica 4754 } 4755 4756 } 4757 4758 public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup, 4759 long nonce) throws IOException { 4760 // As it stands, this is used for 3 things 4761 // * batchMutate with single mutation - put/delete/increment/append, separate or from 4762 // checkAndMutate. 4763 // * coprocessor calls (see ex. BulkDeleteEndpoint). 4764 // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd... 4765 return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce)); 4766 } 4767 4768 @Override 4769 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException { 4770 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically 4771 boolean atomic = 4772 Arrays.stream(mutations).anyMatch(m -> m instanceof Increment || m instanceof Append); 4773 return batchMutate(mutations, atomic); 4774 } 4775 4776 OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException { 4777 return TraceUtil.trace( 4778 () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE), 4779 () -> createRegionSpan("Region.batchMutate")); 4780 } 4781 4782 /** 4783 * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we use 4784 * {@link #replayWALEntry(WALEntry, CellScanner)} for replaying edits at secondary 4785 * replica side. 4786 */ 4787 @Deprecated 4788 OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException { 4789 if ( 4790 !RegionReplicaUtil.isDefaultReplica(getRegionInfo()) 4791 && replaySeqId < lastReplayedOpenRegionSeqId 4792 ) { 4793 // if it is a secondary replica we should ignore these entries silently 4794 // since they are coming out of order 4795 if (LOG.isTraceEnabled()) { 4796 LOG.trace(getRegionInfo().getEncodedName() + " : " + "Skipping " + mutations.length 4797 + " mutations with replaySeqId=" + replaySeqId 4798 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId); 4799 for (MutationReplay mut : mutations) { 4800 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation); 4801 } 4802 } 4803 4804 OperationStatus[] statuses = new OperationStatus[mutations.length]; 4805 for (int i = 0; i < statuses.length; i++) { 4806 statuses[i] = OperationStatus.SUCCESS; 4807 } 4808 return statuses; 4809 } 4810 return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId)); 4811 } 4812 4813 /** 4814 * Perform a batch of mutations. 4815 * <p/> 4816 * Operations in a batch are stored with highest durability specified of for all operations in a 4817 * batch, except for {@link Durability#SKIP_WAL}. 4818 * <p/> 4819 * This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with 4820 * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with 4821 * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch and 4822 * mutation batch is very similar, lot of code is shared by providing generic methods in base 4823 * class {@link BatchOperation}. The logic for this method and 4824 * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which are 4825 * overridden by derived classes to implement special behavior. 4826 * @param batchOp contains the list of mutations 4827 * @return an array of OperationStatus which internally contains the OperationStatusCode and the 4828 * exceptionMessage if any. 4829 * @throws IOException if an IO problem is encountered 4830 */ 4831 private OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException { 4832 boolean initialized = false; 4833 batchOp.startRegionOperation(); 4834 try { 4835 while (!batchOp.isDone()) { 4836 if (!batchOp.isInReplay()) { 4837 checkReadOnly(); 4838 } 4839 checkResources(); 4840 4841 if (!initialized) { 4842 this.writeRequestsCount.add(batchOp.size()); 4843 // validate and prepare batch for write, for MutationBatchOperation it also calls CP 4844 // prePut()/preDelete()/preIncrement()/preAppend() hooks 4845 batchOp.checkAndPrepare(); 4846 initialized = true; 4847 } 4848 doMiniBatchMutate(batchOp); 4849 requestFlushIfNeeded(); 4850 } 4851 } finally { 4852 if (rsServices != null && rsServices.getMetrics() != null) { 4853 rsServices.getMetrics().updateWriteQueryMeter(this, batchOp.size()); 4854 } 4855 batchOp.closeRegionOperation(); 4856 } 4857 return batchOp.retCodeDetails; 4858 } 4859 4860 /** 4861 * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])} In here we 4862 * also handle replay of edits on region recover. Also gets change in size brought about by 4863 * applying {@code batchOp}. 4864 */ 4865 private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException { 4866 boolean success = false; 4867 WALEdit walEdit = null; 4868 WriteEntry writeEntry = null; 4869 boolean locked = false; 4870 // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive) 4871 MiniBatchOperationInProgress<Mutation> miniBatchOp = null; 4872 /** Keep track of the locks we hold so we can release them in finally clause */ 4873 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size()); 4874 4875 // Check for thread interrupt status in case we have been signaled from 4876 // #interruptRegionOperation. 4877 checkInterrupt(); 4878 4879 try { 4880 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with 4881 // locked rows 4882 miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks); 4883 4884 // We've now grabbed as many mutations off the list as we can 4885 // Ensure we acquire at least one. 4886 if (miniBatchOp.getReadyToWriteCount() <= 0) { 4887 // Nothing to put/delete/increment/append -- an exception in the above such as 4888 // NoSuchColumnFamily? 4889 return; 4890 } 4891 4892 // Check for thread interrupt status in case we have been signaled from 4893 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for 4894 // the WAL append. 4895 checkInterrupt(); 4896 4897 lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount()); 4898 locked = true; 4899 4900 // From this point until memstore update this operation should not be interrupted. 4901 disableInterrupts(); 4902 4903 // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp 4904 // We should record the timestamp only after we have acquired the rowLock, 4905 // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer 4906 // timestamp 4907 4908 long now = EnvironmentEdgeManager.currentTime(); 4909 batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks); 4910 4911 // STEP 3. Build WAL edit 4912 4913 List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp); 4914 4915 // STEP 4. Append the WALEdits to WAL and sync. 4916 4917 for (Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) { 4918 Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next(); 4919 walEdit = nonceKeyWALEditPair.getSecond(); 4920 NonceKey nonceKey = nonceKeyWALEditPair.getFirst(); 4921 4922 if (walEdit != null && !walEdit.isEmpty()) { 4923 writeEntry = doWALAppend(walEdit, batchOp, miniBatchOp, now, nonceKey); 4924 } 4925 4926 // Complete mvcc for all but last writeEntry (for replay case) 4927 if (it.hasNext() && writeEntry != null) { 4928 mvcc.complete(writeEntry); 4929 writeEntry = null; 4930 } 4931 } 4932 4933 // STEP 5. Write back to memStore 4934 // NOTE: writeEntry can be null here 4935 writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry, now); 4936 4937 // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and 4938 // complete mvcc for last writeEntry 4939 batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry); 4940 writeEntry = null; 4941 success = true; 4942 } finally { 4943 // Call complete rather than completeAndWait because we probably had error if walKey != null 4944 if (writeEntry != null) mvcc.complete(writeEntry); 4945 4946 if (locked) { 4947 this.updatesLock.readLock().unlock(); 4948 } 4949 releaseRowLocks(acquiredRowLocks); 4950 4951 enableInterrupts(); 4952 4953 final int finalLastIndexExclusive = 4954 miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size(); 4955 final boolean finalSuccess = success; 4956 batchOp.visitBatchOperations(true, finalLastIndexExclusive, (int i) -> { 4957 Mutation mutation = batchOp.getMutation(i); 4958 if (mutation instanceof Increment || mutation instanceof Append) { 4959 if (finalSuccess) { 4960 batchOp.retCodeDetails[i] = 4961 new OperationStatus(OperationStatusCode.SUCCESS, batchOp.results[i]); 4962 } else { 4963 batchOp.retCodeDetails[i] = OperationStatus.FAILURE; 4964 } 4965 } else { 4966 batchOp.retCodeDetails[i] = 4967 finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE; 4968 } 4969 return true; 4970 }); 4971 4972 batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess); 4973 4974 batchOp.nextIndexToProcess = finalLastIndexExclusive; 4975 } 4976 } 4977 4978 /** 4979 * Returns effective durability from the passed durability and the table descriptor. 4980 */ 4981 private Durability getEffectiveDurability(Durability d) { 4982 return d == Durability.USE_DEFAULT ? this.regionDurability : d; 4983 } 4984 4985 @Override 4986 @Deprecated 4987 public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, 4988 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException { 4989 CheckAndMutate checkAndMutate; 4990 try { 4991 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row) 4992 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange); 4993 if (mutation instanceof Put) { 4994 checkAndMutate = builder.build((Put) mutation); 4995 } else if (mutation instanceof Delete) { 4996 checkAndMutate = builder.build((Delete) mutation); 4997 } else { 4998 throw new DoNotRetryIOException( 4999 "Unsupported mutate type: " + mutation.getClass().getSimpleName().toUpperCase()); 5000 } 5001 } catch (IllegalArgumentException e) { 5002 throw new DoNotRetryIOException(e.getMessage()); 5003 } 5004 return checkAndMutate(checkAndMutate).isSuccess(); 5005 } 5006 5007 @Override 5008 @Deprecated 5009 public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation) 5010 throws IOException { 5011 CheckAndMutate checkAndMutate; 5012 try { 5013 CheckAndMutate.Builder builder = 5014 CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange); 5015 if (mutation instanceof Put) { 5016 checkAndMutate = builder.build((Put) mutation); 5017 } else if (mutation instanceof Delete) { 5018 checkAndMutate = builder.build((Delete) mutation); 5019 } else { 5020 throw new DoNotRetryIOException( 5021 "Unsupported mutate type: " + mutation.getClass().getSimpleName().toUpperCase()); 5022 } 5023 } catch (IllegalArgumentException e) { 5024 throw new DoNotRetryIOException(e.getMessage()); 5025 } 5026 return checkAndMutate(checkAndMutate).isSuccess(); 5027 } 5028 5029 @Override 5030 @Deprecated 5031 public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op, 5032 ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException { 5033 CheckAndMutate checkAndMutate; 5034 try { 5035 checkAndMutate = CheckAndMutate.newBuilder(row) 5036 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm); 5037 } catch (IllegalArgumentException e) { 5038 throw new DoNotRetryIOException(e.getMessage()); 5039 } 5040 return checkAndMutate(checkAndMutate).isSuccess(); 5041 } 5042 5043 @Override 5044 @Deprecated 5045 public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm) 5046 throws IOException { 5047 CheckAndMutate checkAndMutate; 5048 try { 5049 checkAndMutate = 5050 CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange).build(rm); 5051 } catch (IllegalArgumentException e) { 5052 throw new DoNotRetryIOException(e.getMessage()); 5053 } 5054 return checkAndMutate(checkAndMutate).isSuccess(); 5055 } 5056 5057 @Override 5058 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException { 5059 return checkAndMutate(checkAndMutate, HConstants.NO_NONCE, HConstants.NO_NONCE); 5060 } 5061 5062 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup, 5063 long nonce) throws IOException { 5064 return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce), 5065 () -> createRegionSpan("Region.checkAndMutate")); 5066 } 5067 5068 private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate, 5069 long nonceGroup, long nonce) throws IOException { 5070 byte[] row = checkAndMutate.getRow(); 5071 Filter filter = null; 5072 byte[] family = null; 5073 byte[] qualifier = null; 5074 CompareOperator op = null; 5075 ByteArrayComparable comparator = null; 5076 if (checkAndMutate.hasFilter()) { 5077 filter = checkAndMutate.getFilter(); 5078 } else { 5079 family = checkAndMutate.getFamily(); 5080 qualifier = checkAndMutate.getQualifier(); 5081 op = checkAndMutate.getCompareOp(); 5082 comparator = new BinaryComparator(checkAndMutate.getValue()); 5083 } 5084 TimeRange timeRange = checkAndMutate.getTimeRange(); 5085 5086 Mutation mutation = null; 5087 RowMutations rowMutations = null; 5088 if (checkAndMutate.getAction() instanceof Mutation) { 5089 mutation = (Mutation) checkAndMutate.getAction(); 5090 } else { 5091 rowMutations = (RowMutations) checkAndMutate.getAction(); 5092 } 5093 5094 if (mutation != null) { 5095 checkMutationType(mutation); 5096 checkRow(mutation, row); 5097 } else { 5098 checkRow(rowMutations, row); 5099 } 5100 checkReadOnly(); 5101 // TODO, add check for value length also move this check to the client 5102 checkResources(); 5103 startRegionOperation(); 5104 try { 5105 Get get = new Get(row); 5106 if (family != null) { 5107 checkFamily(family); 5108 get.addColumn(family, qualifier); 5109 } 5110 if (filter != null) { 5111 get.setFilter(filter); 5112 } 5113 if (timeRange != null) { 5114 get.setTimeRange(timeRange.getMin(), timeRange.getMax()); 5115 } 5116 // Lock row - note that doBatchMutate will relock this row if called 5117 checkRow(row, "doCheckAndRowMutate"); 5118 RowLock rowLock = getRowLock(get.getRow(), false, null); 5119 try { 5120 if (this.getCoprocessorHost() != null) { 5121 CheckAndMutateResult result = 5122 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate); 5123 if (result != null) { 5124 return result; 5125 } 5126 } 5127 5128 // NOTE: We used to wait here until mvcc caught up: mvcc.await(); 5129 // Supposition is that now all changes are done under row locks, then when we go to read, 5130 // we'll get the latest on this row. 5131 boolean matches = false; 5132 long cellTs = 0; 5133 QueryMetrics metrics = null; 5134 try (RegionScannerImpl scanner = getScanner(new Scan(get))) { 5135 // NOTE: Please don't use HRegion.get() instead, 5136 // because it will copy cells to heap. See HBASE-26036 5137 List<ExtendedCell> result = new ArrayList<>(1); 5138 scanner.next(result); 5139 if (filter != null) { 5140 if (!result.isEmpty()) { 5141 matches = true; 5142 cellTs = result.get(0).getTimestamp(); 5143 } 5144 } else { 5145 boolean valueIsNull = 5146 comparator.getValue() == null || comparator.getValue().length == 0; 5147 if (result.isEmpty() && valueIsNull) { 5148 matches = op != CompareOperator.NOT_EQUAL; 5149 } else if (result.size() > 0 && valueIsNull) { 5150 matches = (result.get(0).getValueLength() == 0) == (op != CompareOperator.NOT_EQUAL); 5151 cellTs = result.get(0).getTimestamp(); 5152 } else if (result.size() == 1) { 5153 ExtendedCell kv = result.get(0); 5154 cellTs = kv.getTimestamp(); 5155 int compareResult = PrivateCellUtil.compareValue(kv, comparator); 5156 matches = matches(op, compareResult); 5157 } 5158 } 5159 if (checkAndMutate.isQueryMetricsEnabled()) { 5160 metrics = new QueryMetrics(scanner.getContext().getBlockSizeProgress()); 5161 } 5162 } 5163 5164 // If matches, perform the mutation or the rowMutations 5165 if (matches) { 5166 // We have acquired the row lock already. If the system clock is NOT monotonically 5167 // non-decreasing (see HBASE-14070) we should make sure that the mutation has a 5168 // larger timestamp than what was observed via Get. doBatchMutate already does this, but 5169 // there is no way to pass the cellTs. See HBASE-14054. 5170 long now = EnvironmentEdgeManager.currentTime(); 5171 long ts = Math.max(now, cellTs); // ensure write is not eclipsed 5172 byte[] byteTs = Bytes.toBytes(ts); 5173 if (mutation != null) { 5174 if (mutation instanceof Put) { 5175 updateCellTimestamps(ClientInternalHelper.getExtendedFamilyCellMap(mutation).values(), 5176 byteTs); 5177 } 5178 // And else 'delete' is not needed since it already does a second get, and sets the 5179 // timestamp from get (see prepareDeleteTimestamps). 5180 } else { 5181 for (Mutation m : rowMutations.getMutations()) { 5182 if (m instanceof Put) { 5183 updateCellTimestamps(ClientInternalHelper.getExtendedFamilyCellMap(m).values(), 5184 byteTs); 5185 } 5186 } 5187 // And else 'delete' is not needed since it already does a second get, and sets the 5188 // timestamp from get (see prepareDeleteTimestamps). 5189 } 5190 // All edits for the given row (across all column families) must happen atomically. 5191 Result r; 5192 if (mutation != null) { 5193 r = mutate(mutation, true, nonceGroup, nonce).getResult(); 5194 } else { 5195 r = mutateRow(rowMutations, nonceGroup, nonce); 5196 } 5197 this.checkAndMutateChecksPassed.increment(); 5198 return new CheckAndMutateResult(true, r).setMetrics(metrics); 5199 } 5200 this.checkAndMutateChecksFailed.increment(); 5201 return new CheckAndMutateResult(false, null).setMetrics(metrics); 5202 } finally { 5203 rowLock.release(); 5204 } 5205 } finally { 5206 closeRegionOperation(); 5207 } 5208 } 5209 5210 private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException { 5211 if ( 5212 !(mutation instanceof Put) && !(mutation instanceof Delete) 5213 && !(mutation instanceof Increment) && !(mutation instanceof Append) 5214 ) { 5215 throw new org.apache.hadoop.hbase.DoNotRetryIOException( 5216 "Action must be Put or Delete or Increment or Delete"); 5217 } 5218 } 5219 5220 private void checkRow(final Row action, final byte[] row) throws DoNotRetryIOException { 5221 if (!Bytes.equals(row, action.getRow())) { 5222 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match"); 5223 } 5224 } 5225 5226 private boolean matches(final CompareOperator op, final int compareResult) { 5227 boolean matches = false; 5228 switch (op) { 5229 case LESS: 5230 matches = compareResult < 0; 5231 break; 5232 case LESS_OR_EQUAL: 5233 matches = compareResult <= 0; 5234 break; 5235 case EQUAL: 5236 matches = compareResult == 0; 5237 break; 5238 case NOT_EQUAL: 5239 matches = compareResult != 0; 5240 break; 5241 case GREATER_OR_EQUAL: 5242 matches = compareResult >= 0; 5243 break; 5244 case GREATER: 5245 matches = compareResult > 0; 5246 break; 5247 default: 5248 throw new RuntimeException("Unknown Compare op " + op.name()); 5249 } 5250 return matches; 5251 } 5252 5253 private OperationStatus mutate(Mutation mutation) throws IOException { 5254 return mutate(mutation, false); 5255 } 5256 5257 private OperationStatus mutate(Mutation mutation, boolean atomic) throws IOException { 5258 return mutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE); 5259 } 5260 5261 private OperationStatus mutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce) 5262 throws IOException { 5263 OperationStatus[] status = 5264 this.batchMutate(new Mutation[] { mutation }, atomic, nonceGroup, nonce); 5265 if (status[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) { 5266 throw new FailedSanityCheckException(status[0].getExceptionMsg()); 5267 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) { 5268 throw new NoSuchColumnFamilyException(status[0].getExceptionMsg()); 5269 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.STORE_TOO_BUSY)) { 5270 throw new RegionTooBusyException(status[0].getExceptionMsg()); 5271 } 5272 return status[0]; 5273 } 5274 5275 /** 5276 * Complete taking the snapshot on the region. Writes the region info and adds references to the 5277 * working snapshot directory. TODO for api consistency, consider adding another version with no 5278 * {@link ForeignExceptionSnare} arg. (In the future other cancellable HRegion methods could 5279 * eventually add a {@link ForeignExceptionSnare}, or we could do something fancier). 5280 * @param desc snapshot description object 5281 * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to bail 5282 * out. This is allowed to be null and will just be ignored in that case. 5283 * @throws IOException if there is an external or internal error causing the snapshot to fail 5284 */ 5285 public void addRegionToSnapshot(SnapshotDescription desc, ForeignExceptionSnare exnSnare) 5286 throws IOException { 5287 Path rootDir = CommonFSUtils.getRootDir(conf); 5288 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf); 5289 5290 SnapshotManifest manifest = 5291 SnapshotManifest.create(conf, getFilesystem(), snapshotDir, desc, exnSnare); 5292 manifest.addRegion(this); 5293 } 5294 5295 private void updateSequenceId(final Iterable<List<ExtendedCell>> cellItr, final long sequenceId) 5296 throws IOException { 5297 for (List<ExtendedCell> cells : cellItr) { 5298 if (cells == null) { 5299 return; 5300 } 5301 for (ExtendedCell cell : cells) { 5302 cell.setSequenceId(sequenceId); 5303 } 5304 } 5305 } 5306 5307 /** 5308 * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP} 5309 * provided current timestamp. 5310 */ 5311 private static void updateCellTimestamps(final Iterable<List<ExtendedCell>> cellItr, 5312 final byte[] now) throws IOException { 5313 for (List<ExtendedCell> cells : cellItr) { 5314 if (cells == null) { 5315 continue; 5316 } 5317 // Optimization: 'foreach' loop is not used. See: 5318 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 5319 assert cells instanceof RandomAccess; 5320 int listSize = cells.size(); 5321 for (int i = 0; i < listSize; i++) { 5322 PrivateCellUtil.updateLatestStamp(cells.get(i), now); 5323 } 5324 } 5325 } 5326 5327 /** 5328 * Possibly rewrite incoming cell tags. 5329 */ 5330 private void rewriteCellTags(Map<byte[], List<ExtendedCell>> familyMap, final Mutation m) { 5331 // Check if we have any work to do and early out otherwise 5332 // Update these checks as more logic is added here 5333 if (m.getTTL() == Long.MAX_VALUE) { 5334 return; 5335 } 5336 5337 // From this point we know we have some work to do 5338 for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) { 5339 List<ExtendedCell> cells = e.getValue(); 5340 assert cells instanceof RandomAccess; 5341 int listSize = cells.size(); 5342 for (int i = 0; i < listSize; i++) { 5343 ExtendedCell cell = cells.get(i); 5344 List<Tag> newTags = TagUtil.carryForwardTags(null, cell); 5345 newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL()); 5346 // Rewrite the cell with the updated set of tags 5347 cells.set(i, PrivateCellUtil.createCell(cell, newTags)); 5348 } 5349 } 5350 } 5351 5352 /** 5353 * Check if resources to support an update. 5354 * <p/> 5355 * We throw RegionTooBusyException if above memstore limit and expect client to retry using some 5356 * kind of backoff 5357 */ 5358 private void checkResources() throws RegionTooBusyException { 5359 // If catalog region, do not impose resource constraints or block updates. 5360 if (this.getRegionInfo().isMetaRegion()) { 5361 return; 5362 } 5363 5364 MemStoreSize mss = this.memStoreSizing.getMemStoreSize(); 5365 if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) { 5366 blockedRequestsCount.increment(); 5367 requestFlush(); 5368 // Don't print current limit because it will vary too much. The message is used as a key 5369 // over in RetriesExhaustedWithDetailsException processing. 5370 final String regionName = 5371 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName(); 5372 final String serverName = this.getRegionServerServices() == null 5373 ? "unknown" 5374 : (this.getRegionServerServices().getServerName() == null 5375 ? "unknown" 5376 : this.getRegionServerServices().getServerName().toString()); 5377 RegionTooBusyException rtbe = new RegionTooBusyException("Over memstore limit=" 5378 + org.apache.hadoop.hbase.procedure2.util.StringUtils.humanSize(this.blockingMemStoreSize) 5379 + ", regionName=" + regionName + ", server=" + serverName); 5380 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe); 5381 throw rtbe; 5382 } 5383 } 5384 5385 /** 5386 * @throws IOException Throws exception if region is in read-only mode. 5387 */ 5388 private void checkReadOnly() throws IOException { 5389 if (isReadOnly()) { 5390 throw new DoNotRetryIOException("region is read only"); 5391 } 5392 } 5393 5394 private void checkReadsEnabled() throws IOException { 5395 if (!this.writestate.readsEnabled) { 5396 throw new IOException(getRegionInfo().getEncodedName() 5397 + ": The region's reads are disabled. Cannot serve the request"); 5398 } 5399 } 5400 5401 public void setReadsEnabled(boolean readsEnabled) { 5402 if (readsEnabled && !this.writestate.readsEnabled) { 5403 LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName()); 5404 } 5405 this.writestate.setReadsEnabled(readsEnabled); 5406 } 5407 5408 /** 5409 * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be 5410 * set; when set we will run operations that make sense in the increment/append 5411 * scenario but that do not make sense otherwise. 5412 */ 5413 private void applyToMemStore(HStore store, List<ExtendedCell> cells, boolean delta, 5414 MemStoreSizing memstoreAccounting) { 5415 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! 5416 boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1; 5417 if (upsert) { 5418 store.upsert(cells, getSmallestReadPoint(), memstoreAccounting); 5419 } else { 5420 store.add(cells, memstoreAccounting); 5421 } 5422 } 5423 5424 private void checkFamilies(Collection<byte[]> families, Durability durability) 5425 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException { 5426 for (byte[] family : families) { 5427 checkFamily(family, durability); 5428 } 5429 } 5430 5431 private void checkFamily(final byte[] family, Durability durability) 5432 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException { 5433 checkFamily(family); 5434 if ( 5435 durability.equals(Durability.SKIP_WAL) 5436 && htableDescriptor.getColumnFamily(family).getScope() != HConstants.REPLICATION_SCOPE_LOCAL 5437 ) { 5438 throw new InvalidMutationDurabilityException( 5439 "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family) 5440 + " need replication"); 5441 } 5442 } 5443 5444 private void checkFamily(final byte[] family) throws NoSuchColumnFamilyException { 5445 if (!this.htableDescriptor.hasColumnFamily(family)) { 5446 throw new NoSuchColumnFamilyException("Column family " + Bytes.toString(family) 5447 + " does not exist in region " + this + " in table " + this.htableDescriptor); 5448 } 5449 } 5450 5451 /** 5452 * Check the collection of families for valid timestamps 5453 * @param now current timestamp 5454 */ 5455 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now) 5456 throws FailedSanityCheckException { 5457 if (timestampSlop == HConstants.LATEST_TIMESTAMP) { 5458 return; 5459 } 5460 long maxTs = now + timestampSlop; 5461 for (List<Cell> kvs : familyMap.values()) { 5462 // Optimization: 'foreach' loop is not used. See: 5463 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 5464 assert kvs instanceof RandomAccess; 5465 int listSize = kvs.size(); 5466 for (int i = 0; i < listSize; i++) { 5467 Cell cell = kvs.get(i); 5468 // see if the user-side TS is out of range. latest = server-side 5469 long ts = cell.getTimestamp(); 5470 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) { 5471 throw new FailedSanityCheckException( 5472 "Timestamp for KV out of range " + cell + " (too.new=" + timestampSlop + ")"); 5473 } 5474 } 5475 } 5476 } 5477 5478 /* 5479 * @return True if size is over the flush threshold 5480 */ 5481 private boolean isFlushSize(MemStoreSize size) { 5482 return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize(); 5483 } 5484 5485 private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException { 5486 for (Path file : files) { 5487 if (!fs.delete(file, false)) { 5488 LOG.error("Failed delete of {}", file); 5489 } else { 5490 LOG.debug("Deleted recovered.edits file={}", file); 5491 } 5492 } 5493 } 5494 5495 /** 5496 * Read the edits put under this region by wal splitting process. Put the recovered edits back up 5497 * into this region. 5498 * <p> 5499 * We can ignore any wal message that has a sequence ID that's equal to or lower than minSeqId. 5500 * (Because we know such messages are already reflected in the HFiles.) 5501 * <p> 5502 * While this is running we are putting pressure on memory yet we are outside of our usual 5503 * accounting because we are not yet an onlined region (this stuff is being run as part of Region 5504 * initialization). This means that if we're up against global memory limits, we'll not be flagged 5505 * to flush because we are not online. We can't be flushed by usual mechanisms anyways; we're not 5506 * yet online so our relative sequenceids are not yet aligned with WAL sequenceids -- not till we 5507 * come up online, post processing of split edits. 5508 * <p> 5509 * But to help relieve memory pressure, at least manage our own heap size flushing if are in 5510 * excess of per-region limits. Flushing, though, we have to be careful and avoid using the 5511 * regionserver/wal sequenceid. Its running on a different line to whats going on in here in this 5512 * region context so if we crashed replaying these edits, but in the midst had a flush that used 5513 * the regionserver wal with a sequenceid in excess of whats going on in here in this region and 5514 * with its split editlogs, then we could miss edits the next time we go to recover. So, we have 5515 * to flush inline, using seqids that make sense in a this single region context only -- until we 5516 * online. 5517 * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of the maxSeqId 5518 * for the store to be applied, else its skipped. 5519 * @return the sequence id of the last edit added to this region out of the recovered edits log or 5520 * <code>minSeqId</code> if nothing added from editlogs. 5521 */ 5522 long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores, 5523 final CancelableProgressable reporter, final MonitoredTask status) throws IOException { 5524 long minSeqIdForTheRegion = -1; 5525 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) { 5526 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) { 5527 minSeqIdForTheRegion = maxSeqIdInStore; 5528 } 5529 } 5530 long seqId = minSeqIdForTheRegion; 5531 String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR); 5532 if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) { 5533 FileSystem walFS = getWalFileSystem(); 5534 FileSystem rootFS = getFilesystem(); 5535 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(), 5536 getRegionInfo().getEncodedName()); 5537 Path regionWALDir = getWALRegionDir(); 5538 Path regionDir = 5539 FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo()); 5540 5541 // We made a mistake in HBASE-20734 so we need to do this dirty hack... 5542 NavigableSet<Path> filesUnderWrongRegionWALDir = 5543 WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir); 5544 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, 5545 filesUnderWrongRegionWALDir, reporter, regionDir)); 5546 // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear 5547 // under the root dir even if walDir is set. 5548 NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet(); 5549 if (!regionWALDir.equals(regionDir)) { 5550 filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir); 5551 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS, 5552 filesUnderRootDir, reporter, regionDir)); 5553 } 5554 5555 NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir); 5556 seqId = Math.max(seqId, 5557 replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir)); 5558 if (seqId > minSeqIdForTheRegion) { 5559 // Then we added some edits to memory. Flush and cleanup split edit files. 5560 internalFlushcache(null, seqId, stores.values(), status, false, 5561 FlushLifeCycleTracker.DUMMY); 5562 } 5563 // Now delete the content of recovered edits. We're done w/ them. 5564 if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) { 5565 // For debugging data loss issues! 5566 // If this flag is set, make use of the hfile archiving by making recovered.edits a fake 5567 // column family. Have to fake out file type too by casting our recovered.edits as 5568 // storefiles 5569 String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName(); 5570 StoreContext storeContext = 5571 StoreContext.getBuilder().withRegionFileSystem(getRegionFileSystem()).build(); 5572 StoreFileTracker sft = StoreFileTrackerFactory.create(this.conf, true, storeContext); 5573 Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size()); 5574 for (Path file : files) { 5575 fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true, sft)); 5576 } 5577 getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles); 5578 } else { 5579 deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir)); 5580 deleteRecoveredEdits(rootFS, filesUnderRootDir); 5581 } 5582 } else { 5583 Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr); 5584 FileSystem fs = recoveredEditsDir.getFileSystem(conf); 5585 FileStatus[] files = fs.listStatus(recoveredEditsDir); 5586 LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length, 5587 recoveredEditsDir); 5588 if (files != null) { 5589 for (FileStatus file : files) { 5590 // it is safe to trust the zero-length in this case because we've been through rename and 5591 // lease recovery in the above. 5592 if (isZeroLengthThenDelete(fs, file, file.getPath())) { 5593 continue; 5594 } 5595 seqId = 5596 Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs)); 5597 } 5598 } 5599 if (seqId > minSeqIdForTheRegion) { 5600 // Then we added some edits to memory. Flush and cleanup split edit files. 5601 internalFlushcache(null, seqId, stores.values(), status, false, 5602 FlushLifeCycleTracker.DUMMY); 5603 } 5604 deleteRecoveredEdits(fs, 5605 Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList())); 5606 } 5607 5608 return seqId; 5609 } 5610 5611 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs, 5612 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir) 5613 throws IOException { 5614 long seqid = minSeqIdForTheRegion; 5615 if (LOG.isDebugEnabled()) { 5616 LOG.debug("Found " + (files == null ? 0 : files.size()) + " recovered edits file(s) under " 5617 + regionDir); 5618 } 5619 5620 if (files == null || files.isEmpty()) { 5621 return minSeqIdForTheRegion; 5622 } 5623 5624 for (Path edits : files) { 5625 if (edits == null || !fs.exists(edits)) { 5626 LOG.warn("Null or non-existent edits file: " + edits); 5627 continue; 5628 } 5629 if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) { 5630 continue; 5631 } 5632 5633 long maxSeqId; 5634 String fileName = edits.getName(); 5635 maxSeqId = Math.abs(Long.parseLong(fileName)); 5636 if (maxSeqId <= minSeqIdForTheRegion) { 5637 if (LOG.isDebugEnabled()) { 5638 String msg = "Maximum sequenceid for this wal is " + maxSeqId 5639 + " and minimum sequenceid for the region " + this + " is " + minSeqIdForTheRegion 5640 + ", skipped the whole file, path=" + edits; 5641 LOG.debug(msg); 5642 } 5643 continue; 5644 } 5645 5646 try { 5647 // replay the edits. Replay can return -1 if everything is skipped, only update 5648 // if seqId is greater 5649 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs)); 5650 } catch (IOException e) { 5651 handleException(fs, edits, e); 5652 } 5653 } 5654 return seqid; 5655 } 5656 5657 private void handleException(FileSystem fs, Path edits, IOException e) throws IOException { 5658 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS, 5659 conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS)); 5660 if (conf.get("hbase.skip.errors") != null) { 5661 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use " 5662 + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead."); 5663 } 5664 if (skipErrors) { 5665 Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits); 5666 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed " 5667 + edits + " as " + p, e); 5668 } else { 5669 throw e; 5670 } 5671 } 5672 5673 /** 5674 * @param edits File of recovered edits. 5675 * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger 5676 * than this to be replayed for each store. 5677 * @return the sequence id of the last edit added to this region out of the recovered edits log or 5678 * <code>minSeqId</code> if nothing added from editlogs. 5679 */ 5680 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores, 5681 final CancelableProgressable reporter, FileSystem fs) throws IOException { 5682 String msg = "Replaying edits from " + edits; 5683 LOG.info(msg); 5684 MonitoredTask status = TaskMonitor.get().createStatus(msg); 5685 5686 status.setStatus("Opening recovered edits"); 5687 try (WALStreamReader reader = WALFactory.createStreamReader(fs, edits, conf)) { 5688 long currentEditSeqId = -1; 5689 long currentReplaySeqId = -1; 5690 long firstSeqIdInLog = -1; 5691 long skippedEdits = 0; 5692 long editsCount = 0; 5693 long intervalEdits = 0; 5694 WAL.Entry entry; 5695 HStore store = null; 5696 boolean reported_once = false; 5697 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager(); 5698 5699 try { 5700 // How many edits seen before we check elapsed time 5701 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000); 5702 // How often to send a progress report (default 1/2 master timeout) 5703 int period = this.conf.getInt("hbase.hstore.report.period", 300000); 5704 long lastReport = EnvironmentEdgeManager.currentTime(); 5705 5706 if (coprocessorHost != null) { 5707 coprocessorHost.preReplayWALs(this.getRegionInfo(), edits); 5708 } 5709 5710 while ((entry = reader.next()) != null) { 5711 WALKey key = entry.getKey(); 5712 WALEdit val = entry.getEdit(); 5713 5714 if (ng != null) { // some test, or nonces disabled 5715 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime()); 5716 } 5717 5718 if (reporter != null) { 5719 intervalEdits += val.size(); 5720 if (intervalEdits >= interval) { 5721 // Number of edits interval reached 5722 intervalEdits = 0; 5723 long cur = EnvironmentEdgeManager.currentTime(); 5724 if (lastReport + period <= cur) { 5725 status.setStatus( 5726 "Replaying edits..." + " skipped=" + skippedEdits + " edits=" + editsCount); 5727 // Timeout reached 5728 if (!reporter.progress()) { 5729 msg = "Progressable reporter failed, stopping replay for region " + this; 5730 LOG.warn(msg); 5731 status.abort(msg); 5732 throw new IOException(msg); 5733 } 5734 reported_once = true; 5735 lastReport = cur; 5736 } 5737 } 5738 } 5739 5740 if (firstSeqIdInLog == -1) { 5741 firstSeqIdInLog = key.getSequenceId(); 5742 } 5743 if (currentEditSeqId > key.getSequenceId()) { 5744 // when this condition is true, it means we have a serious defect because we need to 5745 // maintain increasing SeqId for WAL edits per region 5746 LOG.error(getRegionInfo().getEncodedName() + " : " + "Found decreasing SeqId. PreId=" 5747 + currentEditSeqId + " key=" + key + "; edit=" + val); 5748 } else { 5749 currentEditSeqId = key.getSequenceId(); 5750 } 5751 currentReplaySeqId = 5752 (key.getOrigLogSeqNum() > 0) ? key.getOrigLogSeqNum() : currentEditSeqId; 5753 5754 boolean checkRowWithinBoundary = false; 5755 // Check this edit is for this region. 5756 if ( 5757 !Bytes.equals(key.getEncodedRegionName(), this.getRegionInfo().getEncodedNameAsBytes()) 5758 ) { 5759 checkRowWithinBoundary = true; 5760 } 5761 5762 boolean flush = false; 5763 MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing(); 5764 for (Cell c : val.getCells()) { 5765 assert c instanceof ExtendedCell; 5766 ExtendedCell cell = (ExtendedCell) c; 5767 // Check this edit is for me. Also, guard against writing the special 5768 // METACOLUMN info such as HBASE::CACHEFLUSH entries 5769 if (WALEdit.isMetaEditFamily(cell)) { 5770 // if region names don't match, skipp replaying compaction marker 5771 if (!checkRowWithinBoundary) { 5772 // this is a special edit, we should handle it 5773 CompactionDescriptor compaction = WALEdit.getCompaction(cell); 5774 if (compaction != null) { 5775 // replay the compaction 5776 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE); 5777 } 5778 } 5779 skippedEdits++; 5780 continue; 5781 } 5782 // Figure which store the edit is meant for. 5783 if ( 5784 store == null 5785 || !CellUtil.matchingFamily(cell, store.getColumnFamilyDescriptor().getName()) 5786 ) { 5787 store = getStore(cell); 5788 } 5789 if (store == null) { 5790 // This should never happen. Perhaps schema was changed between 5791 // crash and redeploy? 5792 LOG.warn("No family for cell {} in region {}", cell, this); 5793 skippedEdits++; 5794 continue; 5795 } 5796 if ( 5797 checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(), cell.getRowArray(), 5798 cell.getRowOffset(), cell.getRowLength()) 5799 ) { 5800 LOG.warn("Row of {} is not within region boundary for region {}", cell, this); 5801 skippedEdits++; 5802 continue; 5803 } 5804 // Now, figure if we should skip this edit. 5805 if ( 5806 key.getSequenceId() 5807 <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor().getName()) 5808 ) { 5809 skippedEdits++; 5810 continue; 5811 } 5812 PrivateCellUtil.setSequenceId(cell, currentReplaySeqId); 5813 5814 restoreEdit(store, cell, memStoreSizing); 5815 editsCount++; 5816 } 5817 MemStoreSize mss = memStoreSizing.getMemStoreSize(); 5818 incMemStoreSize(mss); 5819 flush = isFlushSize(this.memStoreSizing.getMemStoreSize()); 5820 if (flush) { 5821 internalFlushcache(null, currentEditSeqId, stores.values(), status, false, 5822 FlushLifeCycleTracker.DUMMY); 5823 } 5824 } 5825 5826 if (coprocessorHost != null) { 5827 coprocessorHost.postReplayWALs(this.getRegionInfo(), edits); 5828 } 5829 } catch (EOFException eof) { 5830 if (!conf.getBoolean(RECOVERED_EDITS_IGNORE_EOF, false)) { 5831 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits); 5832 msg = "EnLongAddered EOF. Most likely due to Master failure during " 5833 + "wal splitting, so we have this data in another edit. Continuing, but renaming " 5834 + edits + " as " + p + " for region " + this; 5835 LOG.warn(msg, eof); 5836 status.abort(msg); 5837 } else { 5838 LOG.warn("EOF while replaying recover edits and config '{}' is true so " 5839 + "we will ignore it and continue", RECOVERED_EDITS_IGNORE_EOF, eof); 5840 } 5841 } catch (IOException ioe) { 5842 // If the IOE resulted from bad file format, 5843 // then this problem is idempotent and retrying won't help 5844 if (ioe.getCause() instanceof ParseException) { 5845 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits); 5846 msg = 5847 "File corruption enLongAddered! " + "Continuing, but renaming " + edits + " as " + p; 5848 LOG.warn(msg, ioe); 5849 status.setStatus(msg); 5850 } else { 5851 status.abort(StringUtils.stringifyException(ioe)); 5852 // other IO errors may be transient (bad network connection, 5853 // checksum exception on one datanode, etc). throw & retry 5854 throw ioe; 5855 } 5856 } 5857 if (reporter != null && !reported_once) { 5858 reporter.progress(); 5859 } 5860 msg = "Applied " + editsCount + ", skipped " + skippedEdits + ", firstSequenceIdInLog=" 5861 + firstSeqIdInLog + ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits; 5862 status.markComplete(msg); 5863 LOG.debug(msg); 5864 return currentEditSeqId; 5865 } finally { 5866 status.cleanup(); 5867 } 5868 } 5869 5870 /** 5871 * Call to complete a compaction. Its for the case where we find in the WAL a compaction that was 5872 * not finished. We could find one recovering a WAL after a regionserver crash. See HBASE-2331. 5873 */ 5874 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles, 5875 boolean removeFiles, long replaySeqId) throws IOException { 5876 try { 5877 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(), 5878 "Compaction marker from WAL ", compaction); 5879 } catch (WrongRegionException wre) { 5880 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 5881 // skip the compaction marker since it is not for this region 5882 return; 5883 } 5884 throw wre; 5885 } 5886 5887 synchronized (writestate) { 5888 if (replaySeqId < lastReplayedOpenRegionSeqId) { 5889 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying compaction event :" 5890 + TextFormat.shortDebugString(compaction) + " because its sequence id " + replaySeqId 5891 + " is smaller than this regions " + "lastReplayedOpenRegionSeqId of " 5892 + lastReplayedOpenRegionSeqId); 5893 return; 5894 } 5895 if (replaySeqId < lastReplayedCompactionSeqId) { 5896 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying compaction event :" 5897 + TextFormat.shortDebugString(compaction) + " because its sequence id " + replaySeqId 5898 + " is smaller than this regions " + "lastReplayedCompactionSeqId of " 5899 + lastReplayedCompactionSeqId); 5900 return; 5901 } else { 5902 lastReplayedCompactionSeqId = replaySeqId; 5903 } 5904 5905 if (LOG.isDebugEnabled()) { 5906 LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying compaction marker " 5907 + TextFormat.shortDebugString(compaction) + " with seqId=" + replaySeqId 5908 + " and lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId); 5909 } 5910 5911 startRegionOperation(Operation.REPLAY_EVENT); 5912 try { 5913 HStore store = this.getStore(compaction.getFamilyName().toByteArray()); 5914 if (store == null) { 5915 LOG.warn(getRegionInfo().getEncodedName() + " : " 5916 + "Found Compaction WAL edit for deleted family:" 5917 + Bytes.toString(compaction.getFamilyName().toByteArray())); 5918 return; 5919 } 5920 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles); 5921 logRegionFiles(); 5922 } catch (FileNotFoundException ex) { 5923 LOG.warn(getRegionInfo().getEncodedName() + " : " 5924 + "At least one of the store files in compaction: " 5925 + TextFormat.shortDebugString(compaction) 5926 + " doesn't exist any more. Skip loading the file(s)", ex); 5927 } finally { 5928 closeRegionOperation(Operation.REPLAY_EVENT); 5929 } 5930 } 5931 } 5932 5933 /** 5934 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 5935 * replica implementation. 5936 */ 5937 @Deprecated 5938 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException { 5939 checkTargetRegion(flush.getEncodedRegionName().toByteArray(), "Flush marker from WAL ", flush); 5940 5941 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 5942 return; // if primary nothing to do 5943 } 5944 5945 if (LOG.isDebugEnabled()) { 5946 LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying flush marker " 5947 + TextFormat.shortDebugString(flush)); 5948 } 5949 5950 startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close 5951 try { 5952 FlushAction action = flush.getAction(); 5953 switch (action) { 5954 case START_FLUSH: 5955 replayWALFlushStartMarker(flush); 5956 break; 5957 case COMMIT_FLUSH: 5958 replayWALFlushCommitMarker(flush); 5959 break; 5960 case ABORT_FLUSH: 5961 replayWALFlushAbortMarker(flush); 5962 break; 5963 case CANNOT_FLUSH: 5964 replayWALFlushCannotFlushMarker(flush, replaySeqId); 5965 break; 5966 default: 5967 LOG.warn(getRegionInfo().getEncodedName() + " : " 5968 + "Received a flush event with unknown action, ignoring. " 5969 + TextFormat.shortDebugString(flush)); 5970 break; 5971 } 5972 5973 logRegionFiles(); 5974 } finally { 5975 closeRegionOperation(Operation.REPLAY_EVENT); 5976 } 5977 } 5978 5979 private Collection<HStore> getStoresToFlush(FlushDescriptor flushDesc) { 5980 List<HStore> storesToFlush = new ArrayList<>(); 5981 for (StoreFlushDescriptor storeFlush : flushDesc.getStoreFlushesList()) { 5982 byte[] family = storeFlush.getFamilyName().toByteArray(); 5983 HStore store = getStore(family); 5984 if (store == null) { 5985 LOG.warn(getRegionInfo().getEncodedName() + " : " 5986 + "Received a flush start marker from primary, but the family is not found. Ignoring" 5987 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush)); 5988 continue; 5989 } 5990 storesToFlush.add(store); 5991 } 5992 return storesToFlush; 5993 } 5994 5995 /** 5996 * Replay the flush marker from primary region by creating a corresponding snapshot of the store 5997 * memstores, only if the memstores do not have a higher seqId from an earlier wal edit (because 5998 * the events may be coming out of order). 5999 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 6000 * replica implementation. 6001 */ 6002 @Deprecated 6003 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException { 6004 long flushSeqId = flush.getFlushSequenceNumber(); 6005 6006 Collection<HStore> storesToFlush = getStoresToFlush(flush); 6007 6008 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this); 6009 6010 // we will use writestate as a coarse-grain lock for all the replay events 6011 // (flush, compaction, region open etc) 6012 synchronized (writestate) { 6013 try { 6014 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) { 6015 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :" 6016 + TextFormat.shortDebugString(flush) 6017 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 6018 + " of " + lastReplayedOpenRegionSeqId); 6019 return null; 6020 } 6021 if (numMutationsWithoutWAL.sum() > 0) { 6022 numMutationsWithoutWAL.reset(); 6023 dataInMemoryWithoutWAL.reset(); 6024 } 6025 6026 if (!writestate.flushing) { 6027 // we do not have an active snapshot and corresponding this.prepareResult. This means 6028 // we can just snapshot our memstores and continue as normal. 6029 6030 // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal 6031 PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId, 6032 storesToFlush, status, false, FlushLifeCycleTracker.DUMMY); 6033 if (prepareResult.result == null) { 6034 // save the PrepareFlushResult so that we can use it later from commit flush 6035 this.writestate.flushing = true; 6036 this.prepareFlushResult = prepareResult; 6037 status.markComplete("Flush prepare successful"); 6038 if (LOG.isDebugEnabled()) { 6039 LOG.debug(getRegionInfo().getEncodedName() + " : " + " Prepared flush with seqId:" 6040 + flush.getFlushSequenceNumber()); 6041 } 6042 } else { 6043 // special case empty memstore. We will still save the flush result in this case, since 6044 // our memstore ie empty, but the primary is still flushing 6045 if ( 6046 prepareResult.getResult().getResult() 6047 == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY 6048 ) { 6049 this.writestate.flushing = true; 6050 this.prepareFlushResult = prepareResult; 6051 if (LOG.isDebugEnabled()) { 6052 LOG.debug(getRegionInfo().getEncodedName() + " : " 6053 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber()); 6054 } 6055 } 6056 status.abort("Flush prepare failed with " + prepareResult.result); 6057 // nothing much to do. prepare flush failed because of some reason. 6058 } 6059 return prepareResult; 6060 } else { 6061 // we already have an active snapshot. 6062 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) { 6063 // They define the same flush. Log and continue. 6064 LOG.warn(getRegionInfo().getEncodedName() + " : " 6065 + "Received a flush prepare marker with the same seqId: " 6066 + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 6067 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 6068 // ignore 6069 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) { 6070 // We received a flush with a smaller seqNum than what we have prepared. We can only 6071 // ignore this prepare flush request. 6072 LOG.warn(getRegionInfo().getEncodedName() + " : " 6073 + "Received a flush prepare marker with a smaller seqId: " 6074 + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 6075 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 6076 // ignore 6077 } else { 6078 // We received a flush with a larger seqNum than what we have prepared 6079 LOG.warn(getRegionInfo().getEncodedName() + " : " 6080 + "Received a flush prepare marker with a larger seqId: " 6081 + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: " 6082 + prepareFlushResult.flushOpSeqId + ". Ignoring"); 6083 // We do not have multiple active snapshots in the memstore or a way to merge current 6084 // memstore snapshot with the contents and resnapshot for now. We cannot take 6085 // another snapshot and drop the previous one because that will cause temporary 6086 // data loss in the secondary. So we ignore this for now, deferring the resolution 6087 // to happen when we see the corresponding flush commit marker. If we have a memstore 6088 // snapshot with x, and later received another prepare snapshot with y (where x < y), 6089 // when we see flush commit for y, we will drop snapshot for x, and can also drop all 6090 // the memstore edits if everything in memstore is < y. This is the usual case for 6091 // RS crash + recovery where we might see consequtive prepare flush wal markers. 6092 // Otherwise, this will cause more memory to be used in secondary replica until a 6093 // further prapare + commit flush is seen and replayed. 6094 } 6095 } 6096 } finally { 6097 status.cleanup(); 6098 writestate.notifyAll(); 6099 } 6100 } 6101 return null; 6102 } 6103 6104 /** 6105 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 6106 * replica implementation. 6107 */ 6108 @Deprecated 6109 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY", 6110 justification = "Intentional; post memstore flush") 6111 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException { 6112 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this); 6113 6114 // check whether we have the memstore snapshot with the corresponding seqId. Replay to 6115 // secondary region replicas are in order, except for when the region moves or then the 6116 // region server crashes. In those cases, we may receive replay requests out of order from 6117 // the original seqIds. 6118 synchronized (writestate) { 6119 try { 6120 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) { 6121 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :" 6122 + TextFormat.shortDebugString(flush) 6123 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 6124 + " of " + lastReplayedOpenRegionSeqId); 6125 return; 6126 } 6127 6128 if (writestate.flushing) { 6129 PrepareFlushResult prepareFlushResult = this.prepareFlushResult; 6130 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) { 6131 if (LOG.isDebugEnabled()) { 6132 LOG.debug(getRegionInfo().getEncodedName() + " : " 6133 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber() 6134 + " and a previous prepared snapshot was found"); 6135 } 6136 // This is the regular case where we received commit flush after prepare flush 6137 // corresponding to the same seqId. 6138 replayFlushInStores(flush, prepareFlushResult, true); 6139 6140 // Set down the memstore size by amount of flush. 6141 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize()); 6142 this.prepareFlushResult = null; 6143 writestate.flushing = false; 6144 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) { 6145 // This should not happen normally. However, lets be safe and guard against these cases 6146 // we received a flush commit with a smaller seqId than what we have prepared 6147 // we will pick the flush file up from this commit (if we have not seen it), but we 6148 // will not drop the memstore 6149 LOG.warn(getRegionInfo().getEncodedName() + " : " 6150 + "Received a flush commit marker with smaller seqId: " 6151 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " 6152 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping" 6153 + " prepared memstore snapshot"); 6154 replayFlushInStores(flush, prepareFlushResult, false); 6155 6156 // snapshot is not dropped, so memstore sizes should not be decremented 6157 // we still have the prepared snapshot, flushing should still be true 6158 } else { 6159 // This should not happen normally. However, lets be safe and guard against these cases 6160 // we received a flush commit with a larger seqId than what we have prepared 6161 // we will pick the flush file for this. We will also obtain the updates lock and 6162 // look for contents of the memstore to see whether we have edits after this seqId. 6163 // If not, we will drop all the memstore edits and the snapshot as well. 6164 LOG.warn(getRegionInfo().getEncodedName() + " : " 6165 + "Received a flush commit marker with larger seqId: " 6166 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " 6167 + prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared" 6168 + " memstore snapshot"); 6169 6170 replayFlushInStores(flush, prepareFlushResult, true); 6171 6172 // Set down the memstore size by amount of flush. 6173 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize()); 6174 6175 // Inspect the memstore contents to see whether the memstore contains only edits 6176 // with seqId smaller than the flush seqId. If so, we can discard those edits. 6177 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); 6178 6179 this.prepareFlushResult = null; 6180 writestate.flushing = false; 6181 } 6182 // If we were waiting for observing a flush or region opening event for not showing 6183 // partial data after a secondary region crash, we can allow reads now. We can only make 6184 // sure that we are not showing partial data (for example skipping some previous edits) 6185 // until we observe a full flush start and flush commit. So if we were not able to find 6186 // a previous flush we will not enable reads now. 6187 this.setReadsEnabled(true); 6188 } else { 6189 LOG.warn( 6190 getRegionInfo().getEncodedName() + " : " + "Received a flush commit marker with seqId:" 6191 + flush.getFlushSequenceNumber() + ", but no previous prepared snapshot was found"); 6192 // There is no corresponding prepare snapshot from before. 6193 // We will pick up the new flushed file 6194 replayFlushInStores(flush, null, false); 6195 6196 // Inspect the memstore contents to see whether the memstore contains only edits 6197 // with seqId smaller than the flush seqId. If so, we can discard those edits. 6198 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); 6199 } 6200 6201 status.markComplete("Flush commit successful"); 6202 6203 // Update the last flushed sequence id for region. 6204 this.maxFlushedSeqId = flush.getFlushSequenceNumber(); 6205 6206 // advance the mvcc read point so that the new flushed file is visible. 6207 mvcc.advanceTo(flush.getFlushSequenceNumber()); 6208 6209 } catch (FileNotFoundException ex) { 6210 LOG.warn(getRegionInfo().getEncodedName() + " : " 6211 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush) 6212 + " doesn't exist any more. Skip loading the file(s)", ex); 6213 } finally { 6214 status.cleanup(); 6215 writestate.notifyAll(); 6216 } 6217 } 6218 6219 // C. Finally notify anyone waiting on memstore to clear: 6220 // e.g. checkResources(). 6221 synchronized (this) { 6222 notifyAll(); // FindBugs NN_NAKED_NOTIFY 6223 } 6224 } 6225 6226 /** 6227 * Replays the given flush descriptor by opening the flush files in stores and dropping the 6228 * memstore snapshots if requested. 6229 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 6230 * replica implementation. 6231 */ 6232 @Deprecated 6233 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult, 6234 boolean dropMemstoreSnapshot) throws IOException { 6235 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) { 6236 byte[] family = storeFlush.getFamilyName().toByteArray(); 6237 HStore store = getStore(family); 6238 if (store == null) { 6239 LOG.warn(getRegionInfo().getEncodedName() + " : " 6240 + "Received a flush commit marker from primary, but the family is not found." 6241 + "Ignoring StoreFlushDescriptor:" + storeFlush); 6242 continue; 6243 } 6244 List<String> flushFiles = storeFlush.getFlushOutputList(); 6245 StoreFlushContext ctx = null; 6246 long startTime = EnvironmentEdgeManager.currentTime(); 6247 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) { 6248 ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY); 6249 } else { 6250 ctx = prepareFlushResult.storeFlushCtxs.get(family); 6251 startTime = prepareFlushResult.startTime; 6252 } 6253 6254 if (ctx == null) { 6255 LOG.warn(getRegionInfo().getEncodedName() + " : " 6256 + "Unexpected: flush commit marker received from store " + Bytes.toString(family) 6257 + " but no associated flush context. Ignoring"); 6258 continue; 6259 } 6260 6261 ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush 6262 6263 // Record latest flush time 6264 this.lastStoreFlushTimeMap.put(store, startTime); 6265 } 6266 } 6267 6268 private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException { 6269 Path regionDir = fs.getRegionDir(); 6270 long maxSeqId = -1; 6271 for (HStore store : stores) { 6272 String familyName = store.getColumnFamilyName(); 6273 FileStatus[] files = 6274 WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName); 6275 if (files != null && files.length != 0) { 6276 for (FileStatus file : files) { 6277 Path filePath = file.getPath(); 6278 // If file length is zero then delete it 6279 if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) { 6280 continue; 6281 } 6282 try { 6283 HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath()); 6284 maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID()); 6285 } catch (IOException e) { 6286 handleException(fs.getFileSystem(), filePath, e); 6287 continue; 6288 } 6289 } 6290 if (this.rsServices != null && store.needsCompaction()) { 6291 this.rsServices.getCompactionRequestor().requestCompaction(this, store, 6292 "load recovered hfiles request compaction", Store.PRIORITY_USER + 1, 6293 CompactionLifeCycleTracker.DUMMY, null); 6294 } 6295 } 6296 } 6297 return maxSeqId; 6298 } 6299 6300 /** 6301 * Be careful, this method will drop all data in the memstore of this region. Currently, this 6302 * method is used to drop memstore to prevent memory leak when replaying recovered.edits while 6303 * opening region. 6304 */ 6305 private MemStoreSize dropMemStoreContents() throws IOException { 6306 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing(); 6307 this.updatesLock.writeLock().lock(); 6308 try { 6309 for (HStore s : stores.values()) { 6310 MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM); 6311 LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region " 6312 + this.getRegionInfo().getRegionNameAsString() + " , dropped memstoresize: [" 6313 + memStoreSize + " }"); 6314 totalFreedSize.incMemStoreSize(memStoreSize); 6315 } 6316 return totalFreedSize.getMemStoreSize(); 6317 } finally { 6318 this.updatesLock.writeLock().unlock(); 6319 } 6320 } 6321 6322 /** 6323 * Drops the memstore contents after replaying a flush descriptor or region open event replay if 6324 * the memstore edits have seqNums smaller than the given seq id 6325 */ 6326 private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException { 6327 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing(); 6328 this.updatesLock.writeLock().lock(); 6329 try { 6330 6331 long currentSeqId = mvcc.getReadPoint(); 6332 if (seqId >= currentSeqId) { 6333 // then we can drop the memstore contents since everything is below this seqId 6334 LOG.info(getRegionInfo().getEncodedName() + " : " 6335 + "Dropping memstore contents as well since replayed flush seqId: " + seqId 6336 + " is greater than current seqId:" + currentSeqId); 6337 6338 // Prepare flush (take a snapshot) and then abort (drop the snapshot) 6339 if (store == null) { 6340 for (HStore s : stores.values()) { 6341 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId)); 6342 } 6343 } else { 6344 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId)); 6345 } 6346 } else { 6347 LOG.info(getRegionInfo().getEncodedName() + " : " 6348 + "Not dropping memstore contents since replayed flush seqId: " + seqId 6349 + " is smaller than current seqId:" + currentSeqId); 6350 } 6351 } finally { 6352 this.updatesLock.writeLock().unlock(); 6353 } 6354 return totalFreedSize.getMemStoreSize(); 6355 } 6356 6357 private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId) 6358 throws IOException { 6359 MemStoreSize flushableSize = s.getFlushableSize(); 6360 this.decrMemStoreSize(flushableSize); 6361 StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY); 6362 ctx.prepare(); 6363 ctx.abort(); 6364 return flushableSize; 6365 } 6366 6367 private void replayWALFlushAbortMarker(FlushDescriptor flush) { 6368 // nothing to do for now. A flush abort will cause a RS abort which means that the region 6369 // will be opened somewhere else later. We will see the region open event soon, and replaying 6370 // that will drop the snapshot 6371 } 6372 6373 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) { 6374 synchronized (writestate) { 6375 if (this.lastReplayedOpenRegionSeqId > replaySeqId) { 6376 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :" 6377 + TextFormat.shortDebugString(flush) + " because its sequence id " + replaySeqId 6378 + " is smaller than this regions " + "lastReplayedOpenRegionSeqId of " 6379 + lastReplayedOpenRegionSeqId); 6380 return; 6381 } 6382 6383 // If we were waiting for observing a flush or region opening event for not showing partial 6384 // data after a secondary region crash, we can allow reads now. This event means that the 6385 // primary was not able to flush because memstore is empty when we requested flush. By the 6386 // time we observe this, we are guaranteed to have up to date seqId with our previous 6387 // assignment. 6388 this.setReadsEnabled(true); 6389 } 6390 } 6391 6392 PrepareFlushResult getPrepareFlushResult() { 6393 return prepareFlushResult; 6394 } 6395 6396 /** 6397 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 6398 * replica implementation. 6399 */ 6400 @Deprecated 6401 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY", 6402 justification = "Intentional; cleared the memstore") 6403 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException { 6404 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(), 6405 "RegionEvent marker from WAL ", regionEvent); 6406 6407 startRegionOperation(Operation.REPLAY_EVENT); 6408 try { 6409 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6410 return; // if primary nothing to do 6411 } 6412 6413 if (regionEvent.getEventType() == EventType.REGION_CLOSE) { 6414 // nothing to do on REGION_CLOSE for now. 6415 return; 6416 } 6417 if (regionEvent.getEventType() != EventType.REGION_OPEN) { 6418 LOG.warn(getRegionInfo().getEncodedName() + " : " 6419 + "Unknown region event received, ignoring :" + TextFormat.shortDebugString(regionEvent)); 6420 return; 6421 } 6422 6423 if (LOG.isDebugEnabled()) { 6424 LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying region open event marker " 6425 + TextFormat.shortDebugString(regionEvent)); 6426 } 6427 6428 // we will use writestate as a coarse-grain lock for all the replay events 6429 synchronized (writestate) { 6430 // Replication can deliver events out of order when primary region moves or the region 6431 // server crashes, since there is no coordination between replication of different wal files 6432 // belonging to different region servers. We have to safe guard against this case by using 6433 // region open event's seqid. Since this is the first event that the region puts (after 6434 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit 6435 // smaller than this seqId 6436 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) { 6437 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber(); 6438 } else { 6439 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying region event :" 6440 + TextFormat.shortDebugString(regionEvent) 6441 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId " 6442 + " of " + lastReplayedOpenRegionSeqId); 6443 return; 6444 } 6445 6446 // region open lists all the files that the region has at the time of the opening. Just pick 6447 // all the files and drop prepared flushes and empty memstores 6448 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) { 6449 // stores of primary may be different now 6450 byte[] family = storeDescriptor.getFamilyName().toByteArray(); 6451 HStore store = getStore(family); 6452 if (store == null) { 6453 LOG.warn(getRegionInfo().getEncodedName() + " : " 6454 + "Received a region open marker from primary, but the family is not found. " 6455 + "Ignoring. StoreDescriptor:" + storeDescriptor); 6456 continue; 6457 } 6458 6459 long storeSeqId = store.getMaxSequenceId().orElse(0L); 6460 List<String> storeFiles = storeDescriptor.getStoreFileList(); 6461 try { 6462 store.refreshStoreFiles(storeFiles); // replace the files with the new ones 6463 } catch (FileNotFoundException ex) { 6464 LOG.warn(getRegionInfo().getEncodedName() + " : " + "At least one of the store files: " 6465 + storeFiles + " doesn't exist any more. Skip loading the file(s)", ex); 6466 continue; 6467 } 6468 if (store.getMaxSequenceId().orElse(0L) != storeSeqId) { 6469 // Record latest flush time if we picked up new files 6470 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime()); 6471 } 6472 6473 if (writestate.flushing) { 6474 // only drop memstore snapshots if they are smaller than last flush for the store 6475 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) { 6476 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null 6477 ? null 6478 : this.prepareFlushResult.storeFlushCtxs.get(family); 6479 if (ctx != null) { 6480 MemStoreSize mss = store.getFlushableSize(); 6481 ctx.abort(); 6482 this.decrMemStoreSize(mss); 6483 this.prepareFlushResult.storeFlushCtxs.remove(family); 6484 } 6485 } 6486 } 6487 6488 // Drop the memstore contents if they are now smaller than the latest seen flushed file 6489 dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store); 6490 if (storeSeqId > this.maxFlushedSeqId) { 6491 this.maxFlushedSeqId = storeSeqId; 6492 } 6493 } 6494 6495 // if all stores ended up dropping their snapshots, we can safely drop the 6496 // prepareFlushResult 6497 dropPrepareFlushIfPossible(); 6498 6499 // advance the mvcc read point so that the new flushed file is visible. 6500 mvcc.await(); 6501 6502 // If we were waiting for observing a flush or region opening event for not showing partial 6503 // data after a secondary region crash, we can allow reads now. 6504 this.setReadsEnabled(true); 6505 6506 // C. Finally notify anyone waiting on memstore to clear: 6507 // e.g. checkResources(). 6508 synchronized (this) { 6509 notifyAll(); // FindBugs NN_NAKED_NOTIFY 6510 } 6511 } 6512 logRegionFiles(); 6513 } finally { 6514 closeRegionOperation(Operation.REPLAY_EVENT); 6515 } 6516 } 6517 6518 /** 6519 * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region 6520 * replica implementation. 6521 */ 6522 @Deprecated 6523 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException { 6524 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(), 6525 "BulkLoad marker from WAL ", bulkLoadEvent); 6526 6527 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6528 return; // if primary nothing to do 6529 } 6530 6531 if (LOG.isDebugEnabled()) { 6532 LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying bulkload event marker " 6533 + TextFormat.shortDebugString(bulkLoadEvent)); 6534 } 6535 // check if multiple families involved 6536 boolean multipleFamilies = false; 6537 byte[] family = null; 6538 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) { 6539 byte[] fam = storeDescriptor.getFamilyName().toByteArray(); 6540 if (family == null) { 6541 family = fam; 6542 } else if (!Bytes.equals(family, fam)) { 6543 multipleFamilies = true; 6544 break; 6545 } 6546 } 6547 6548 startBulkRegionOperation(multipleFamilies); 6549 try { 6550 // we will use writestate as a coarse-grain lock for all the replay events 6551 synchronized (writestate) { 6552 // Replication can deliver events out of order when primary region moves or the region 6553 // server crashes, since there is no coordination between replication of different wal files 6554 // belonging to different region servers. We have to safe guard against this case by using 6555 // region open event's seqid. Since this is the first event that the region puts (after 6556 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit 6557 // smaller than this seqId 6558 if ( 6559 bulkLoadEvent.getBulkloadSeqNum() >= 0 6560 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum() 6561 ) { 6562 LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying bulkload event :" 6563 + TextFormat.shortDebugString(bulkLoadEvent) 6564 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId" 6565 + " =" + lastReplayedOpenRegionSeqId); 6566 6567 return; 6568 } 6569 6570 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) { 6571 // stores of primary may be different now 6572 family = storeDescriptor.getFamilyName().toByteArray(); 6573 HStore store = getStore(family); 6574 if (store == null) { 6575 LOG.warn(getRegionInfo().getEncodedName() + " : " 6576 + "Received a bulk load marker from primary, but the family is not found. " 6577 + "Ignoring. StoreDescriptor:" + storeDescriptor); 6578 continue; 6579 } 6580 6581 StoreContext storeContext = store.getStoreContext(); 6582 StoreFileTracker sft = StoreFileTrackerFactory.create(conf, false, storeContext); 6583 6584 List<StoreFileInfo> storeFiles = sft.load(); 6585 for (StoreFileInfo storeFileInfo : storeFiles) { 6586 try { 6587 store.bulkLoadHFile(storeFileInfo); 6588 } catch (FileNotFoundException ex) { 6589 LOG.warn(getRegionInfo().getEncodedName() + " : " + storeFileInfo.toString() 6590 + " doesn't exist any more. Skip loading the file"); 6591 } 6592 } 6593 } 6594 } 6595 if (bulkLoadEvent.getBulkloadSeqNum() > 0) { 6596 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum()); 6597 } 6598 } finally { 6599 closeBulkRegionOperation(); 6600 } 6601 } 6602 6603 /** 6604 * Replay the batch mutate for secondary replica. 6605 * <p/> 6606 * We will directly apply the cells to the memstore. This is because: 6607 * <ol> 6608 * <li>All the cells are gotten from {@link WALEdit}, so we only have {@link Put} and 6609 * {@link Delete} here</li> 6610 * <li>The replay is single threaded, we do not need to acquire row lock, as the region is read 6611 * only so no one else can write it.</li> 6612 * <li>We do not need to write WAL.</li> 6613 * <li>We will advance MVCC in the caller directly.</li> 6614 * </ol> 6615 */ 6616 private void replayWALBatchMutate(Map<byte[], List<ExtendedCell>> family2Cells) 6617 throws IOException { 6618 startRegionOperation(Operation.REPLAY_BATCH_MUTATE); 6619 try { 6620 for (Map.Entry<byte[], List<ExtendedCell>> entry : family2Cells.entrySet()) { 6621 applyToMemStore(getStore(entry.getKey()), entry.getValue(), false, memStoreSizing); 6622 } 6623 } finally { 6624 closeRegionOperation(Operation.REPLAY_BATCH_MUTATE); 6625 } 6626 } 6627 6628 /** 6629 * Replay the meta edits, i.e, flush marker, compaction marker, bulk load marker, region event 6630 * marker, etc. 6631 * <p/> 6632 * For all events other than start flush, we will just call {@link #refreshStoreFiles()} as the 6633 * logic is straight-forward and robust. For start flush, we need to snapshot the memstore, so 6634 * later {@link #refreshStoreFiles()} call could drop the snapshot, otherwise we may run out of 6635 * memory. 6636 */ 6637 private void replayWALMetaEdit(Cell cell) throws IOException { 6638 startRegionOperation(Operation.REPLAY_EVENT); 6639 try { 6640 FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell); 6641 if (flushDesc != null) { 6642 switch (flushDesc.getAction()) { 6643 case START_FLUSH: 6644 // for start flush, we need to take a snapshot of the current memstore 6645 synchronized (writestate) { 6646 if (!writestate.flushing) { 6647 this.writestate.flushing = true; 6648 } else { 6649 // usually this should not happen but let's make the code more robust, it is not a 6650 // big deal to just ignore it, the refreshStoreFiles call should have the ability to 6651 // clean up the inconsistent state. 6652 LOG.debug("NOT flushing {} as already flushing", getRegionInfo()); 6653 break; 6654 } 6655 } 6656 MonitoredTask status = 6657 TaskMonitor.get().createStatus("Preparing flush " + getRegionInfo()); 6658 Collection<HStore> storesToFlush = getStoresToFlush(flushDesc); 6659 try { 6660 PrepareFlushResult prepareResult = 6661 internalPrepareFlushCache(null, flushDesc.getFlushSequenceNumber(), storesToFlush, 6662 status, false, FlushLifeCycleTracker.DUMMY); 6663 if (prepareResult.result == null) { 6664 // save the PrepareFlushResult so that we can use it later from commit flush 6665 this.prepareFlushResult = prepareResult; 6666 status.markComplete("Flush prepare successful"); 6667 if (LOG.isDebugEnabled()) { 6668 LOG.debug("{} prepared flush with seqId: {}", getRegionInfo(), 6669 flushDesc.getFlushSequenceNumber()); 6670 } 6671 } else { 6672 // special case empty memstore. We will still save the flush result in this case, 6673 // since our memstore is empty, but the primary is still flushing 6674 if ( 6675 prepareResult.getResult().getResult() 6676 == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY 6677 ) { 6678 this.prepareFlushResult = prepareResult; 6679 if (LOG.isDebugEnabled()) { 6680 LOG.debug("{} prepared empty flush with seqId: {}", getRegionInfo(), 6681 flushDesc.getFlushSequenceNumber()); 6682 } 6683 } 6684 status.abort("Flush prepare failed with " + prepareResult.result); 6685 // nothing much to do. prepare flush failed because of some reason. 6686 } 6687 } finally { 6688 status.cleanup(); 6689 } 6690 break; 6691 case ABORT_FLUSH: 6692 // do nothing, an abort flush means the source region server will crash itself, after 6693 // the primary region online, it will send us an open region marker, then we can clean 6694 // up the memstore. 6695 synchronized (writestate) { 6696 writestate.flushing = false; 6697 } 6698 break; 6699 case COMMIT_FLUSH: 6700 case CANNOT_FLUSH: 6701 // just call refreshStoreFiles 6702 refreshStoreFiles(); 6703 logRegionFiles(); 6704 synchronized (writestate) { 6705 writestate.flushing = false; 6706 } 6707 break; 6708 default: 6709 LOG.warn("{} received a flush event with unknown action: {}", getRegionInfo(), 6710 TextFormat.shortDebugString(flushDesc)); 6711 } 6712 } else { 6713 // for all other region events, we will do a refreshStoreFiles 6714 refreshStoreFiles(); 6715 logRegionFiles(); 6716 } 6717 } finally { 6718 closeRegionOperation(Operation.REPLAY_EVENT); 6719 } 6720 } 6721 6722 /** 6723 * Replay remote wal entry sent by primary replica. 6724 * <p/> 6725 * Should only call this method on secondary replicas. 6726 */ 6727 void replayWALEntry(WALEntry entry, CellScanner cells) throws IOException { 6728 long timeout = -1L; 6729 Optional<RpcCall> call = RpcServer.getCurrentCall(); 6730 if (call.isPresent()) { 6731 long deadline = call.get().getDeadline(); 6732 if (deadline < Long.MAX_VALUE) { 6733 timeout = deadline - EnvironmentEdgeManager.currentTime(); 6734 if (timeout <= 0) { 6735 throw new TimeoutIOException("Timeout while replaying edits for " + getRegionInfo()); 6736 } 6737 } 6738 } 6739 if (timeout > 0) { 6740 try { 6741 if (!replayLock.tryLock(timeout, TimeUnit.MILLISECONDS)) { 6742 throw new TimeoutIOException( 6743 "Timeout while waiting for lock when replaying edits for " + getRegionInfo()); 6744 } 6745 } catch (InterruptedException e) { 6746 throw throwOnInterrupt(e); 6747 } 6748 } else { 6749 replayLock.lock(); 6750 } 6751 try { 6752 int count = entry.getAssociatedCellCount(); 6753 long sequenceId = entry.getKey().getLogSequenceNumber(); 6754 if (lastReplayedSequenceId >= sequenceId) { 6755 // we have already replayed this edit, skip 6756 // remember to advance the CellScanner, as we may have multiple WALEntries, we may still 6757 // need apply later WALEntries 6758 for (int i = 0; i < count; i++) { 6759 // Throw index out of bounds if our cell count is off 6760 if (!cells.advance()) { 6761 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i); 6762 } 6763 } 6764 return; 6765 } 6766 Map<byte[], List<ExtendedCell>> family2Cells = new TreeMap<>(Bytes.BYTES_COMPARATOR); 6767 for (int i = 0; i < count; i++) { 6768 // Throw index out of bounds if our cell count is off 6769 if (!cells.advance()) { 6770 throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i); 6771 } 6772 Cell c = cells.current(); 6773 assert c instanceof ExtendedCell; 6774 ExtendedCell cell = (ExtendedCell) c; 6775 if (WALEdit.isMetaEditFamily(cell)) { 6776 // If there is meta edit, i.e, we have done flush/compaction/open, then we need to apply 6777 // the previous cells first, and then replay the special meta edit. The meta edit is like 6778 // a barrier, We need to keep the order. For example, the flush marker will contain a 6779 // flush sequence number, which makes us possible to drop memstore content, but if we 6780 // apply some edits which have greater sequence id first, then we can not drop the 6781 // memstore content when replaying the flush marker, which is not good as we could run out 6782 // of memory. 6783 // And usually, a meta edit will have a special WALEntry for it, so this is just a safe 6784 // guard logic to make sure we do not break things in the worst case. 6785 if (!family2Cells.isEmpty()) { 6786 replayWALBatchMutate(family2Cells); 6787 family2Cells.clear(); 6788 } 6789 replayWALMetaEdit(cell); 6790 } else { 6791 family2Cells.computeIfAbsent(CellUtil.cloneFamily(cell), k -> new ArrayList<>()) 6792 .add(cell); 6793 } 6794 } 6795 // do not forget to apply the remaining cells 6796 if (!family2Cells.isEmpty()) { 6797 replayWALBatchMutate(family2Cells); 6798 } 6799 mvcc.advanceTo(sequenceId); 6800 lastReplayedSequenceId = sequenceId; 6801 } finally { 6802 replayLock.unlock(); 6803 } 6804 } 6805 6806 /** 6807 * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult 6808 */ 6809 private void dropPrepareFlushIfPossible() { 6810 if (writestate.flushing) { 6811 boolean canDrop = true; 6812 if (prepareFlushResult.storeFlushCtxs != null) { 6813 for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs 6814 .entrySet()) { 6815 HStore store = getStore(entry.getKey()); 6816 if (store == null) { 6817 continue; 6818 } 6819 if (store.getSnapshotSize().getDataSize() > 0) { 6820 canDrop = false; 6821 break; 6822 } 6823 } 6824 } 6825 6826 // this means that all the stores in the region has finished flushing, but the WAL marker 6827 // may not have been written or we did not receive it yet. 6828 if (canDrop) { 6829 writestate.flushing = false; 6830 this.prepareFlushResult = null; 6831 } 6832 } 6833 } 6834 6835 @Override 6836 public boolean refreshStoreFiles() throws IOException { 6837 return refreshStoreFiles(false); 6838 } 6839 6840 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY", 6841 justification = "Notify is about post replay. Intentional") 6842 protected boolean refreshStoreFiles(boolean force) throws IOException { 6843 if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) { 6844 return false; // if primary nothing to do 6845 } 6846 6847 if (LOG.isDebugEnabled()) { 6848 LOG.debug(getRegionInfo().getEncodedName() + " : " 6849 + "Refreshing store files to see whether we can free up memstore"); 6850 } 6851 6852 long totalFreedDataSize = 0; 6853 6854 long smallestSeqIdInStores = Long.MAX_VALUE; 6855 6856 startRegionOperation(); // obtain region close lock 6857 try { 6858 Map<HStore, Long> map = new HashMap<>(); 6859 synchronized (writestate) { 6860 for (HStore store : stores.values()) { 6861 // TODO: some stores might see new data from flush, while others do not which 6862 // MIGHT break atomic edits across column families. 6863 long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L); 6864 6865 // refresh the store files. This is similar to observing a region open wal marker. 6866 store.refreshStoreFiles(); 6867 6868 long storeSeqId = store.getMaxSequenceId().orElse(0L); 6869 if (storeSeqId < smallestSeqIdInStores) { 6870 smallestSeqIdInStores = storeSeqId; 6871 } 6872 6873 // see whether we can drop the memstore or the snapshot 6874 if (storeSeqId > maxSeqIdBefore) { 6875 if (writestate.flushing) { 6876 // only drop memstore snapshots if they are smaller than last flush for the store 6877 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) { 6878 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null 6879 ? null 6880 : this.prepareFlushResult.storeFlushCtxs 6881 .get(store.getColumnFamilyDescriptor().getName()); 6882 if (ctx != null) { 6883 MemStoreSize mss = store.getFlushableSize(); 6884 ctx.abort(); 6885 this.decrMemStoreSize(mss); 6886 this.prepareFlushResult.storeFlushCtxs 6887 .remove(store.getColumnFamilyDescriptor().getName()); 6888 totalFreedDataSize += mss.getDataSize(); 6889 } 6890 } 6891 } 6892 6893 map.put(store, storeSeqId); 6894 } 6895 } 6896 6897 // if all stores ended up dropping their snapshots, we can safely drop the 6898 // prepareFlushResult 6899 dropPrepareFlushIfPossible(); 6900 6901 // advance the mvcc read point so that the new flushed files are visible. 6902 // either greater than flush seq number or they were already picked up via flush. 6903 for (HStore s : stores.values()) { 6904 mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L)); 6905 } 6906 6907 // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely 6908 // skip all edits that are to be replayed in the future with that has a smaller seqId 6909 // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits 6910 // that we have picked the flush files for 6911 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) { 6912 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores; 6913 } 6914 } 6915 if (!map.isEmpty()) { 6916 for (Map.Entry<HStore, Long> entry : map.entrySet()) { 6917 // Drop the memstore contents if they are now smaller than the latest seen flushed file 6918 totalFreedDataSize += 6919 dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey()).getDataSize(); 6920 } 6921 } 6922 // C. Finally notify anyone waiting on memstore to clear: 6923 // e.g. checkResources(). 6924 synchronized (this) { 6925 notifyAll(); // FindBugs NN_NAKED_NOTIFY 6926 } 6927 return totalFreedDataSize > 0; 6928 } finally { 6929 closeRegionOperation(); 6930 } 6931 } 6932 6933 private void logRegionFiles() { 6934 if (LOG.isTraceEnabled()) { 6935 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: "); 6936 stores.values().stream().filter(s -> s.getStorefiles() != null) 6937 .flatMap(s -> s.getStorefiles().stream()) 6938 .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf)); 6939 } 6940 } 6941 6942 /** 6943 * Checks whether the given regionName is either equal to our region, or that the regionName is 6944 * the primary region to our corresponding range for the secondary replica. 6945 */ 6946 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload) 6947 throws WrongRegionException { 6948 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) { 6949 return; 6950 } 6951 6952 if ( 6953 !RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) 6954 && Bytes.equals(encodedRegionName, this.fs.getRegionInfoForFS().getEncodedNameAsBytes()) 6955 ) { 6956 return; 6957 } 6958 6959 throw new WrongRegionException( 6960 exceptionMsg + payload + " targetted for region " + Bytes.toStringBinary(encodedRegionName) 6961 + " does not match this region: " + this.getRegionInfo()); 6962 } 6963 6964 /** 6965 * Used by tests 6966 * @param s Store to add edit too. 6967 * @param cell Cell to add. 6968 */ 6969 protected void restoreEdit(HStore s, ExtendedCell cell, MemStoreSizing memstoreAccounting) { 6970 s.add(cell, memstoreAccounting); 6971 } 6972 6973 /** 6974 * make sure have been through lease recovery before get file status, so the file length can be 6975 * trusted. 6976 * @param p File to check. 6977 * @return True if file was zero-length (and if so, we'll delete it in here). 6978 */ 6979 private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat, 6980 final Path p) throws IOException { 6981 if (stat.getLen() > 0) { 6982 return false; 6983 } 6984 LOG.warn("File " + p + " is zero-length, deleting."); 6985 fs.delete(p, false); 6986 return true; 6987 } 6988 6989 protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup) 6990 throws IOException { 6991 if (family.isMobEnabled()) { 6992 if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) { 6993 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS 6994 + " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY 6995 + " accordingly."); 6996 } 6997 return new HMobStore(this, family, this.conf, warmup); 6998 } 6999 return new HStore(this, family, this.conf, warmup); 7000 } 7001 7002 @Override 7003 public HStore getStore(byte[] column) { 7004 return this.stores.get(column); 7005 } 7006 7007 /** 7008 * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on 7009 * the list. 7010 */ 7011 private HStore getStore(Cell cell) { 7012 return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey())) 7013 .map(e -> e.getValue()).findFirst().orElse(null); 7014 } 7015 7016 @Override 7017 public List<HStore> getStores() { 7018 return new ArrayList<>(stores.values()); 7019 } 7020 7021 @Override 7022 public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException { 7023 List<String> storeFileNames = new ArrayList<>(); 7024 synchronized (closeLock) { 7025 for (byte[] column : columns) { 7026 HStore store = this.stores.get(column); 7027 if (store == null) { 7028 throw new IllegalArgumentException( 7029 "No column family : " + new String(column, StandardCharsets.UTF_8) + " available"); 7030 } 7031 Collection<HStoreFile> storeFiles = store.getStorefiles(); 7032 if (storeFiles == null) { 7033 continue; 7034 } 7035 for (HStoreFile storeFile : storeFiles) { 7036 storeFileNames.add(storeFile.getPath().toString()); 7037 } 7038 7039 logRegionFiles(); 7040 } 7041 } 7042 return storeFileNames; 7043 } 7044 7045 ////////////////////////////////////////////////////////////////////////////// 7046 // Support code 7047 ////////////////////////////////////////////////////////////////////////////// 7048 7049 /** Make sure this is a valid row for the HRegion */ 7050 void checkRow(byte[] row, String op) throws IOException { 7051 if (!rowIsInRange(getRegionInfo(), row)) { 7052 throw new WrongRegionException("Requested row out of range for " + op + " on HRegion " + this 7053 + ", startKey='" + Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" 7054 + Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" + Bytes.toStringBinary(row) 7055 + "'"); 7056 } 7057 } 7058 7059 /** 7060 * Get an exclusive ( write lock ) lock on a given row. 7061 * @param row Which row to lock. 7062 * @return A locked RowLock. The lock is exclusive and already aqquired. 7063 */ 7064 public RowLock getRowLock(byte[] row) throws IOException { 7065 return getRowLock(row, false); 7066 } 7067 7068 @Override 7069 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException { 7070 checkRow(row, "row lock"); 7071 return getRowLock(row, readLock, null); 7072 } 7073 7074 Span createRegionSpan(String name) { 7075 return TraceUtil.createSpan(name).setAttribute(REGION_NAMES_KEY, 7076 Collections.singletonList(getRegionInfo().getRegionNameAsString())); 7077 } 7078 7079 // will be override in tests 7080 protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock) 7081 throws IOException { 7082 // create an object to use a a key in the row lock map 7083 HashedBytes rowKey = new HashedBytes(row); 7084 7085 RowLockContext rowLockContext = null; 7086 RowLockImpl result = null; 7087 7088 boolean success = false; 7089 try { 7090 // Keep trying until we have a lock or error out. 7091 // TODO: do we need to add a time component here? 7092 while (result == null) { 7093 rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey)); 7094 // Now try an get the lock. 7095 // This can fail as 7096 if (readLock) { 7097 // For read lock, if the caller has locked the same row previously, it will not try 7098 // to acquire the same read lock. It simply returns the previous row lock. 7099 RowLockImpl prevRowLockImpl = (RowLockImpl) prevRowLock; 7100 if ( 7101 (prevRowLockImpl != null) 7102 && (prevRowLockImpl.getLock() == rowLockContext.readWriteLock.readLock()) 7103 ) { 7104 success = true; 7105 return prevRowLock; 7106 } 7107 result = rowLockContext.newReadLock(); 7108 } else { 7109 result = rowLockContext.newWriteLock(); 7110 } 7111 } 7112 7113 int timeout = rowLockWaitDuration; 7114 boolean reachDeadlineFirst = false; 7115 Optional<RpcCall> call = RpcServer.getCurrentCall(); 7116 if (call.isPresent()) { 7117 long deadline = call.get().getDeadline(); 7118 if (deadline < Long.MAX_VALUE) { 7119 int timeToDeadline = (int) (deadline - EnvironmentEdgeManager.currentTime()); 7120 if (timeToDeadline <= this.rowLockWaitDuration) { 7121 reachDeadlineFirst = true; 7122 timeout = timeToDeadline; 7123 } 7124 } 7125 } 7126 7127 if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) { 7128 String message = "Timed out waiting for lock for row: " + rowKey + " in region " 7129 + getRegionInfo().getEncodedName(); 7130 if (reachDeadlineFirst) { 7131 throw new TimeoutIOException(message); 7132 } else { 7133 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request. 7134 throw new IOException(message); 7135 } 7136 } 7137 rowLockContext.setThreadName(Thread.currentThread().getName()); 7138 success = true; 7139 return result; 7140 } catch (InterruptedException ie) { 7141 if (LOG.isDebugEnabled()) { 7142 LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey, 7143 getRegionInfo().getRegionNameAsString()); 7144 } 7145 throw throwOnInterrupt(ie); 7146 } catch (Error error) { 7147 // The maximum lock count for read lock is 64K (hardcoded), when this maximum count 7148 // is reached, it will throw out an Error. This Error needs to be caught so it can 7149 // go ahead to process the minibatch with lock acquired. 7150 LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row), 7151 getRegionInfo().getRegionNameAsString(), error); 7152 IOException ioe = new IOException(error); 7153 throw ioe; 7154 } finally { 7155 // Clean up the counts just in case this was the thing keeping the context alive. 7156 if (!success && rowLockContext != null) { 7157 rowLockContext.cleanUp(); 7158 } 7159 } 7160 } 7161 7162 private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock) 7163 throws IOException { 7164 return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock), 7165 () -> createRegionSpan("Region.getRowLock").setAttribute(ROW_LOCK_READ_LOCK_KEY, readLock)); 7166 } 7167 7168 private void releaseRowLocks(List<RowLock> rowLocks) { 7169 if (rowLocks != null) { 7170 for (RowLock rowLock : rowLocks) { 7171 rowLock.release(); 7172 } 7173 rowLocks.clear(); 7174 } 7175 } 7176 7177 public int getReadLockCount() { 7178 return lock.getReadLockCount(); 7179 } 7180 7181 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() { 7182 return lockedRows; 7183 } 7184 7185 class RowLockContext { 7186 private final HashedBytes row; 7187 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true); 7188 final AtomicBoolean usable = new AtomicBoolean(true); 7189 final AtomicInteger count = new AtomicInteger(0); 7190 final Object lock = new Object(); 7191 private String threadName; 7192 7193 RowLockContext(HashedBytes row) { 7194 this.row = row; 7195 } 7196 7197 RowLockImpl newWriteLock() { 7198 Lock l = readWriteLock.writeLock(); 7199 return getRowLock(l); 7200 } 7201 7202 RowLockImpl newReadLock() { 7203 Lock l = readWriteLock.readLock(); 7204 return getRowLock(l); 7205 } 7206 7207 private RowLockImpl getRowLock(Lock l) { 7208 count.incrementAndGet(); 7209 synchronized (lock) { 7210 if (usable.get()) { 7211 return new RowLockImpl(this, l); 7212 } else { 7213 return null; 7214 } 7215 } 7216 } 7217 7218 void cleanUp() { 7219 long c = count.decrementAndGet(); 7220 if (c <= 0) { 7221 synchronized (lock) { 7222 if (count.get() <= 0 && usable.get()) { // Don't attempt to remove row if already removed 7223 usable.set(false); 7224 RowLockContext removed = lockedRows.remove(row); 7225 assert removed == this : "we should never remove a different context"; 7226 } 7227 } 7228 } 7229 } 7230 7231 public void setThreadName(String threadName) { 7232 this.threadName = threadName; 7233 } 7234 7235 @Override 7236 public String toString() { 7237 return "RowLockContext{" + "row=" + row + ", readWriteLock=" + readWriteLock + ", count=" 7238 + count + ", threadName=" + threadName + '}'; 7239 } 7240 } 7241 7242 /** 7243 * Class used to represent a lock on a row. 7244 */ 7245 public static class RowLockImpl implements RowLock { 7246 private final RowLockContext context; 7247 private final Lock lock; 7248 7249 public RowLockImpl(RowLockContext context, Lock lock) { 7250 this.context = context; 7251 this.lock = lock; 7252 } 7253 7254 public Lock getLock() { 7255 return lock; 7256 } 7257 7258 public RowLockContext getContext() { 7259 return context; 7260 } 7261 7262 @Override 7263 public void release() { 7264 lock.unlock(); 7265 context.cleanUp(); 7266 } 7267 7268 @Override 7269 public String toString() { 7270 return "RowLockImpl{" + "context=" + context + ", lock=" + lock + '}'; 7271 } 7272 } 7273 7274 /** 7275 * Determines whether multiple column families are present Precondition: familyPaths is not null 7276 * @param familyPaths List of (column family, hfilePath) 7277 */ 7278 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) { 7279 boolean multipleFamilies = false; 7280 byte[] family = null; 7281 for (Pair<byte[], String> pair : familyPaths) { 7282 byte[] fam = pair.getFirst(); 7283 if (family == null) { 7284 family = fam; 7285 } else if (!Bytes.equals(family, fam)) { 7286 multipleFamilies = true; 7287 break; 7288 } 7289 } 7290 return multipleFamilies; 7291 } 7292 7293 /** 7294 * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple 7295 * column families atomically. 7296 * @param familyPaths List of Pair<byte[] column family, String hfilePath> 7297 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be 7298 * bulk loaded 7299 * @return Map from family to List of store file paths if successful, null if failed recoverably 7300 * @throws IOException if failed unrecoverably. 7301 */ 7302 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, 7303 boolean assignSeqId, BulkLoadListener bulkLoadListener) throws IOException { 7304 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false, null, true); 7305 } 7306 7307 /** 7308 * Listener class to enable callers of bulkLoadHFile() to perform any necessary pre/post 7309 * processing of a given bulkload call 7310 */ 7311 public interface BulkLoadListener { 7312 /** 7313 * Called before an HFile is actually loaded 7314 * @param family family being loaded to 7315 * @param srcPath path of HFile 7316 * @return final path to be used for actual loading 7317 */ 7318 String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile, String customStaging) 7319 throws IOException; 7320 7321 /** 7322 * Called after a successful HFile load 7323 * @param family family being loaded to 7324 * @param srcPath path of HFile 7325 */ 7326 void doneBulkLoad(byte[] family, String srcPath) throws IOException; 7327 7328 /** 7329 * Called after a failed HFile load 7330 * @param family family being loaded to 7331 * @param srcPath path of HFile 7332 */ 7333 void failedBulkLoad(byte[] family, String srcPath) throws IOException; 7334 } 7335 7336 /** 7337 * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple 7338 * column families atomically. 7339 * @param familyPaths List of Pair<byte[] column family, String hfilePath> 7340 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be 7341 * bulk loaded 7342 * @param copyFile always copy hfiles if true 7343 * @param clusterIds ids from clusters that had already handled the given bulkload event. 7344 * @return Map from family to List of store file paths if successful, null if failed recoverably 7345 * @throws IOException if failed unrecoverably. 7346 */ 7347 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, 7348 boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile, 7349 List<String> clusterIds, boolean replicate) throws IOException { 7350 long seqId = -1; 7351 Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); 7352 Map<String, Long> storeFilesSizes = new HashMap<>(); 7353 Preconditions.checkNotNull(familyPaths); 7354 // we need writeLock for multi-family bulk load 7355 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths)); 7356 boolean isSuccessful = false; 7357 try { 7358 this.writeRequestsCount.increment(); 7359 7360 // There possibly was a split that happened between when the split keys 7361 // were gathered and before the HRegion's write lock was taken. We need 7362 // to validate the HFile region before attempting to bulk load all of them 7363 IOException ioException = null; 7364 List<Pair<byte[], String>> failures = new ArrayList<>(); 7365 for (Pair<byte[], String> p : familyPaths) { 7366 byte[] familyName = p.getFirst(); 7367 String path = p.getSecond(); 7368 7369 HStore store = getStore(familyName); 7370 if (store == null) { 7371 ioException = new org.apache.hadoop.hbase.DoNotRetryIOException( 7372 "No such column family " + Bytes.toStringBinary(familyName)); 7373 } else { 7374 try { 7375 store.assertBulkLoadHFileOk(new Path(path)); 7376 } catch (WrongRegionException wre) { 7377 // recoverable (file doesn't fit in region) 7378 failures.add(p); 7379 } catch (IOException ioe) { 7380 // unrecoverable (hdfs problem) 7381 ioException = ioe; 7382 } 7383 } 7384 7385 // validation failed because of some sort of IO problem. 7386 if (ioException != null) { 7387 LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this, 7388 ioException); 7389 throw ioException; 7390 } 7391 } 7392 // validation failed, bail out before doing anything permanent. 7393 if (failures.size() != 0) { 7394 StringBuilder list = new StringBuilder(); 7395 for (Pair<byte[], String> p : failures) { 7396 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ") 7397 .append(p.getSecond()); 7398 } 7399 // problem when validating 7400 LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family," 7401 + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this); 7402 return null; 7403 } 7404 7405 // We need to assign a sequential ID that's in between two memstores in order to preserve 7406 // the guarantee that all the edits lower than the highest sequential ID from all the 7407 // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is 7408 // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is 7409 // a sequence id that we can be sure is beyond the last hfile written). 7410 if (assignSeqId) { 7411 FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY); 7412 if (fs.isFlushSucceeded()) { 7413 seqId = ((FlushResultImpl) fs).flushSequenceId; 7414 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) { 7415 seqId = ((FlushResultImpl) fs).flushSequenceId; 7416 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) { 7417 // CANNOT_FLUSH may mean that a flush is already on-going 7418 // we need to wait for that flush to complete 7419 waitForFlushes(); 7420 } else { 7421 throw new IOException("Could not bulk load with an assigned sequential ID because the " 7422 + "flush didn't run. Reason for not flushing: " + ((FlushResultImpl) fs).failureReason); 7423 } 7424 } 7425 7426 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath = 7427 new TreeMap<>(Bytes.BYTES_COMPARATOR); 7428 for (Pair<byte[], String> p : familyPaths) { 7429 byte[] familyName = p.getFirst(); 7430 String path = p.getSecond(); 7431 HStore store = getStore(familyName); 7432 if (!familyWithFinalPath.containsKey(familyName)) { 7433 familyWithFinalPath.put(familyName, new ArrayList<>()); 7434 } 7435 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName); 7436 String finalPath = path; 7437 try { 7438 boolean reqTmp = store.storeEngine.requireWritingToTmpDirFirst(); 7439 if (bulkLoadListener != null) { 7440 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile, 7441 reqTmp ? null : fs.getRegionDir().toString()); 7442 } 7443 Pair<Path, Path> pair = null; 7444 if (reqTmp || !StoreFileInfo.isHFile(finalPath)) { 7445 pair = store.preBulkLoadHFile(finalPath, seqId); 7446 } else { 7447 Path livePath = new Path(finalPath); 7448 pair = new Pair<>(livePath, livePath); 7449 } 7450 lst.add(pair); 7451 } catch (IOException ioe) { 7452 // A failure here can cause an atomicity violation that we currently 7453 // cannot recover from since it is likely a failed HDFS operation. 7454 7455 LOG.error("There was a partial failure due to IO when attempting to" + " load " 7456 + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe); 7457 if (bulkLoadListener != null) { 7458 try { 7459 bulkLoadListener.failedBulkLoad(familyName, finalPath); 7460 } catch (Exception ex) { 7461 LOG.error("Error while calling failedBulkLoad for family " 7462 + Bytes.toString(familyName) + " with path " + path, ex); 7463 } 7464 } 7465 throw ioe; 7466 } 7467 } 7468 7469 if (this.getCoprocessorHost() != null) { 7470 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) { 7471 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue()); 7472 } 7473 } 7474 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) { 7475 byte[] familyName = entry.getKey(); 7476 for (Pair<Path, Path> p : entry.getValue()) { 7477 String path = p.getFirst().toString(); 7478 Path commitedStoreFile = p.getSecond(); 7479 HStore store = getStore(familyName); 7480 try { 7481 store.bulkLoadHFile(familyName, path, commitedStoreFile); 7482 // Note the size of the store file 7483 try { 7484 FileSystem fs = commitedStoreFile.getFileSystem(baseConf); 7485 storeFilesSizes.put(commitedStoreFile.getName(), 7486 fs.getFileStatus(commitedStoreFile).getLen()); 7487 } catch (IOException e) { 7488 LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e); 7489 storeFilesSizes.put(commitedStoreFile.getName(), 0L); 7490 } 7491 7492 if (storeFiles.containsKey(familyName)) { 7493 storeFiles.get(familyName).add(commitedStoreFile); 7494 } else { 7495 List<Path> storeFileNames = new ArrayList<>(); 7496 storeFileNames.add(commitedStoreFile); 7497 storeFiles.put(familyName, storeFileNames); 7498 } 7499 if (bulkLoadListener != null) { 7500 bulkLoadListener.doneBulkLoad(familyName, path); 7501 } 7502 } catch (IOException ioe) { 7503 // A failure here can cause an atomicity violation that we currently 7504 // cannot recover from since it is likely a failed HDFS operation. 7505 7506 // TODO Need a better story for reverting partial failures due to HDFS. 7507 LOG.error("There was a partial failure due to IO when attempting to" + " load " 7508 + Bytes.toString(familyName) + " : " + p.getSecond(), ioe); 7509 if (bulkLoadListener != null) { 7510 try { 7511 bulkLoadListener.failedBulkLoad(familyName, path); 7512 } catch (Exception ex) { 7513 LOG.error("Error while calling failedBulkLoad for family " 7514 + Bytes.toString(familyName) + " with path " + path, ex); 7515 } 7516 } 7517 throw ioe; 7518 } 7519 } 7520 } 7521 7522 isSuccessful = true; 7523 if (conf.getBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, false)) { 7524 // request compaction 7525 familyWithFinalPath.keySet().forEach(family -> { 7526 HStore store = getStore(family); 7527 try { 7528 if (this.rsServices != null && store.needsCompaction()) { 7529 this.rsServices.getCompactionRequestor().requestSystemCompaction(this, store, 7530 "bulkload hfiles request compaction", true); 7531 LOG.info("Request compaction for region {} family {} after bulk load", 7532 this.getRegionInfo().getEncodedName(), store.getColumnFamilyName()); 7533 } 7534 } catch (IOException e) { 7535 LOG.error("bulkload hfiles request compaction error ", e); 7536 } 7537 }); 7538 } 7539 } finally { 7540 if (wal != null && !storeFiles.isEmpty()) { 7541 // Write a bulk load event for hfiles that are loaded 7542 try { 7543 WALProtos.BulkLoadDescriptor loadDescriptor = 7544 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(), 7545 UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()), 7546 storeFiles, storeFilesSizes, seqId, clusterIds, replicate); 7547 WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(), 7548 loadDescriptor, mvcc, regionReplicationSink.orElse(null)); 7549 } catch (IOException ioe) { 7550 if (this.rsServices != null) { 7551 // Have to abort region server because some hfiles has been loaded but we can't write 7552 // the event into WAL 7553 isSuccessful = false; 7554 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe); 7555 } 7556 } 7557 } 7558 7559 closeBulkRegionOperation(); 7560 } 7561 return isSuccessful ? storeFiles : null; 7562 } 7563 7564 @Override 7565 public boolean equals(Object o) { 7566 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(), 7567 ((HRegion) o).getRegionInfo().getRegionName()); 7568 } 7569 7570 @Override 7571 public int hashCode() { 7572 return Bytes.hashCode(getRegionInfo().getRegionName()); 7573 } 7574 7575 @Override 7576 public String toString() { 7577 return getRegionInfo().getRegionNameAsString(); 7578 } 7579 7580 // Utility methods 7581 /** 7582 * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL} 7583 * configuration property. 7584 * @param tableDir qualified path of directory where region should be located, usually the table 7585 * directory. 7586 * @param wal The WAL is the outbound log for any updates to the HRegion The wal file is a 7587 * logfile from the previous execution that's custom-computed for this HRegion. 7588 * The HRegionServer computes and sorts the appropriate wal info for this 7589 * HRegion. If there is a previous file (implying that the HRegion has been 7590 * written-to before), then read it from the supplied path. 7591 * @param fs is the filesystem. 7592 * @param conf is global configuration settings. 7593 * @param regionInfo - RegionInfo that describes the region is new), then read them from the 7594 * supplied path. 7595 * @param htd the table descriptor 7596 * @return the new instance 7597 */ 7598 public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration conf, 7599 RegionInfo regionInfo, final TableDescriptor htd, RegionServerServices rsServices) { 7600 try { 7601 @SuppressWarnings("unchecked") 7602 Class<? extends HRegion> regionClass = 7603 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class); 7604 7605 Constructor<? extends HRegion> c = 7606 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class, Configuration.class, 7607 RegionInfo.class, TableDescriptor.class, RegionServerServices.class); 7608 7609 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices); 7610 } catch (Throwable e) { 7611 // todo: what should I throw here? 7612 throw new IllegalStateException("Could not instantiate a region instance.", e); 7613 } 7614 } 7615 7616 /** 7617 * Convenience method creating new HRegions. Used by createTable. 7618 * @param info Info for region to create. 7619 * @param rootDir Root directory for HBase instance 7620 * @param wal shared WAL 7621 * @param initialize - true to initialize the region 7622 * @return new HRegion 7623 */ 7624 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7625 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal, 7626 final boolean initialize) throws IOException { 7627 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null); 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 * @param rsRpcServices An interface we can request flushes against. 7637 * @return new HRegion 7638 */ 7639 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7640 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal, 7641 final boolean initialize, RegionServerServices rsRpcServices) throws IOException { 7642 LOG.info("creating " + info + ", tableDescriptor=" 7643 + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir); 7644 createRegionDir(conf, info, rootDir); 7645 FileSystem fs = rootDir.getFileSystem(conf); 7646 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 7647 HRegion region = 7648 HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices); 7649 if (initialize) { 7650 region.initialize(null); 7651 } 7652 return region; 7653 } 7654 7655 /** 7656 * Create a region under the given table directory. 7657 */ 7658 public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs, 7659 Path tableDir, TableDescriptor tableDesc) throws IOException { 7660 LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc, 7661 tableDir); 7662 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo); 7663 HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null); 7664 return region; 7665 } 7666 7667 /** 7668 * Create the region directory in the filesystem. 7669 */ 7670 public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri, 7671 Path rootDir) throws IOException { 7672 FileSystem fs = rootDir.getFileSystem(configuration); 7673 Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable()); 7674 // If directory already exists, will log warning and keep going. Will try to create 7675 // .regioninfo. If one exists, will overwrite. 7676 return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri); 7677 } 7678 7679 public static HRegion createHRegion(final RegionInfo info, final Path rootDir, 7680 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal) 7681 throws IOException { 7682 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true); 7683 } 7684 7685 /** 7686 * Open a Region. 7687 * @param info Info for region to be opened. 7688 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) passing 7689 * the result of the call to HRegion#getMinSequenceId() to ensure the wal id is 7690 * properly kept up. HRegionStore does this every time it opens a new region. 7691 * @return new HRegion 7692 */ 7693 public static HRegion openHRegion(final RegionInfo info, final TableDescriptor htd, final WAL wal, 7694 final Configuration conf) throws IOException { 7695 return openHRegion(info, htd, wal, conf, null, null); 7696 } 7697 7698 /** 7699 * Open a Region. 7700 * @param info Info for region to be opened 7701 * @param htd the table descriptor 7702 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) 7703 * passing the result of the call to HRegion#getMinSequenceId() to ensure the 7704 * wal id is properly kept up. HRegionStore does this every time it opens a new 7705 * region. 7706 * @param conf The Configuration object to use. 7707 * @param rsServices An interface we can request flushes against. 7708 * @param reporter An interface we can report progress against. 7709 * @return new HRegion 7710 */ 7711 public static HRegion openHRegion(final RegionInfo info, final TableDescriptor htd, final WAL wal, 7712 final Configuration conf, final RegionServerServices rsServices, 7713 final CancelableProgressable reporter) throws IOException { 7714 return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter); 7715 } 7716 7717 /** 7718 * Open a Region. 7719 * @param rootDir Root directory for HBase instance 7720 * @param info Info for region to be opened. 7721 * @param htd the table descriptor 7722 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) passing 7723 * the result of the call to HRegion#getMinSequenceId() to ensure the wal id is 7724 * properly kept up. HRegionStore does this every time it opens a new region. 7725 * @param conf The Configuration object to use. 7726 * @return new HRegion 7727 */ 7728 public static HRegion openHRegion(Path rootDir, final RegionInfo info, final TableDescriptor htd, 7729 final WAL wal, final Configuration conf) throws IOException { 7730 return openHRegion(rootDir, info, htd, wal, conf, null, null); 7731 } 7732 7733 /** 7734 * Open a Region. 7735 * @param rootDir Root directory for HBase instance 7736 * @param info Info for region to be opened. 7737 * @param htd the table descriptor 7738 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) 7739 * passing the result of the call to HRegion#getMinSequenceId() to ensure the 7740 * wal id is properly kept up. HRegionStore does this every time it opens a new 7741 * region. 7742 * @param conf The Configuration object to use. 7743 * @param rsServices An interface we can request flushes against. 7744 * @param reporter An interface we can report progress against. 7745 * @return new HRegion 7746 */ 7747 public static HRegion openHRegion(final Path rootDir, final RegionInfo info, 7748 final TableDescriptor htd, final WAL wal, final Configuration conf, 7749 final RegionServerServices rsServices, final CancelableProgressable reporter) 7750 throws IOException { 7751 FileSystem fs = null; 7752 if (rsServices != null) { 7753 fs = rsServices.getFileSystem(); 7754 } 7755 if (fs == null) { 7756 fs = rootDir.getFileSystem(conf); 7757 } 7758 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter); 7759 } 7760 7761 /** 7762 * Open a Region. 7763 * @param conf The Configuration object to use. 7764 * @param fs Filesystem to use 7765 * @param rootDir Root directory for HBase instance 7766 * @param info Info for region to be opened. 7767 * @param htd the table descriptor 7768 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) passing 7769 * the result of the call to HRegion#getMinSequenceId() to ensure the wal id is 7770 * properly kept up. HRegionStore does this every time it opens a new region. 7771 * @return new HRegion 7772 */ 7773 public static HRegion openHRegion(final Configuration conf, final FileSystem fs, 7774 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal) 7775 throws IOException { 7776 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null); 7777 } 7778 7779 /** 7780 * Open a Region. 7781 * @param conf The Configuration object to use. 7782 * @param fs Filesystem to use 7783 * @param rootDir Root directory for HBase instance 7784 * @param info Info for region to be opened. 7785 * @param htd the table descriptor 7786 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) 7787 * passing the result of the call to HRegion#getMinSequenceId() to ensure the 7788 * wal id is properly kept up. HRegionStore does this every time it opens a new 7789 * region. 7790 * @param rsServices An interface we can request flushes against. 7791 * @param reporter An interface we can report progress against. 7792 * @return new HRegion 7793 */ 7794 public static HRegion openHRegion(final Configuration conf, final FileSystem fs, 7795 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal, 7796 final RegionServerServices rsServices, final CancelableProgressable reporter) 7797 throws IOException { 7798 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 7799 return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter); 7800 } 7801 7802 /** 7803 * Open a Region. 7804 * @param conf The Configuration object to use. 7805 * @param fs Filesystem to use 7806 * @param info Info for region to be opened. 7807 * @param htd the table descriptor 7808 * @param wal WAL for region to use. This method will call WAL#setSequenceNumber(long) 7809 * passing the result of the call to HRegion#getMinSequenceId() to ensure the 7810 * wal id is properly kept up. HRegionStore does this every time it opens a new 7811 * region. 7812 * @param rsServices An interface we can request flushes against. 7813 * @param reporter An interface we can report progress against. 7814 * @return new HRegion 7815 * @throws NullPointerException if {@code info} is {@code null} 7816 */ 7817 public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs, 7818 final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal, 7819 final RegionServerServices rsServices, final CancelableProgressable reporter) 7820 throws IOException { 7821 Objects.requireNonNull(info, "RegionInfo cannot be null"); 7822 LOG.debug("Opening region: {}", info); 7823 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices); 7824 return r.openHRegion(reporter); 7825 } 7826 7827 public NavigableMap<byte[], Integer> getReplicationScope() { 7828 return this.replicationScope; 7829 } 7830 7831 /** 7832 * Useful when reopening a closed region (normally for unit tests) 7833 * @param other original object 7834 * @param reporter An interface we can report progress against. 7835 * @return new HRegion 7836 */ 7837 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter) 7838 throws IOException { 7839 HRegionFileSystem regionFs = other.getRegionFileSystem(); 7840 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(), 7841 other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null); 7842 return r.openHRegion(reporter); 7843 } 7844 7845 public static Region openHRegion(final Region other, final CancelableProgressable reporter) 7846 throws IOException { 7847 return openHRegion((HRegion) other, reporter); 7848 } 7849 7850 /** 7851 * Open HRegion. 7852 * <p/> 7853 * Calls initialize and sets sequenceId. 7854 * @return Returns <code>this</code> 7855 */ 7856 private HRegion openHRegion(final CancelableProgressable reporter) throws IOException { 7857 try { 7858 CompoundConfiguration cConfig = 7859 new CompoundConfiguration().add(conf).addBytesMap(htableDescriptor.getValues()); 7860 // Refuse to open the region if we are missing local compression support 7861 TableDescriptorChecker.checkCompression(cConfig, htableDescriptor); 7862 // Refuse to open the region if encryption configuration is incorrect or 7863 // codec support is missing 7864 LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName()); 7865 TableDescriptorChecker.checkEncryption(cConfig, htableDescriptor); 7866 // Refuse to open the region if a required class cannot be loaded 7867 LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName()); 7868 TableDescriptorChecker.checkClassLoading(cConfig, htableDescriptor); 7869 this.openSeqNum = initialize(reporter); 7870 this.mvcc.advanceTo(openSeqNum); 7871 // The openSeqNum must be increased every time when a region is assigned, as we rely on it to 7872 // determine whether a region has been successfully reopened. So here we always write open 7873 // marker, even if the table is read only. 7874 if ( 7875 wal != null && getRegionServerServices() != null 7876 && RegionReplicaUtil.isDefaultReplica(getRegionInfo()) 7877 ) { 7878 writeRegionOpenMarker(wal, openSeqNum); 7879 } 7880 } catch (Throwable t) { 7881 // By coprocessor path wrong region will open failed, 7882 // MetricsRegionWrapperImpl is already init and not close, 7883 // add region close when open failed 7884 try { 7885 // It is not required to write sequence id file when region open is failed. 7886 // Passing true to skip the sequence id file write. 7887 this.close(true); 7888 } catch (Throwable e) { 7889 LOG.warn("Open region: {} failed. Try close region but got exception ", 7890 this.getRegionInfo(), e); 7891 } 7892 throw t; 7893 } 7894 return this; 7895 } 7896 7897 /** 7898 * Open a Region on a read-only file-system (like hdfs snapshots) 7899 * @param conf The Configuration object to use. 7900 * @param fs Filesystem to use 7901 * @param info Info for region to be opened. 7902 * @param htd the table descriptor 7903 * @return new HRegion 7904 * @throws NullPointerException if {@code info} is {@code null} 7905 */ 7906 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs, 7907 final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException { 7908 Objects.requireNonNull(info, "RegionInfo cannot be null"); 7909 if (LOG.isDebugEnabled()) { 7910 LOG.debug("Opening region (readOnly filesystem): " + info); 7911 } 7912 if (info.getReplicaId() <= 0) { 7913 info = RegionReplicaUtil.getRegionInfoForReplica(info, 1); 7914 } 7915 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null); 7916 r.writestate.setReadOnly(true); 7917 return r.openHRegion(null); 7918 } 7919 7920 public static HRegion warmupHRegion(final RegionInfo info, final TableDescriptor htd, 7921 final WAL wal, final Configuration conf, final RegionServerServices rsServices, 7922 final CancelableProgressable reporter) throws IOException { 7923 7924 Objects.requireNonNull(info, "RegionInfo cannot be null"); 7925 LOG.debug("Warmup {}", info); 7926 Path rootDir = CommonFSUtils.getRootDir(conf); 7927 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable()); 7928 FileSystem fs = null; 7929 if (rsServices != null) { 7930 fs = rsServices.getFileSystem(); 7931 } 7932 if (fs == null) { 7933 fs = rootDir.getFileSystem(conf); 7934 } 7935 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null); 7936 r.initializeWarmup(reporter); 7937 r.close(); 7938 return r; 7939 } 7940 7941 /** 7942 * Computes the Path of the HRegion 7943 * @param tabledir qualified path for table 7944 * @param name ENCODED region name 7945 * @return Path of HRegion directory 7946 * @deprecated For tests only; to be removed. 7947 */ 7948 @Deprecated 7949 public static Path getRegionDir(final Path tabledir, final String name) { 7950 return new Path(tabledir, name); 7951 } 7952 7953 /** 7954 * Determines if the specified row is within the row range specified by the specified RegionInfo 7955 * @param info RegionInfo that specifies the row range 7956 * @param row row to be checked 7957 * @return true if the row is within the range specified by the RegionInfo 7958 */ 7959 public static boolean rowIsInRange(RegionInfo info, final byte[] row) { 7960 return ((info.getStartKey().length == 0) || (Bytes.compareTo(info.getStartKey(), row) <= 0)) 7961 && ((info.getEndKey().length == 0) || (Bytes.compareTo(info.getEndKey(), row) > 0)); 7962 } 7963 7964 public static boolean rowIsInRange(RegionInfo info, final byte[] row, final int offset, 7965 final short length) { 7966 return ((info.getStartKey().length == 0) 7967 || (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length, row, offset, length) 7968 <= 0)) 7969 && ((info.getEndKey().length == 0) 7970 || (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) 7971 > 0)); 7972 } 7973 7974 @Override 7975 public Result get(final Get get) throws IOException { 7976 prepareGet(get); 7977 List<Cell> results = get(get, true); 7978 boolean stale = this.getRegionInfo().getReplicaId() != 0; 7979 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale); 7980 } 7981 7982 void prepareGet(final Get get) throws IOException { 7983 checkRow(get.getRow(), "Get"); 7984 // Verify families are all valid 7985 if (get.hasFamilies()) { 7986 for (byte[] family : get.familySet()) { 7987 checkFamily(family); 7988 } 7989 } else { // Adding all families to scanner 7990 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) { 7991 get.addFamily(family); 7992 } 7993 } 7994 } 7995 7996 @Override 7997 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException { 7998 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE); 7999 } 8000 8001 private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce) 8002 throws IOException { 8003 return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce), 8004 () -> createRegionSpan("Region.get")); 8005 } 8006 8007 private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce) 8008 throws IOException { 8009 List<Cell> results = new ArrayList<>(); 8010 8011 // pre-get CP hook 8012 if (withCoprocessor && (coprocessorHost != null)) { 8013 if (coprocessorHost.preGet(get, results)) { 8014 metricsUpdateForGet(); 8015 return results; 8016 } 8017 } 8018 Scan scan = new Scan(get); 8019 if (scan.getLoadColumnFamiliesOnDemandValue() == null) { 8020 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault()); 8021 } 8022 try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) { 8023 List<Cell> tmp = new ArrayList<>(); 8024 scanner.next(tmp); 8025 // Copy EC to heap, then close the scanner. 8026 // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers. 8027 // See more details in HBASE-26036. 8028 for (Cell cell : tmp) { 8029 results.add(CellUtil.cloneIfNecessary(cell)); 8030 } 8031 } 8032 8033 // post-get CP hook 8034 if (withCoprocessor && (coprocessorHost != null)) { 8035 coprocessorHost.postGet(get, results); 8036 } 8037 8038 metricsUpdateForGet(); 8039 8040 return results; 8041 } 8042 8043 void metricsUpdateForGet() { 8044 if (this.metricsRegion != null) { 8045 this.metricsRegion.updateGet(); 8046 } 8047 if (this.rsServices != null && this.rsServices.getMetrics() != null) { 8048 rsServices.getMetrics().updateReadQueryMeter(this, 1); 8049 } 8050 8051 } 8052 8053 @Override 8054 public Result mutateRow(RowMutations rm) throws IOException { 8055 return mutateRow(rm, HConstants.NO_NONCE, HConstants.NO_NONCE); 8056 } 8057 8058 public Result mutateRow(RowMutations rm, long nonceGroup, long nonce) throws IOException { 8059 final List<Mutation> m = rm.getMutations(); 8060 OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, nonceGroup, nonce); 8061 8062 List<Result> results = new ArrayList<>(); 8063 for (OperationStatus status : statuses) { 8064 if (status.getResult() != null) { 8065 results.add(status.getResult()); 8066 } 8067 } 8068 8069 if (results.isEmpty()) { 8070 return null; 8071 } 8072 8073 // Merge the results of the Increment/Append operations 8074 List<Cell> cells = new ArrayList<>(); 8075 for (Result result : results) { 8076 if (result.rawCells() != null) { 8077 cells.addAll(Arrays.asList(result.rawCells())); 8078 } 8079 } 8080 return Result.create(cells); 8081 } 8082 8083 /** 8084 * Perform atomic (all or none) mutations within the region. 8085 * @param mutations The list of mutations to perform. <code>mutations</code> can contain 8086 * operations for multiple rows. Caller has to ensure that all rows are 8087 * contained in this region. 8088 * @param rowsToLock Rows to lock 8089 * @param nonceGroup Optional nonce group of the operation (client Id) 8090 * @param nonce Optional nonce of the operation (unique random id to ensure "more 8091 * idempotence") If multiple rows are locked care should be taken that 8092 * <code>rowsToLock</code> is sorted in order to avoid deadlocks. 8093 */ 8094 @Override 8095 public void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock, 8096 long nonceGroup, long nonce) throws IOException { 8097 batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]), 8098 true, nonceGroup, nonce) { 8099 @Override 8100 public MiniBatchOperationInProgress<Mutation> 8101 lockRowsAndBuildMiniBatch(List<RowLock> acquiredRowLocks) throws IOException { 8102 RowLock prevRowLock = null; 8103 for (byte[] row : rowsToLock) { 8104 try { 8105 RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock 8106 if (rowLock != prevRowLock) { 8107 acquiredRowLocks.add(rowLock); 8108 prevRowLock = rowLock; 8109 } 8110 } catch (IOException ioe) { 8111 LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this, 8112 ioe); 8113 throw ioe; 8114 } 8115 } 8116 return createMiniBatch(size(), size()); 8117 } 8118 }); 8119 } 8120 8121 /** Returns statistics about the current load of the region */ 8122 public ClientProtos.RegionLoadStats getLoadStatistics() { 8123 if (!regionStatsEnabled) { 8124 return null; 8125 } 8126 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder(); 8127 stats.setMemStoreLoad((int) (Math.min(100, 8128 (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize))); 8129 if (rsServices.getHeapMemoryManager() != null) { 8130 // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM, 8131 // so we could just do the calculation below and we'll get a 0. 8132 // treating it as a special case analogous to no HMM instead so that it can be 8133 // programatically treated different from using <1% of heap. 8134 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent(); 8135 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) { 8136 stats.setHeapOccupancy((int) (occupancy * 100)); 8137 } 8138 } 8139 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 8140 ? 100 8141 : rsServices.getCompactionPressure() * 100)); 8142 return stats.build(); 8143 } 8144 8145 @Override 8146 public Result append(Append append) throws IOException { 8147 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE); 8148 } 8149 8150 public Result append(Append append, long nonceGroup, long nonce) throws IOException { 8151 return TraceUtil.trace(() -> { 8152 checkReadOnly(); 8153 checkResources(); 8154 startRegionOperation(Operation.APPEND); 8155 try { 8156 // All edits for the given row (across all column families) must happen atomically. 8157 return mutate(append, true, nonceGroup, nonce).getResult(); 8158 } finally { 8159 closeRegionOperation(Operation.APPEND); 8160 } 8161 }, () -> createRegionSpan("Region.append")); 8162 } 8163 8164 @Override 8165 public Result increment(Increment increment) throws IOException { 8166 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE); 8167 } 8168 8169 public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException { 8170 return TraceUtil.trace(() -> { 8171 checkReadOnly(); 8172 checkResources(); 8173 startRegionOperation(Operation.INCREMENT); 8174 try { 8175 // All edits for the given row (across all column families) must happen atomically. 8176 return mutate(increment, true, nonceGroup, nonce).getResult(); 8177 } finally { 8178 closeRegionOperation(Operation.INCREMENT); 8179 } 8180 }, () -> createRegionSpan("Region.increment")); 8181 } 8182 8183 private WALKeyImpl createWALKeyForWALAppend(boolean isReplay, BatchOperation<?> batchOp, long now, 8184 long nonceGroup, long nonce) { 8185 WALKeyImpl walKey = isReplay 8186 ? new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(), 8187 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, 8188 batchOp.getClusterIds(), nonceGroup, nonce, mvcc) 8189 : new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(), 8190 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, 8191 batchOp.getClusterIds(), nonceGroup, nonce, mvcc, this.getReplicationScope()); 8192 if (isReplay) { 8193 walKey.setOrigLogSeqNum(batchOp.getOrigLogSeqNum()); 8194 } 8195 return walKey; 8196 } 8197 8198 /** Returns writeEntry associated with this append */ 8199 private WriteEntry doWALAppend(WALEdit walEdit, BatchOperation<?> batchOp, 8200 MiniBatchOperationInProgress<Mutation> miniBatchOp, long now, NonceKey nonceKey) 8201 throws IOException { 8202 Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(), "WALEdit is null or empty!"); 8203 Preconditions.checkArgument( 8204 !walEdit.isReplay() || batchOp.getOrigLogSeqNum() != SequenceId.NO_SEQUENCE_ID, 8205 "Invalid replay sequence Id for replay WALEdit!"); 8206 8207 WALKeyImpl walKey = createWALKeyForWALAppend(walEdit.isReplay(), batchOp, now, 8208 nonceKey.getNonceGroup(), nonceKey.getNonce()); 8209 // don't call the coproc hook for writes to the WAL caused by 8210 // system lifecycle events like flushes or compactions 8211 if (this.coprocessorHost != null && !walEdit.isMetaEdit()) { 8212 this.coprocessorHost.preWALAppend(walKey, walEdit); 8213 } 8214 try { 8215 long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit); 8216 WriteEntry writeEntry = walKey.getWriteEntry(); 8217 // Call sync on our edit. 8218 if (txid != 0) { 8219 sync(txid, batchOp.durability); 8220 } 8221 /** 8222 * If above {@link HRegion#sync} throws Exception, the RegionServer should be aborted and 8223 * following {@link BatchOperation#writeMiniBatchOperationsToMemStore} will not be executed, 8224 * so there is no need to replicate to secondary replica, for this reason here we attach the 8225 * region replication action after the {@link HRegion#sync} is successful. 8226 */ 8227 this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry); 8228 return writeEntry; 8229 } catch (IOException ioe) { 8230 if (walKey.getWriteEntry() != null) { 8231 mvcc.complete(walKey.getWriteEntry()); 8232 } 8233 8234 /** 8235 * If {@link WAL#sync} get a timeout exception, the only correct way is to abort the region 8236 * server, as the design of {@link WAL#sync}, is to succeed or die, there is no 'failure'. It 8237 * is usually not a big deal is because we set a very large default value(5 minutes) for 8238 * {@link AbstractFSWAL#WAL_SYNC_TIMEOUT_MS}, usually the WAL system will abort the region 8239 * server if it can not finish the sync within 5 minutes. 8240 */ 8241 if (ioe instanceof WALSyncTimeoutIOException) { 8242 if (rsServices != null) { 8243 rsServices.abort("WAL sync timeout,forcing server shutdown", ioe); 8244 } 8245 } 8246 throw ioe; 8247 } 8248 } 8249 8250 /** 8251 * Attach {@link RegionReplicationSink#add} to the mvcc writeEntry for replicating to region 8252 * replica. 8253 */ 8254 private void attachRegionReplicationInWALAppend(BatchOperation<?> batchOp, 8255 MiniBatchOperationInProgress<Mutation> miniBatchOp, WALKeyImpl walKey, WALEdit walEdit, 8256 WriteEntry writeEntry) { 8257 if (!regionReplicationSink.isPresent()) { 8258 return; 8259 } 8260 /** 8261 * If {@link HRegion#regionReplicationSink} is present,only {@link MutationBatchOperation} is 8262 * used and {@link NonceKey} is all the same for {@link Mutation}s in 8263 * {@link MutationBatchOperation},so for HBASE-26993 case 1,if 8264 * {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} is not null and we could 8265 * enter {@link HRegion#doWALAppend},that means partial {@link Mutation}s are 8266 * {@link Durability#SKIP_WAL}, we use 8267 * {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} to replicate to region 8268 * replica,but if {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} is 8269 * null,that means there is no {@link Mutation} is {@link Durability#SKIP_WAL},so we just use 8270 * walEdit to replicate. 8271 */ 8272 assert batchOp instanceof MutationBatchOperation; 8273 WALEdit walEditToUse = miniBatchOp.getWalEditForReplicateIfExistsSkipWAL(); 8274 if (walEditToUse == null) { 8275 walEditToUse = walEdit; 8276 } 8277 doAttachReplicateRegionReplicaAction(walKey, walEditToUse, writeEntry); 8278 } 8279 8280 /** 8281 * Attach {@link RegionReplicationSink#add} to the mvcc writeEntry for replicating to region 8282 * replica. 8283 */ 8284 private void doAttachReplicateRegionReplicaAction(WALKeyImpl walKey, WALEdit walEdit, 8285 WriteEntry writeEntry) { 8286 if (walEdit == null || walEdit.isEmpty()) { 8287 return; 8288 } 8289 final ServerCall<?> rpcCall = RpcServer.getCurrentServerCallWithCellScanner().orElse(null); 8290 regionReplicationSink.ifPresent(sink -> writeEntry.attachCompletionAction(() -> { 8291 sink.add(walKey, walEdit, rpcCall); 8292 })); 8293 } 8294 8295 public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false); 8296 8297 // woefully out of date - currently missing: 8298 // 1 x HashMap - coprocessorServiceHandlers 8299 // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL, 8300 // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount, 8301 // writeRequestsCount, cpRequestsCount 8302 // 1 x HRegion$WriteState - writestate 8303 // 1 x RegionCoprocessorHost - coprocessorHost 8304 // 1 x RegionSplitPolicy - splitPolicy 8305 // 1 x MetricsRegion - metricsRegion 8306 // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper 8307 // 1 x ReadPointCalculationLock - smallestReadPointCalcLock 8308 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + ClassSize.OBJECT + // closeLock 8309 (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing 8310 (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL, 8311 // compactionsFailed 8312 (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders 8313 WriteState.HEAP_SIZE + // writestate 8314 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores 8315 (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock 8316 MultiVersionConcurrencyControl.FIXED_SIZE // mvcc 8317 + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes 8318 + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress 8319 + ClassSize.STORE_SERVICES // store services 8320 + StoreHotnessProtector.FIXED_SIZE; 8321 8322 @Override 8323 public long heapSize() { 8324 // this does not take into account row locks, recent flushes, mvcc entries, and more 8325 return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum(); 8326 } 8327 8328 /** 8329 * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be 8330 * available for handling {@link #execService(RpcController, CoprocessorServiceCall)} calls. 8331 * <p/> 8332 * Only a single instance may be registered per region for a given {@link Service} subclass (the 8333 * instances are keyed on {@link ServiceDescriptor#getFullName()}.. After the first registration, 8334 * subsequent calls with the same service name will fail with a return value of {@code false}. 8335 * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint 8336 * @return {@code true} if the registration was successful, {@code false} otherwise 8337 */ 8338 public boolean registerService(Service instance) { 8339 // No stacking of instances is allowed for a single service name 8340 ServiceDescriptor serviceDesc = instance.getDescriptorForType(); 8341 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc); 8342 if (coprocessorServiceHandlers.containsKey(serviceName)) { 8343 LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}", 8344 serviceName, instance, this); 8345 return false; 8346 } 8347 8348 coprocessorServiceHandlers.put(serviceName, instance); 8349 if (LOG.isDebugEnabled()) { 8350 LOG.debug("Registered coprocessor service: region=" 8351 + Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName); 8352 } 8353 return true; 8354 } 8355 8356 /** 8357 * Executes a single protocol buffer coprocessor endpoint {@link Service} method using the 8358 * registered protocol handlers. {@link Service} implementations must be registered via the 8359 * {@link #registerService(Service)} method before they are available. 8360 * @param controller an {@code RpcContoller} implementation to pass to the invoked service 8361 * @param call a {@code CoprocessorServiceCall} instance identifying the service, method, 8362 * and parameters for the method invocation 8363 * @return a protocol buffer {@code Message} instance containing the method's result 8364 * @throws IOException if no registered service handler is found or an error occurs during the 8365 * invocation 8366 * @see #registerService(Service) 8367 */ 8368 public Message execService(RpcController controller, CoprocessorServiceCall call) 8369 throws IOException { 8370 String serviceName = call.getServiceName(); 8371 Service service = coprocessorServiceHandlers.get(serviceName); 8372 if (service == null) { 8373 throw new UnknownProtocolException(null, "No registered coprocessor service found for " 8374 + serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName())); 8375 } 8376 ServiceDescriptor serviceDesc = service.getDescriptorForType(); 8377 8378 cpRequestsCount.increment(); 8379 String methodName = call.getMethodName(); 8380 MethodDescriptor methodDesc = CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc); 8381 8382 Message.Builder builder = service.getRequestPrototype(methodDesc).newBuilderForType(); 8383 8384 ProtobufUtil.mergeFrom(builder, call.getRequest().toByteArray()); 8385 Message request = CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest()); 8386 8387 if (coprocessorHost != null) { 8388 request = coprocessorHost.preEndpointInvocation(service, methodName, request); 8389 } 8390 8391 final Message.Builder responseBuilder = 8392 service.getResponsePrototype(methodDesc).newBuilderForType(); 8393 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() { 8394 @Override 8395 public void run(Message message) { 8396 if (message != null) { 8397 responseBuilder.mergeFrom(message); 8398 } 8399 } 8400 }); 8401 8402 if (coprocessorHost != null) { 8403 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder); 8404 } 8405 IOException exception = 8406 org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller); 8407 if (exception != null) { 8408 throw exception; 8409 } 8410 8411 return responseBuilder.build(); 8412 } 8413 8414 public Optional<byte[]> checkSplit() { 8415 return checkSplit(false); 8416 } 8417 8418 /** 8419 * Return the split point. An empty result indicates the region isn't splittable. 8420 */ 8421 public Optional<byte[]> checkSplit(boolean force) { 8422 // Can't split META 8423 if (this.getRegionInfo().isMetaRegion()) { 8424 return Optional.empty(); 8425 } 8426 8427 // Can't split a region that is closing. 8428 if (this.isClosing()) { 8429 return Optional.empty(); 8430 } 8431 8432 if (!force && !splitPolicy.shouldSplit()) { 8433 return Optional.empty(); 8434 } 8435 8436 byte[] ret = splitPolicy.getSplitPoint(); 8437 if (ret != null && ret.length > 0) { 8438 ret = splitRestriction.getRestrictedSplitPoint(ret); 8439 } 8440 8441 if (ret != null) { 8442 try { 8443 checkRow(ret, "calculated split"); 8444 } catch (IOException e) { 8445 LOG.error("Ignoring invalid split for region {}", this, e); 8446 return Optional.empty(); 8447 } 8448 return Optional.of(ret); 8449 } else { 8450 return Optional.empty(); 8451 } 8452 } 8453 8454 /** Returns The priority that this region should have in the compaction queue */ 8455 public int getCompactPriority() { 8456 if (checkSplit().isPresent() && conf.getBoolean(SPLIT_IGNORE_BLOCKING_ENABLED_KEY, false)) { 8457 // if a region should split, split it before compact 8458 return Store.PRIORITY_USER; 8459 } 8460 return stores.values().stream().mapToInt(HStore::getCompactPriority).min() 8461 .orElse(Store.NO_PRIORITY); 8462 } 8463 8464 /** Returns the coprocessor host */ 8465 public RegionCoprocessorHost getCoprocessorHost() { 8466 return coprocessorHost; 8467 } 8468 8469 /** @param coprocessorHost the new coprocessor host */ 8470 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) { 8471 this.coprocessorHost = coprocessorHost; 8472 } 8473 8474 @Override 8475 public void startRegionOperation() throws IOException { 8476 startRegionOperation(Operation.ANY); 8477 } 8478 8479 @Override 8480 public void startRegionOperation(Operation op) throws IOException { 8481 boolean isInterruptableOp = false; 8482 switch (op) { 8483 case GET: // interruptible read operations 8484 case SCAN: 8485 isInterruptableOp = true; 8486 checkReadsEnabled(); 8487 break; 8488 case INCREMENT: // interruptible write operations 8489 case APPEND: 8490 case PUT: 8491 case DELETE: 8492 case BATCH_MUTATE: 8493 case CHECK_AND_MUTATE: 8494 isInterruptableOp = true; 8495 break; 8496 default: // all others 8497 break; 8498 } 8499 if ( 8500 op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION || op == Operation.COMPACT_REGION 8501 || op == Operation.COMPACT_SWITCH 8502 ) { 8503 // split, merge or compact region doesn't need to check the closing/closed state or lock the 8504 // region 8505 return; 8506 } 8507 if (this.closing.get()) { 8508 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); 8509 } 8510 lock(lock.readLock()); 8511 // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from 8512 // an RPC handler 8513 Thread thisThread = Thread.currentThread(); 8514 if (isInterruptableOp) { 8515 regionLockHolders.put(thisThread, true); 8516 } 8517 if (this.closed.get()) { 8518 lock.readLock().unlock(); 8519 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); 8520 } 8521 // The unit for snapshot is a region. So, all stores for this region must be 8522 // prepared for snapshot operation before proceeding. 8523 if (op == Operation.SNAPSHOT) { 8524 stores.values().forEach(HStore::preSnapshotOperation); 8525 } 8526 try { 8527 if (coprocessorHost != null) { 8528 coprocessorHost.postStartRegionOperation(op); 8529 } 8530 } catch (Exception e) { 8531 if (isInterruptableOp) { 8532 // would be harmless to remove what we didn't add but we know by 'isInterruptableOp' 8533 // if we added this thread to regionLockHolders 8534 regionLockHolders.remove(thisThread); 8535 } 8536 lock.readLock().unlock(); 8537 throw new IOException(e); 8538 } 8539 } 8540 8541 @Override 8542 public void closeRegionOperation() throws IOException { 8543 closeRegionOperation(Operation.ANY); 8544 } 8545 8546 @Override 8547 public void closeRegionOperation(Operation operation) throws IOException { 8548 if (operation == Operation.SNAPSHOT) { 8549 stores.values().forEach(HStore::postSnapshotOperation); 8550 } 8551 Thread thisThread = Thread.currentThread(); 8552 regionLockHolders.remove(thisThread); 8553 lock.readLock().unlock(); 8554 if (coprocessorHost != null) { 8555 coprocessorHost.postCloseRegionOperation(operation); 8556 } 8557 } 8558 8559 /** 8560 * This method needs to be called before any public call that reads or modifies stores in bulk. It 8561 * has to be called just before a try. #closeBulkRegionOperation needs to be called in the try's 8562 * finally block Acquires a writelock and checks if the region is closing or closed. 8563 * @throws NotServingRegionException when the region is closing or closed 8564 * @throws RegionTooBusyException if failed to get the lock in time 8565 * @throws InterruptedIOException if interrupted while waiting for a lock 8566 */ 8567 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException { 8568 if (this.closing.get()) { 8569 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing"); 8570 } 8571 if (writeLockNeeded) lock(lock.writeLock()); 8572 else lock(lock.readLock()); 8573 if (this.closed.get()) { 8574 if (writeLockNeeded) lock.writeLock().unlock(); 8575 else lock.readLock().unlock(); 8576 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed"); 8577 } 8578 regionLockHolders.put(Thread.currentThread(), true); 8579 } 8580 8581 /** 8582 * Closes the lock. This needs to be called in the finally block corresponding to the try block of 8583 * #startRegionOperation 8584 */ 8585 private void closeBulkRegionOperation() { 8586 regionLockHolders.remove(Thread.currentThread()); 8587 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock(); 8588 else lock.readLock().unlock(); 8589 } 8590 8591 /** 8592 * Update LongAdders for number of puts without wal and the size of possible data loss. These 8593 * information are exposed by the region server metrics. 8594 */ 8595 private void recordMutationWithoutWal(final Map<byte[], List<Cell>> familyMap) { 8596 numMutationsWithoutWAL.increment(); 8597 if (numMutationsWithoutWAL.sum() <= 1) { 8598 LOG.info("writing data to region " + this 8599 + " with WAL disabled. Data may be lost in the event of a crash."); 8600 } 8601 8602 long mutationSize = 0; 8603 for (List<Cell> cells : familyMap.values()) { 8604 // Optimization: 'foreach' loop is not used. See: 8605 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects 8606 assert cells instanceof RandomAccess; 8607 int listSize = cells.size(); 8608 for (int i = 0; i < listSize; i++) { 8609 Cell cell = cells.get(i); 8610 mutationSize += cell.getSerializedSize(); 8611 } 8612 } 8613 8614 dataInMemoryWithoutWAL.add(mutationSize); 8615 } 8616 8617 private void lock(final Lock lock) throws IOException { 8618 lock(lock, 1); 8619 } 8620 8621 /** 8622 * Try to acquire a lock. Throw RegionTooBusyException if failed to get the lock in time. Throw 8623 * InterruptedIOException if interrupted while waiting for the lock. 8624 */ 8625 private void lock(final Lock lock, final int multiplier) throws IOException { 8626 try { 8627 final long waitTime = Math.min(maxBusyWaitDuration, 8628 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier)); 8629 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) { 8630 // Don't print millis. Message is used as a key over in 8631 // RetriesExhaustedWithDetailsException processing. 8632 final String regionName = 8633 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString(); 8634 final String serverName = this.getRegionServerServices() == null 8635 ? "unknown" 8636 : (this.getRegionServerServices().getServerName() == null 8637 ? "unknown" 8638 : this.getRegionServerServices().getServerName().toString()); 8639 RegionTooBusyException rtbe = new RegionTooBusyException( 8640 "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName); 8641 LOG.warn("Region is too busy to allow lock acquisition.", rtbe); 8642 throw rtbe; 8643 } 8644 } catch (InterruptedException ie) { 8645 if (LOG.isDebugEnabled()) { 8646 LOG.debug("Interrupted while waiting for a lock in region {}", this); 8647 } 8648 throw throwOnInterrupt(ie); 8649 } 8650 } 8651 8652 /** 8653 * Calls sync with the given transaction ID 8654 * @param txid should sync up to which transaction 8655 * @throws IOException If anything goes wrong with DFS 8656 */ 8657 private void sync(long txid, Durability durability) throws IOException { 8658 if (this.getRegionInfo().isMetaRegion()) { 8659 this.wal.sync(txid); 8660 } else { 8661 switch (durability) { 8662 case USE_DEFAULT: 8663 // do what table defaults to 8664 if (shouldSyncWAL()) { 8665 this.wal.sync(txid); 8666 } 8667 break; 8668 case SKIP_WAL: 8669 // nothing do to 8670 break; 8671 case ASYNC_WAL: 8672 // nothing do to 8673 break; 8674 case SYNC_WAL: 8675 this.wal.sync(txid, false); 8676 break; 8677 case FSYNC_WAL: 8678 this.wal.sync(txid, true); 8679 break; 8680 default: 8681 throw new RuntimeException("Unknown durability " + durability); 8682 } 8683 } 8684 } 8685 8686 /** 8687 * Check whether we should sync the wal from the table's durability settings 8688 */ 8689 private boolean shouldSyncWAL() { 8690 return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal(); 8691 } 8692 8693 /** Returns the latest sequence number that was read from storage when this region was opened */ 8694 public long getOpenSeqNum() { 8695 return this.openSeqNum; 8696 } 8697 8698 @Override 8699 public Map<byte[], Long> getMaxStoreSeqId() { 8700 return this.maxSeqIdInStores; 8701 } 8702 8703 public long getOldestSeqIdOfStore(byte[] familyName) { 8704 return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName); 8705 } 8706 8707 @Override 8708 public CompactionState getCompactionState() { 8709 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0; 8710 return (hasMajor 8711 ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR) 8712 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE)); 8713 } 8714 8715 public void reportCompactionRequestStart(boolean isMajor) { 8716 (isMajor ? majorInProgress : minorInProgress).incrementAndGet(); 8717 } 8718 8719 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) { 8720 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet(); 8721 8722 // metrics 8723 compactionsFinished.increment(); 8724 compactionNumFilesCompacted.add(numFiles); 8725 compactionNumBytesCompacted.add(filesSizeCompacted); 8726 8727 assert newValue >= 0; 8728 } 8729 8730 public void reportCompactionRequestFailure() { 8731 compactionsFailed.increment(); 8732 } 8733 8734 public void incrementCompactionsQueuedCount() { 8735 compactionsQueued.increment(); 8736 } 8737 8738 public void decrementCompactionsQueuedCount() { 8739 compactionsQueued.decrement(); 8740 } 8741 8742 public void incrementFlushesQueuedCount() { 8743 flushesQueued.increment(); 8744 } 8745 8746 protected void decrementFlushesQueuedCount() { 8747 flushesQueued.decrement(); 8748 } 8749 8750 /** 8751 * If a handler thread is eligible for interrupt, make it ineligible. Should be paired with 8752 * {{@link #enableInterrupts()}. 8753 */ 8754 void disableInterrupts() { 8755 regionLockHolders.computeIfPresent(Thread.currentThread(), (t, b) -> false); 8756 } 8757 8758 /** 8759 * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()}, make 8760 * it eligible again. No-op if interrupts are already enabled. 8761 */ 8762 void enableInterrupts() { 8763 regionLockHolders.computeIfPresent(Thread.currentThread(), (t, b) -> true); 8764 } 8765 8766 /** 8767 * Interrupt any region options that have acquired the region lock via 8768 * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)}, or 8769 * {@link #startBulkRegionOperation(boolean)}. 8770 */ 8771 private void interruptRegionOperations() { 8772 for (Map.Entry<Thread, Boolean> entry : regionLockHolders.entrySet()) { 8773 // An entry in this map will have a boolean value indicating if it is currently 8774 // eligible for interrupt; if so, we should interrupt it. 8775 if (entry.getValue().booleanValue()) { 8776 entry.getKey().interrupt(); 8777 } 8778 } 8779 } 8780 8781 /** 8782 * Check thread interrupt status and throw an exception if interrupted. 8783 * @throws NotServingRegionException if region is closing 8784 * @throws InterruptedIOException if interrupted but region is not closing 8785 */ 8786 // Package scope for tests 8787 void checkInterrupt() throws NotServingRegionException, InterruptedIOException { 8788 if (Thread.interrupted()) { 8789 if (this.closing.get()) { 8790 throw new NotServingRegionException( 8791 getRegionInfo().getRegionNameAsString() + " is closing"); 8792 } 8793 throw new InterruptedIOException(); 8794 } 8795 } 8796 8797 /** 8798 * Throw the correct exception upon interrupt 8799 * @param t cause 8800 */ 8801 // Package scope for tests 8802 IOException throwOnInterrupt(Throwable t) { 8803 if (this.closing.get()) { 8804 return (NotServingRegionException) new NotServingRegionException( 8805 getRegionInfo().getRegionNameAsString() + " is closing").initCause(t); 8806 } 8807 return (InterruptedIOException) new InterruptedIOException().initCause(t); 8808 } 8809 8810 /** 8811 * {@inheritDoc} 8812 */ 8813 @Override 8814 public void onConfigurationChange(Configuration conf) { 8815 this.storeHotnessProtector.update(conf); 8816 // update coprocessorHost if the configuration has changed. 8817 if ( 8818 CoprocessorConfigurationUtil.checkConfigurationChange(getReadOnlyConfiguration(), conf, 8819 CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, 8820 CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY) 8821 ) { 8822 LOG.info("Update the system coprocessors because the configuration has changed"); 8823 decorateRegionConfiguration(conf); 8824 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf); 8825 } 8826 } 8827 8828 /** 8829 * {@inheritDoc} 8830 */ 8831 @Override 8832 public void registerChildren(ConfigurationManager manager) { 8833 configurationManager = manager; 8834 stores.values().forEach(manager::registerObserver); 8835 } 8836 8837 /** 8838 * {@inheritDoc} 8839 */ 8840 @Override 8841 public void deregisterChildren(ConfigurationManager manager) { 8842 stores.values().forEach(configurationManager::deregisterObserver); 8843 } 8844 8845 @Override 8846 public CellComparator getCellComparator() { 8847 return cellComparator; 8848 } 8849 8850 public long getMemStoreFlushSize() { 8851 return this.memstoreFlushSize; 8852 } 8853 8854 //// method for debugging tests 8855 void throwException(String title, String regionName) { 8856 StringBuilder buf = new StringBuilder(); 8857 buf.append(title + ", "); 8858 buf.append(getRegionInfo().toString()); 8859 buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " "); 8860 buf.append("stores: "); 8861 for (HStore s : stores.values()) { 8862 buf.append(s.getColumnFamilyDescriptor().getNameAsString()); 8863 buf.append(" size: "); 8864 buf.append(s.getMemStoreSize().getDataSize()); 8865 buf.append(" "); 8866 } 8867 buf.append("end-of-stores"); 8868 buf.append(", memstore size "); 8869 buf.append(getMemStoreDataSize()); 8870 if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) { 8871 throw new RuntimeException(buf.toString()); 8872 } 8873 } 8874 8875 @Override 8876 public void requestCompaction(String why, int priority, boolean major, 8877 CompactionLifeCycleTracker tracker) throws IOException { 8878 if (major) { 8879 stores.values().forEach(HStore::triggerMajorCompaction); 8880 } 8881 rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker, 8882 RpcServer.getRequestUser().orElse(null)); 8883 } 8884 8885 @Override 8886 public void requestCompaction(byte[] family, String why, int priority, boolean major, 8887 CompactionLifeCycleTracker tracker) throws IOException { 8888 HStore store = stores.get(family); 8889 if (store == null) { 8890 throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family) 8891 + " does not exist in region " + getRegionInfo().getRegionNameAsString()); 8892 } 8893 if (major) { 8894 store.triggerMajorCompaction(); 8895 } 8896 rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker, 8897 RpcServer.getRequestUser().orElse(null)); 8898 } 8899 8900 private void requestFlushIfNeeded() throws RegionTooBusyException { 8901 if (isFlushSize(this.memStoreSizing.getMemStoreSize())) { 8902 requestFlush(); 8903 } 8904 } 8905 8906 private void requestFlush() { 8907 if (this.rsServices == null) { 8908 return; 8909 } 8910 requestFlush0(FlushLifeCycleTracker.DUMMY); 8911 } 8912 8913 private void requestFlush0(FlushLifeCycleTracker tracker) { 8914 boolean shouldFlush = false; 8915 synchronized (writestate) { 8916 if (!this.writestate.isFlushRequested()) { 8917 shouldFlush = true; 8918 writestate.flushRequested = true; 8919 } 8920 } 8921 if (shouldFlush) { 8922 // Make request outside of synchronize block; HBASE-818. 8923 this.rsServices.getFlushRequester().requestFlush(this, tracker); 8924 if (LOG.isDebugEnabled()) { 8925 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName()); 8926 } 8927 } else { 8928 tracker.notExecuted("Flush already requested on " + this); 8929 } 8930 } 8931 8932 @Override 8933 public void requestFlush(FlushLifeCycleTracker tracker) throws IOException { 8934 requestFlush0(tracker); 8935 } 8936 8937 /** 8938 * This method modifies the region's configuration in order to inject replication-related features 8939 * @param conf region configurations 8940 */ 8941 private static void decorateRegionConfiguration(Configuration conf) { 8942 if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) { 8943 String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, ""); 8944 String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName(); 8945 if (!plugins.contains(replicationCoprocessorClass)) { 8946 conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, 8947 (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass); 8948 } 8949 } 8950 } 8951 8952 public Optional<RegionReplicationSink> getRegionReplicationSink() { 8953 return regionReplicationSink; 8954 } 8955 8956 public void addReadRequestsCount(long readRequestsCount) { 8957 this.readRequestsCount.add(readRequestsCount); 8958 } 8959 8960 public void addWriteRequestsCount(long writeRequestsCount) { 8961 this.writeRequestsCount.add(writeRequestsCount); 8962 } 8963 8964 @RestrictedApi(explanation = "Should only be called in tests", link = "", 8965 allowedOnPath = ".*/src/test/.*") 8966 boolean isReadsEnabled() { 8967 return this.writestate.readsEnabled; 8968 } 8969}