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