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