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