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