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