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