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