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