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