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