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