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