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