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   * Do preparation for pending compaction.
2253   */
2254  protected void doRegionCompactionPrep() throws IOException {
2255  }
2256
2257  /**
2258   * Synchronously compact all stores in the region.
2259   * <p>
2260   * This operation could block for a long time, so don't call it from a time-sensitive thread.
2261   * <p>
2262   * Note that no locks are taken to prevent possible conflicts between compaction and splitting
2263   * activities. The regionserver does not normally compact and split in parallel. However by
2264   * calling this method you may introduce unexpected and unhandled concurrency. Don't do this
2265   * unless you know what you are doing.
2266   * @param majorCompaction True to force a major compaction regardless of thresholds
2267   */
2268  public void compact(boolean majorCompaction) throws IOException {
2269    if (majorCompaction) {
2270      stores.values().forEach(HStore::triggerMajorCompaction);
2271    }
2272    for (HStore s : stores.values()) {
2273      Optional<CompactionContext> compaction = s.requestCompaction();
2274      if (compaction.isPresent()) {
2275        ThroughputController controller = null;
2276        if (rsServices != null) {
2277          controller = CompactionThroughputControllerFactory.create(rsServices, conf);
2278        }
2279        if (controller == null) {
2280          controller = NoLimitThroughputController.INSTANCE;
2281        }
2282        compact(compaction.get(), s, controller, null);
2283      }
2284    }
2285  }
2286
2287  /**
2288   * This is a helper function that compact all the stores synchronously.
2289   * <p>
2290   * It is used by utilities and testing
2291   */
2292  public void compactStores() throws IOException {
2293    for (HStore s : stores.values()) {
2294      Optional<CompactionContext> compaction = s.requestCompaction();
2295      if (compaction.isPresent()) {
2296        compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null);
2297      }
2298    }
2299  }
2300
2301  /**
2302   * This is a helper function that compact the given store.
2303   * <p>
2304   * It is used by utilities and testing
2305   */
2306  void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
2307    HStore s = getStore(family);
2308    Optional<CompactionContext> compaction = s.requestCompaction();
2309    if (compaction.isPresent()) {
2310      compact(compaction.get(), s, throughputController, null);
2311    }
2312  }
2313
2314  /**
2315   * Called by compaction thread and after region is opened to compact the HStores if necessary.
2316   * <p>
2317   * This operation could block for a long time, so don't call it from a time-sensitive thread. Note
2318   * that no locking is necessary at this level because compaction only conflicts with a region
2319   * split, and that cannot happen because the region server does them sequentially and not in
2320   * parallel.
2321   * @param compaction Compaction details, obtained by requestCompaction()
2322   * @return whether the compaction completed
2323   */
2324  public boolean compact(CompactionContext compaction, HStore store,
2325    ThroughputController throughputController) throws IOException {
2326    return compact(compaction, store, throughputController, null);
2327  }
2328
2329  private boolean shouldForbidMajorCompaction() {
2330    if (rsServices != null && rsServices.getReplicationSourceService() != null) {
2331      return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
2332        .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get());
2333    }
2334    return false;
2335  }
2336
2337  /**
2338   * <p>
2339   * We are trying to remove / relax the region read lock for compaction. Let's see what are the
2340   * potential race conditions among the operations (user scan, region split, region close and
2341   * region bulk load).
2342   * </p>
2343   *
2344   * <pre>
2345   *   user scan ---> region read lock
2346   *   region split --> region close first --> region write lock
2347   *   region close --> region write lock
2348   *   region bulk load --> region write lock
2349   * </pre>
2350   * <p>
2351   * read lock is compatible with read lock. ---> no problem with user scan/read region bulk load
2352   * does not cause problem for compaction (no consistency problem, store lock will help the store
2353   * file accounting). They can run almost concurrently at the region level.
2354   * </p>
2355   * <p>
2356   * The only remaining race condition is between the region close and compaction. So we will
2357   * evaluate, below, how region close intervenes with compaction if compaction does not acquire
2358   * region read lock.
2359   * </p>
2360   * <p>
2361   * Here are the steps for compaction:
2362   * <ol>
2363   * <li>obtain list of StoreFile's</li>
2364   * <li>create StoreFileScanner's based on list from #1</li>
2365   * <li>perform compaction and save resulting files under tmp dir</li>
2366   * <li>swap in compacted files</li>
2367   * </ol>
2368   * </p>
2369   * <p>
2370   * #1 is guarded by store lock. This patch does not change this --> no worse or better For #2, we
2371   * obtain smallest read point (for region) across all the Scanners (for both default compactor and
2372   * stripe compactor). The read points are for user scans. Region keeps the read points for all
2373   * currently open user scanners. Compaction needs to know the smallest read point so that during
2374   * re-write of the hfiles, it can remove the mvcc points for the cells if their mvccs are older
2375   * than the smallest since they are not needed anymore. This will not conflict with compaction.
2376   * </p>
2377   * <p>
2378   * For #3, it can be performed in parallel to other operations.
2379   * </p>
2380   * <p>
2381   * For #4 bulk load and compaction don't conflict with each other on the region level (for
2382   * multi-family atomicy).
2383   * </p>
2384   * <p>
2385   * Region close and compaction are guarded pretty well by the 'writestate'. In HRegion#doClose(),
2386   * we have :
2387   *
2388   * <pre>
2389   * synchronized (writestate) {
2390   *   // Disable compacting and flushing by background threads for this
2391   *   // region.
2392   *   canFlush = !writestate.readOnly;
2393   *   writestate.writesEnabled = false;
2394   *   LOG.debug("Closing " + this + ": disabling compactions & flushes");
2395   *   waitForFlushesAndCompactions();
2396   * }
2397   * </pre>
2398   *
2399   * {@code waitForFlushesAndCompactions()} would wait for {@code writestate.compacting} to come
2400   * down to 0. and in {@code HRegion.compact()}
2401   *
2402   * <pre>
2403   *   try {
2404   *     synchronized (writestate) {
2405   *       if (writestate.writesEnabled) {
2406   *         wasStateSet = true;
2407   *         ++writestate.compacting;
2408   *       } else {
2409   *         String msg = "NOT compacting region " + this + ". Writes disabled.";
2410   *         LOG.info(msg);
2411   *         status.abort(msg);
2412   *         return false;
2413   *       }
2414   *     }
2415   *   }
2416   * </pre>
2417   *
2418   * Also in {@code compactor.performCompaction()}: check periodically to see if a system stop is
2419   * requested
2420   *
2421   * <pre>
2422   * if (closeChecker != null && closeChecker.isTimeLimit(store, now)) {
2423   *   progress.cancel();
2424   *   return false;
2425   * }
2426   * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
2427   *   progress.cancel();
2428   *   return false;
2429   * }
2430   * </pre>
2431   * </p>
2432   */
2433  public boolean compact(CompactionContext compaction, HStore store,
2434    ThroughputController throughputController, User user) throws IOException {
2435    assert compaction != null && compaction.hasSelection();
2436    assert !compaction.getRequest().getFiles().isEmpty();
2437    if (this.closing.get() || this.closed.get()) {
2438      LOG.debug("Skipping compaction on " + this + " because closing/closed");
2439      store.cancelRequestedCompaction(compaction);
2440      return false;
2441    }
2442
2443    if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
2444      LOG.warn("Skipping major compaction on " + this
2445        + " because this cluster is transiting sync replication state"
2446        + " from STANDBY to DOWNGRADE_ACTIVE");
2447      store.cancelRequestedCompaction(compaction);
2448      return false;
2449    }
2450
2451    MonitoredTask status = null;
2452    boolean requestNeedsCancellation = true;
2453    try {
2454      byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
2455      if (stores.get(cf) != store) {
2456        LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
2457          + " has been re-instantiated, cancel this compaction request. "
2458          + " It may be caused by the roll back of split transaction");
2459        return false;
2460      }
2461
2462      status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
2463      if (this.closed.get()) {
2464        String msg = "Skipping compaction on " + this + " because closed";
2465        LOG.debug(msg);
2466        status.abort(msg);
2467        return false;
2468      }
2469      boolean wasStateSet = false;
2470      try {
2471        synchronized (writestate) {
2472          if (writestate.writesEnabled) {
2473            wasStateSet = true;
2474            writestate.compacting.incrementAndGet();
2475          } else {
2476            String msg = "NOT compacting region " + this + ". Writes disabled.";
2477            LOG.info(msg);
2478            status.abort(msg);
2479            return false;
2480          }
2481        }
2482        LOG.info("Starting compaction of {} in {}{}", store, this,
2483          (compaction.getRequest().isOffPeak() ? " as an off-peak compaction" : ""));
2484        doRegionCompactionPrep();
2485        try {
2486          status.setStatus("Compacting store " + store);
2487          // We no longer need to cancel the request on the way out of this
2488          // method because Store#compact will clean up unconditionally
2489          requestNeedsCancellation = false;
2490          store.compact(compaction, throughputController, user);
2491        } catch (InterruptedIOException iioe) {
2492          String msg = "region " + this + " compaction interrupted";
2493          LOG.info(msg, iioe);
2494          status.abort(msg);
2495          return false;
2496        }
2497      } finally {
2498        if (wasStateSet) {
2499          synchronized (writestate) {
2500            writestate.compacting.decrementAndGet();
2501            if (writestate.compacting.get() <= 0) {
2502              writestate.notifyAll();
2503            }
2504          }
2505        }
2506      }
2507      status.markComplete("Compaction complete");
2508      return true;
2509    } finally {
2510      if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
2511      if (status != null) {
2512        LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2513          status.prettyPrintJournal());
2514        status.cleanup();
2515      }
2516    }
2517  }
2518
2519  /**
2520   * Flush the cache.
2521   * <p>
2522   * When this method is called the cache will be flushed unless:
2523   * <ol>
2524   * <li>the cache is empty</li>
2525   * <li>the region is closed.</li>
2526   * <li>a flush is already in progress</li>
2527   * <li>writes are disabled</li>
2528   * </ol>
2529   * <p>
2530   * This method may block for some time, so it should not be called from a time-sensitive thread.
2531   * @param flushAllStores whether we want to force a flush of all stores
2532   * @return FlushResult indicating whether the flush was successful or not and if the region needs
2533   *         compacting
2534   * @throws IOException general io exceptions because a snapshot was not properly persisted.
2535   */
2536  // TODO HBASE-18905. We might have to expose a requestFlush API for CPs
2537  public FlushResult flush(boolean flushAllStores) throws IOException {
2538    return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY);
2539  }
2540
2541  public interface FlushResult {
2542    enum Result {
2543      FLUSHED_NO_COMPACTION_NEEDED,
2544      FLUSHED_COMPACTION_NEEDED,
2545      // Special case where a flush didn't run because there's nothing in the memstores. Used when
2546      // bulk loading to know when we can still load even if a flush didn't happen.
2547      CANNOT_FLUSH_MEMSTORE_EMPTY,
2548      CANNOT_FLUSH
2549    }
2550
2551    /** Returns the detailed result code */
2552    Result getResult();
2553
2554    /** Returns true if the memstores were flushed, else false */
2555    boolean isFlushSucceeded();
2556
2557    /** Returns True if the flush requested a compaction, else false */
2558    boolean isCompactionNeeded();
2559  }
2560
2561  public FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker,
2562    FlushLifeCycleTracker tracker) throws IOException {
2563    List<byte[]> families = null;
2564    if (flushAllStores) {
2565      families = new ArrayList<>();
2566      families.addAll(this.getTableDescriptor().getColumnFamilyNames());
2567    }
2568    return this.flushcache(families, writeFlushRequestWalMarker, tracker);
2569  }
2570
2571  /**
2572   * Flush the cache. When this method is called the cache will be flushed unless:
2573   * <ol>
2574   * <li>the cache is empty</li>
2575   * <li>the region is closed.</li>
2576   * <li>a flush is already in progress</li>
2577   * <li>writes are disabled</li>
2578   * </ol>
2579   * <p>
2580   * This method may block for some time, so it should not be called from a time-sensitive thread.
2581   * @param families                   stores of region to flush.
2582   * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL
2583   * @param tracker                    used to track the life cycle of this flush
2584   * @return whether the flush is success and whether the region needs compacting
2585   * @throws IOException              general io exceptions
2586   * @throws DroppedSnapshotException Thrown when replay of wal is required because a Snapshot was
2587   *                                  not properly persisted. The region is put in closing mode, and
2588   *                                  the caller MUST abort after this.
2589   */
2590  public FlushResultImpl flushcache(List<byte[]> families, boolean writeFlushRequestWalMarker,
2591    FlushLifeCycleTracker tracker) throws IOException {
2592    // fail-fast instead of waiting on the lock
2593    if (this.closing.get()) {
2594      String msg = "Skipping flush on " + this + " because closing";
2595      LOG.debug(msg);
2596      return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2597    }
2598    MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
2599    status.setStatus("Acquiring readlock on region");
2600    // block waiting for the lock for flushing cache
2601    lock.readLock().lock();
2602    boolean flushed = true;
2603    try {
2604      if (this.closed.get()) {
2605        String msg = "Skipping flush on " + this + " because closed";
2606        LOG.debug(msg);
2607        status.abort(msg);
2608        flushed = false;
2609        return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2610      }
2611      if (coprocessorHost != null) {
2612        status.setStatus("Running coprocessor pre-flush hooks");
2613        coprocessorHost.preFlush(tracker);
2614      }
2615      // TODO: this should be managed within memstore with the snapshot, updated only after flush
2616      // successful
2617      if (numMutationsWithoutWAL.sum() > 0) {
2618        numMutationsWithoutWAL.reset();
2619        dataInMemoryWithoutWAL.reset();
2620      }
2621      synchronized (writestate) {
2622        if (!writestate.flushing && writestate.writesEnabled) {
2623          this.writestate.flushing = true;
2624        } else {
2625          String msg = "NOT flushing " + this + " as "
2626            + (writestate.flushing ? "already flushing" : "writes are not enabled");
2627          LOG.debug(msg);
2628          status.abort(msg);
2629          flushed = false;
2630          return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2631        }
2632      }
2633
2634      try {
2635        // The reason that we do not always use flushPolicy is, when the flush is
2636        // caused by logRoller, we should select stores which must be flushed
2637        // rather than could be flushed.
2638        Collection<HStore> specificStoresToFlush = null;
2639        if (families != null) {
2640          specificStoresToFlush = getSpecificStores(families);
2641        } else {
2642          specificStoresToFlush = flushPolicy.selectStoresToFlush();
2643        }
2644        FlushResultImpl fs =
2645          internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker);
2646
2647        if (coprocessorHost != null) {
2648          status.setStatus("Running post-flush coprocessor hooks");
2649          coprocessorHost.postFlush(tracker);
2650        }
2651
2652        if (fs.isFlushSucceeded()) {
2653          flushesQueued.reset();
2654        }
2655
2656        status.markComplete("Flush successful " + fs.toString());
2657        return fs;
2658      } finally {
2659        synchronized (writestate) {
2660          writestate.flushing = false;
2661          this.writestate.flushRequested = false;
2662          writestate.notifyAll();
2663        }
2664      }
2665    } finally {
2666      lock.readLock().unlock();
2667      if (flushed) {
2668        // Don't log this journal stuff if no flush -- confusing.
2669        LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2670          status.prettyPrintJournal());
2671      }
2672      status.cleanup();
2673    }
2674  }
2675
2676  /**
2677   * get stores which matches the specified families
2678   * @return the stores need to be flushed.
2679   */
2680  private Collection<HStore> getSpecificStores(List<byte[]> families) {
2681    Collection<HStore> specificStoresToFlush = new ArrayList<>();
2682    for (byte[] family : families) {
2683      specificStoresToFlush.add(stores.get(family));
2684    }
2685    return specificStoresToFlush;
2686  }
2687
2688  /**
2689   * Should the store be flushed because it is old enough.
2690   * <p>
2691   * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
2692   * that you always flush all stores). Otherwise the method will always returns true which will
2693   * make a lot of flush requests.
2694   */
2695  boolean shouldFlushStore(HStore store) {
2696    long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
2697      store.getColumnFamilyDescriptor().getName()) - 1;
2698    if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
2699      if (LOG.isDebugEnabled()) {
2700        LOG.debug("Flush column family " + store.getColumnFamilyName() + " of "
2701          + getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest
2702          + " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
2703      }
2704      return true;
2705    }
2706    if (this.flushCheckInterval <= 0) {
2707      return false;
2708    }
2709    long now = EnvironmentEdgeManager.currentTime();
2710    if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2711      if (LOG.isDebugEnabled()) {
2712        LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of "
2713          + getRegionInfo().getEncodedName() + " because time of oldest edit="
2714          + store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2715      }
2716      return true;
2717    }
2718    return false;
2719  }
2720
2721  /**
2722   * Should the memstore be flushed now
2723   */
2724  boolean shouldFlush(final StringBuilder whyFlush) {
2725    whyFlush.setLength(0);
2726    // This is a rough measure.
2727    if (
2728      this.maxFlushedSeqId > 0
2729        && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())
2730    ) {
2731      whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2732      return true;
2733    }
2734    long modifiedFlushCheckInterval = flushCheckInterval;
2735    if (
2736      getRegionInfo().getTable().isSystemTable()
2737        && getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID
2738    ) {
2739      modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2740    }
2741    if (modifiedFlushCheckInterval <= 0) { // disabled
2742      return false;
2743    }
2744    long now = EnvironmentEdgeManager.currentTime();
2745    // if we flushed in the recent past, we don't need to do again now
2746    if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2747      return false;
2748    }
2749    // since we didn't flush in the recent past, flush now if certain conditions
2750    // are met. Return true on first such memstore hit.
2751    for (HStore s : stores.values()) {
2752      if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2753        // we have an old enough edit in the memstore, flush
2754        whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2755        return true;
2756      }
2757    }
2758    return false;
2759  }
2760
2761  /**
2762   * Flushing all stores.
2763   * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2764   */
2765  private FlushResult internalFlushcache(MonitoredTask status) throws IOException {
2766    return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
2767  }
2768
2769  /**
2770   * Flushing given stores.
2771   * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2772   */
2773  private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status,
2774    boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2775    return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status,
2776      writeFlushWalMarker, tracker);
2777  }
2778
2779  /**
2780   * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the
2781   * memstore, all of which have also been written to the wal. We need to write those updates in the
2782   * memstore out to disk, while being able to process reads/writes as much as possible during the
2783   * flush operation.
2784   * <p>
2785   * This method may block for some time. Every time you call it, we up the regions sequence id even
2786   * if we don't flush; i.e. the returned region id will be at least one larger than the last edit
2787   * applied to this region. The returned id does not refer to an actual edit. The returned id can
2788   * be used for say installing a bulk loaded file just ahead of the last hfile that was the result
2789   * of this flush, etc.
2790   * @param wal           Null if we're NOT to go via wal.
2791   * @param myseqid       The seqid to use if <code>wal</code> is null writing out flush file.
2792   * @param storesToFlush The list of stores to flush.
2793   * @return object describing the flush's state
2794   * @throws IOException              general io exceptions
2795   * @throws DroppedSnapshotException Thrown when replay of WAL is required.
2796   */
2797  protected FlushResultImpl internalFlushcache(WAL wal, long myseqid,
2798    Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2799    FlushLifeCycleTracker tracker) throws IOException {
2800    PrepareFlushResult result =
2801      internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker);
2802    if (result.result == null) {
2803      return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2804    } else {
2805      return result.result; // early exit due to failure from prepare stage
2806    }
2807  }
2808
2809  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "DLS_DEAD_LOCAL_STORE",
2810      justification = "FindBugs seems confused about trxId")
2811  protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid,
2812    Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2813    FlushLifeCycleTracker tracker) throws IOException {
2814    if (this.rsServices != null && this.rsServices.isAborted()) {
2815      // Don't flush when server aborting, it's unsafe
2816      throw new IOException("Aborting flush because server is aborted...");
2817    }
2818    final long startTime = EnvironmentEdgeManager.currentTime();
2819    // If nothing to flush, return, but return with a valid unused sequenceId.
2820    // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a
2821    // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
2822    // to no other that it can use to associate with the bulk load. Hence this little dance below
2823    // to go get one.
2824    if (this.memStoreSizing.getDataSize() <= 0) {
2825      // Take an update lock so no edits can come into memory just yet.
2826      this.updatesLock.writeLock().lock();
2827      WriteEntry writeEntry = null;
2828      try {
2829        if (this.memStoreSizing.getDataSize() <= 0) {
2830          // Presume that if there are still no edits in the memstore, then there are no edits for
2831          // this region out in the WAL subsystem so no need to do any trickery clearing out
2832          // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
2833          // sure just beyond the last appended region edit and not associated with any edit
2834          // (useful as marker when bulk loading, etc.).
2835          if (wal != null) {
2836            writeEntry = mvcc.begin();
2837            long flushOpSeqId = writeEntry.getWriteNumber();
2838            FlushResultImpl flushResult = new FlushResultImpl(
2839              FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId, "Nothing to flush",
2840              writeCanNotFlushMarkerToWAL(writeEntry, wal, writeFlushWalMarker));
2841            mvcc.completeAndWait(writeEntry);
2842            // Set to null so we don't complete it again down in finally block.
2843            writeEntry = null;
2844            return new PrepareFlushResult(flushResult, myseqid);
2845          } else {
2846            return new PrepareFlushResult(new FlushResultImpl(
2847              FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid);
2848          }
2849        }
2850      } finally {
2851        if (writeEntry != null) {
2852          // If writeEntry is non-null, this operation failed; the mvcc transaction failed...
2853          // but complete it anyways so it doesn't block the mvcc queue.
2854          mvcc.complete(writeEntry);
2855        }
2856        this.updatesLock.writeLock().unlock();
2857      }
2858    }
2859    logFatLineOnFlush(storesToFlush, myseqid);
2860    // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
2861    // to do this for a moment. It is quick. We also set the memstore size to zero here before we
2862    // allow updates again so its value will represent the size of the updates received
2863    // during flush
2864
2865    // We have to take an update lock during snapshot, or else a write could end up in both snapshot
2866    // and memstore (makes it difficult to do atomic rows then)
2867    status.setStatus("Obtaining lock to block concurrent updates");
2868    // block waiting for the lock for internal flush
2869    this.updatesLock.writeLock().lock();
2870    status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
2871    MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing();
2872
2873    Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
2874    for (HStore store : storesToFlush) {
2875      flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
2876        store.preFlushSeqIDEstimation());
2877    }
2878
2879    TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2880    TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2881    TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2882    // The sequence id of this flush operation which is used to log FlushMarker and pass to
2883    // createFlushContext to use as the store file's sequence id. It can be in advance of edits
2884    // still in the memstore, edits that are in other column families yet to be flushed.
2885    long flushOpSeqId = HConstants.NO_SEQNUM;
2886    // The max flushed sequence id after this flush operation completes. All edits in memstore
2887    // will be in advance of this sequence id.
2888    long flushedSeqId = HConstants.NO_SEQNUM;
2889    byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2890    try {
2891      if (wal != null) {
2892        Long earliestUnflushedSequenceIdForTheRegion =
2893          wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq);
2894        if (earliestUnflushedSequenceIdForTheRegion == null) {
2895          // This should never happen. This is how startCacheFlush signals flush cannot proceed.
2896          String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2897          status.setStatus(msg);
2898          return new PrepareFlushResult(
2899            new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false), myseqid);
2900        }
2901        flushOpSeqId = getNextSequenceId(wal);
2902        // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2903        flushedSeqId = earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM
2904          ? flushOpSeqId
2905          : earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2906      } else {
2907        // use the provided sequence Id as WAL is not being used for this flush.
2908        flushedSeqId = flushOpSeqId = myseqid;
2909      }
2910
2911      for (HStore s : storesToFlush) {
2912        storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(),
2913          s.createFlushContext(flushOpSeqId, tracker));
2914        // for writing stores to WAL
2915        committedFiles.put(s.getColumnFamilyDescriptor().getName(), null);
2916      }
2917
2918      // write the snapshot start to WAL
2919      if (wal != null && !writestate.readOnly) {
2920        FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2921          getRegionInfo(), flushOpSeqId, committedFiles);
2922        // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
2923        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
2924          mvcc, regionReplicationSink.orElse(null));
2925      }
2926
2927      // Prepare flush (take a snapshot)
2928      storeFlushCtxs.forEach((name, flush) -> {
2929        MemStoreSize snapshotSize = flush.prepare();
2930        totalSizeOfFlushableStores.incMemStoreSize(snapshotSize);
2931        storeFlushableSize.put(name, snapshotSize);
2932      });
2933    } catch (IOException ex) {
2934      doAbortFlushToWAL(wal, flushOpSeqId, committedFiles);
2935      throw ex;
2936    } finally {
2937      this.updatesLock.writeLock().unlock();
2938    }
2939    String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, "
2940      + "flushsize=" + totalSizeOfFlushableStores;
2941    status.setStatus(s);
2942    doSyncOfUnflushedWALChanges(wal, getRegionInfo());
2943    return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2944      flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores);
2945  }
2946
2947  /**
2948   * Utility method broken out of internalPrepareFlushCache so that method is smaller.
2949   */
2950  private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) {
2951    if (!LOG.isInfoEnabled()) {
2952      return;
2953    }
2954    // Log a fat line detailing what is being flushed.
2955    StringBuilder perCfExtras = null;
2956    if (!isAllFamilies(storesToFlush)) {
2957      perCfExtras = new StringBuilder();
2958      for (HStore store : storesToFlush) {
2959        MemStoreSize mss = store.getFlushableSize();
2960        perCfExtras.append("; ").append(store.getColumnFamilyName());
2961        perCfExtras.append("={dataSize=").append(StringUtils.byteDesc(mss.getDataSize()));
2962        perCfExtras.append(", heapSize=").append(StringUtils.byteDesc(mss.getHeapSize()));
2963        perCfExtras.append(", offHeapSize=").append(StringUtils.byteDesc(mss.getOffHeapSize()));
2964        perCfExtras.append("}");
2965      }
2966    }
2967    MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
2968    LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " + storesToFlush.size() + "/"
2969      + stores.size() + " column families," + " dataSize=" + StringUtils.byteDesc(mss.getDataSize())
2970      + " heapSize=" + StringUtils.byteDesc(mss.getHeapSize())
2971      + ((perCfExtras != null && perCfExtras.length() > 0) ? perCfExtras.toString() : "")
2972      + ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
2973  }
2974
2975  private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId,
2976    final Map<byte[], List<Path>> committedFiles) {
2977    if (wal == null) return;
2978    try {
2979      FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2980        getRegionInfo(), flushOpSeqId, committedFiles);
2981      WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false, mvcc,
2982        null);
2983    } catch (Throwable t) {
2984      LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in "
2985        + " region {}", StringUtils.stringifyException(t), this);
2986      // ignore this since we will be aborting the RS with DSE.
2987    }
2988    // we have called wal.startCacheFlush(), now we have to abort it
2989    wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2990  }
2991
2992  /**
2993   * Sync unflushed WAL changes. See HBASE-8208 for details
2994   */
2995  private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri)
2996    throws IOException {
2997    if (wal == null) {
2998      return;
2999    }
3000    try {
3001      wal.sync(); // ensure that flush marker is sync'ed
3002    } catch (IOException ioe) {
3003      wal.abortCacheFlush(hri.getEncodedNameAsBytes());
3004      throw ioe;
3005    }
3006  }
3007
3008  /** Returns True if passed Set is all families in the region. */
3009  private boolean isAllFamilies(Collection<HStore> families) {
3010    return families == null || this.stores.size() == families.size();
3011  }
3012
3013  /**
3014   * This method is only used when we flush but the memstore is empty,if writeFlushWalMarker is
3015   * true,we write the {@link FlushAction#CANNOT_FLUSH} flush marker to WAL when the memstore is
3016   * empty. Ignores exceptions from WAL. Returns whether the write succeeded.
3017   * @return whether WAL write was successful
3018   */
3019  private boolean writeCanNotFlushMarkerToWAL(WriteEntry flushOpSeqIdMVCCEntry, WAL wal,
3020    boolean writeFlushWalMarker) {
3021    FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH, getRegionInfo(),
3022      -1, new TreeMap<>(Bytes.BYTES_COMPARATOR));
3023    RegionReplicationSink sink = regionReplicationSink.orElse(null);
3024
3025    if (sink != null && !writeFlushWalMarker) {
3026      /**
3027       * Here for replication to secondary region replica could use {@link FlushAction#CANNOT_FLUSH}
3028       * to recover when writeFlushWalMarker is false, we create {@link WALEdit} for
3029       * {@link FlushDescriptor} and attach the {@link RegionReplicationSink#add} to the
3030       * flushOpSeqIdMVCCEntry,see HBASE-26960 for more details.
3031       */
3032      this.attachRegionReplicationToFlushOpSeqIdMVCCEntry(flushOpSeqIdMVCCEntry, desc, sink);
3033      return false;
3034    }
3035
3036    if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
3037      try {
3038        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
3039          sink);
3040        return true;
3041      } catch (IOException e) {
3042        LOG.warn(getRegionInfo().getEncodedName() + " : "
3043          + "Received exception while trying to write the flush request to wal", e);
3044      }
3045    }
3046    return false;
3047  }
3048
3049  /**
3050   * Create {@link WALEdit} for {@link FlushDescriptor} and attach {@link RegionReplicationSink#add}
3051   * to the flushOpSeqIdMVCCEntry.
3052   */
3053  private void attachRegionReplicationToFlushOpSeqIdMVCCEntry(WriteEntry flushOpSeqIdMVCCEntry,
3054    FlushDescriptor desc, RegionReplicationSink sink) {
3055    assert !flushOpSeqIdMVCCEntry.getCompletionAction().isPresent();
3056    WALEdit flushMarkerWALEdit = WALEdit.createFlushWALEdit(getRegionInfo(), desc);
3057    WALKeyImpl walKey =
3058      WALUtil.createWALKey(getRegionInfo(), mvcc, this.getReplicationScope(), null);
3059    walKey.setWriteEntry(flushOpSeqIdMVCCEntry);
3060    /**
3061     * Here the {@link ServerCall} is null for {@link RegionReplicationSink#add} because the
3062     * flushMarkerWALEdit is created by ourselves, not from rpc.
3063     */
3064    flushOpSeqIdMVCCEntry.attachCompletionAction(() -> sink.add(walKey, flushMarkerWALEdit, null));
3065  }
3066
3067  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
3068      justification = "Intentional; notify is about completed flush")
3069  FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status,
3070    PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException {
3071    // prepare flush context is carried via PrepareFlushResult
3072    TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
3073    TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
3074    long startTime = prepareResult.startTime;
3075    long flushOpSeqId = prepareResult.flushOpSeqId;
3076    long flushedSeqId = prepareResult.flushedSeqId;
3077
3078    String s = "Flushing stores of " + this;
3079    status.setStatus(s);
3080    if (LOG.isTraceEnabled()) LOG.trace(s);
3081
3082    // Any failure from here on out will be catastrophic requiring server
3083    // restart so wal content can be replayed and put back into the memstore.
3084    // Otherwise, the snapshot content while backed up in the wal, it will not
3085    // be part of the current running servers state.
3086    boolean compactionRequested = false;
3087    long flushedOutputFileSize = 0;
3088    try {
3089      // A. Flush memstore to all the HStores.
3090      // Keep running vector of all store files that includes both old and the
3091      // just-made new flush store file. The new flushed file is still in the
3092      // tmp directory.
3093
3094      for (StoreFlushContext flush : storeFlushCtxs.values()) {
3095        flush.flushCache(status);
3096      }
3097
3098      // Switch snapshot (in memstore) -> new hfile (thus causing
3099      // all the store scanners to reset/reseek).
3100      for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) {
3101        StoreFlushContext sfc = flushEntry.getValue();
3102        boolean needsCompaction = sfc.commit(status);
3103        if (needsCompaction) {
3104          compactionRequested = true;
3105        }
3106        byte[] storeName = flushEntry.getKey();
3107        List<Path> storeCommittedFiles = sfc.getCommittedFiles();
3108        committedFiles.put(storeName, storeCommittedFiles);
3109        // Flush committed no files, indicating flush is empty or flush was canceled
3110        if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
3111          MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName);
3112          prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize);
3113        }
3114        flushedOutputFileSize += sfc.getOutputFileSize();
3115      }
3116      storeFlushCtxs.clear();
3117
3118      // Set down the memstore size by amount of flush.
3119      MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3120      this.decrMemStoreSize(mss);
3121
3122      // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled.
3123      // During startup, quota manager may not be initialized yet.
3124      if (rsServices != null) {
3125        RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager();
3126        if (quotaManager != null) {
3127          quotaManager.getRegionSizeStore().incrementRegionSize(this.getRegionInfo(),
3128            flushedOutputFileSize);
3129        }
3130      }
3131
3132      if (wal != null) {
3133        // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
3134        FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
3135          getRegionInfo(), flushOpSeqId, committedFiles);
3136        WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true, mvcc,
3137          regionReplicationSink.orElse(null));
3138      }
3139    } catch (Throwable t) {
3140      // An exception here means that the snapshot was not persisted.
3141      // The wal needs to be replayed so its content is restored to memstore.
3142      // Currently, only a server restart will do this.
3143      // We used to only catch IOEs but its possible that we'd get other
3144      // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
3145      // all and sundry.
3146      if (wal != null) {
3147        try {
3148          FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
3149            getRegionInfo(), flushOpSeqId, committedFiles);
3150          WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc,
3151            null);
3152        } catch (Throwable ex) {
3153          LOG.warn(
3154            getRegionInfo().getEncodedName() + " : " + "failed writing ABORT_FLUSH marker to WAL",
3155            ex);
3156          // ignore this since we will be aborting the RS with DSE.
3157        }
3158        wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
3159      }
3160      DroppedSnapshotException dse = new DroppedSnapshotException(
3161        "region: " + Bytes.toStringBinary(getRegionInfo().getRegionName()), t);
3162      status.abort("Flush failed: " + StringUtils.stringifyException(t));
3163
3164      // Callers for flushcache() should catch DroppedSnapshotException and abort the region server.
3165      // However, since we may have the region read lock, we cannot call close(true) here since
3166      // we cannot promote to a write lock. Instead we are setting closing so that all other region
3167      // operations except for close will be rejected.
3168      this.closing.set(true);
3169
3170      if (rsServices != null) {
3171        // This is a safeguard against the case where the caller fails to explicitly handle aborting
3172        rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
3173      }
3174
3175      throw dse;
3176    }
3177
3178    // If we get to here, the HStores have been written.
3179    if (wal != null) {
3180      wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId);
3181    }
3182
3183    // Record latest flush time
3184    for (HStore store : storesToFlush) {
3185      this.lastStoreFlushTimeMap.put(store, startTime);
3186    }
3187
3188    this.maxFlushedSeqId = flushedSeqId;
3189    this.lastFlushOpSeqId = flushOpSeqId;
3190
3191    // C. Finally notify anyone waiting on memstore to clear:
3192    // e.g. checkResources().
3193    synchronized (this) {
3194      notifyAll(); // FindBugs NN_NAKED_NOTIFY
3195    }
3196
3197    long time = EnvironmentEdgeManager.currentTime() - startTime;
3198    MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3199    long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize();
3200    String msg = "Finished flush of" + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/"
3201      + mss.getDataSize() + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/"
3202      + mss.getHeapSize() + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/"
3203      + memstoresize + " for " + this.getRegionInfo().getEncodedName() + " in " + time
3204      + "ms, sequenceid=" + flushOpSeqId + ", compaction requested=" + compactionRequested
3205      + ((wal == null) ? "; wal=null" : "");
3206    LOG.info(msg);
3207    status.setStatus(msg);
3208
3209    if (rsServices != null && rsServices.getMetrics() != null) {
3210      rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
3211        time, mss.getDataSize(), flushedOutputFileSize);
3212    }
3213
3214    return new FlushResultImpl(compactionRequested
3215      ? FlushResult.Result.FLUSHED_COMPACTION_NEEDED
3216      : FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
3217  }
3218
3219  /**
3220   * Method to safely get the next sequence number.
3221   * @return Next sequence number unassociated with any actual edit.
3222   */
3223  protected long getNextSequenceId(final WAL wal) throws IOException {
3224    WriteEntry we = mvcc.begin();
3225    mvcc.completeAndWait(we);
3226    return we.getWriteNumber();
3227  }
3228
3229  //////////////////////////////////////////////////////////////////////////////
3230  // get() methods for client use.
3231  //////////////////////////////////////////////////////////////////////////////
3232
3233  @Override
3234  public RegionScannerImpl getScanner(Scan scan) throws IOException {
3235    return getScanner(scan, null);
3236  }
3237
3238  @Override
3239  public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
3240    throws IOException {
3241    return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
3242  }
3243
3244  private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
3245    long nonceGroup, long nonce) throws IOException {
3246    return TraceUtil.trace(() -> {
3247      startRegionOperation(Operation.SCAN);
3248      try {
3249        // Verify families are all valid
3250        if (!scan.hasFamilies()) {
3251          // Adding all families to scanner
3252          for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
3253            scan.addFamily(family);
3254          }
3255        } else {
3256          for (byte[] family : scan.getFamilyMap().keySet()) {
3257            checkFamily(family);
3258          }
3259        }
3260        return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
3261      } finally {
3262        closeRegionOperation(Operation.SCAN);
3263      }
3264    }, () -> createRegionSpan("Region.getScanner"));
3265  }
3266
3267  protected RegionScannerImpl instantiateRegionScanner(Scan scan,
3268    List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
3269    if (scan.isReversed()) {
3270      if (scan.getFilter() != null) {
3271        scan.getFilter().setReversed(true);
3272      }
3273      return new ReversedRegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3274    }
3275    return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3276  }
3277
3278  /**
3279   * Prepare a delete for a row mutation processor
3280   * @param delete The passed delete is modified by this method. WARNING!
3281   */
3282  private void prepareDelete(Delete delete) throws IOException {
3283    // Check to see if this is a deleteRow insert
3284    if (delete.getFamilyCellMap().isEmpty()) {
3285      for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
3286        // Don't eat the timestamp
3287        delete.addFamily(family, delete.getTimestamp());
3288      }
3289    } else {
3290      for (byte[] family : delete.getFamilyCellMap().keySet()) {
3291        if (family == null) {
3292          throw new NoSuchColumnFamilyException("Empty family is invalid");
3293        }
3294        checkFamily(family, delete.getDurability());
3295      }
3296    }
3297  }
3298
3299  @Override
3300  public void delete(Delete delete) throws IOException {
3301    TraceUtil.trace(() -> {
3302      checkReadOnly();
3303      checkResources();
3304      startRegionOperation(Operation.DELETE);
3305      try {
3306        // All edits for the given row (across all column families) must happen atomically.
3307        return mutate(delete);
3308      } finally {
3309        closeRegionOperation(Operation.DELETE);
3310      }
3311    }, () -> createRegionSpan("Region.delete"));
3312  }
3313
3314  /**
3315   * Set up correct timestamps in the KVs in Delete object.
3316   * <p/>
3317   * Caller should have the row and region locks.
3318   */
3319  private void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<ExtendedCell>> familyMap,
3320    byte[] byteNow) throws IOException {
3321    for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) {
3322
3323      byte[] family = e.getKey();
3324      List<ExtendedCell> cells = e.getValue();
3325      assert cells instanceof RandomAccess;
3326
3327      Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR);
3328      int listSize = cells.size();
3329      for (int i = 0; i < listSize; i++) {
3330        ExtendedCell cell = cells.get(i);
3331        // Check if time is LATEST, change to time of most recent addition if so
3332        // This is expensive.
3333        if (
3334          cell.getTimestamp() == HConstants.LATEST_TIMESTAMP && PrivateCellUtil.isDeleteType(cell)
3335        ) {
3336          byte[] qual = CellUtil.cloneQualifier(cell);
3337
3338          Integer count = kvCount.get(qual);
3339          if (count == null) {
3340            kvCount.put(qual, 1);
3341          } else {
3342            kvCount.put(qual, count + 1);
3343          }
3344          count = kvCount.get(qual);
3345
3346          Get get = new Get(CellUtil.cloneRow(cell));
3347          get.readVersions(count);
3348          get.addColumn(family, qual);
3349          if (coprocessorHost != null) {
3350            if (
3351              !coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell, byteNow, get)
3352            ) {
3353              updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3354            }
3355          } else {
3356            updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3357          }
3358        } else {
3359          PrivateCellUtil.updateLatestStamp(cell, byteNow);
3360        }
3361      }
3362    }
3363  }
3364
3365  private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
3366    throws IOException {
3367    try (RegionScanner scanner = getScanner(new Scan(get))) {
3368      // NOTE: Please don't use HRegion.get() instead,
3369      // because it will copy cells to heap. See HBASE-26036
3370      List<ExtendedCell> result = new ArrayList<>();
3371      scanner.next(result);
3372
3373      if (result.size() < count) {
3374        // Nothing to delete
3375        PrivateCellUtil.updateLatestStamp(cell, byteNow);
3376        return;
3377      }
3378      if (result.size() > count) {
3379        throw new RuntimeException("Unexpected size: " + result.size());
3380      }
3381      Cell getCell = result.get(count - 1);
3382      PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
3383    }
3384  }
3385
3386  @Override
3387  public void put(Put put) throws IOException {
3388    TraceUtil.trace(() -> {
3389      checkReadOnly();
3390
3391      // Do a rough check that we have resources to accept a write. The check is
3392      // 'rough' in that between the resource check and the call to obtain a
3393      // read lock, resources may run out. For now, the thought is that this
3394      // will be extremely rare; we'll deal with it when it happens.
3395      checkResources();
3396      startRegionOperation(Operation.PUT);
3397      try {
3398        // All edits for the given row (across all column families) must happen atomically.
3399        return mutate(put);
3400      } finally {
3401        closeRegionOperation(Operation.PUT);
3402      }
3403    }, () -> createRegionSpan("Region.put"));
3404  }
3405
3406  /**
3407   * Class that tracks the progress of a batch operations, accumulating status codes and tracking
3408   * the index at which processing is proceeding. These batch operations may get split into
3409   * mini-batches for processing.
3410   */
3411  private abstract static class BatchOperation<T> {
3412    protected final T[] operations;
3413    protected final OperationStatus[] retCodeDetails;
3414    protected final WALEdit[] walEditsFromCoprocessors;
3415    // reference family cell maps directly so coprocessors can mutate them if desired
3416    protected final Map<byte[], List<ExtendedCell>>[] familyCellMaps;
3417    // For Increment/Append operations
3418    protected final Result[] results;
3419
3420    protected final HRegion region;
3421    protected int nextIndexToProcess = 0;
3422    protected final ObservedExceptionsInBatch observedExceptions;
3423    // Durability of the batch (highest durability of all operations)
3424    protected Durability durability;
3425    protected boolean atomic = false;
3426
3427    public BatchOperation(final HRegion region, T[] operations) {
3428      this.operations = operations;
3429      this.retCodeDetails = new OperationStatus[operations.length];
3430      Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
3431      this.walEditsFromCoprocessors = new WALEdit[operations.length];
3432      familyCellMaps = new Map[operations.length];
3433      this.results = new Result[operations.length];
3434
3435      this.region = region;
3436      observedExceptions = new ObservedExceptionsInBatch();
3437      durability = Durability.USE_DEFAULT;
3438    }
3439
3440    /**
3441     * Visitor interface for batch operations
3442     */
3443    @FunctionalInterface
3444    interface Visitor {
3445      /**
3446       * @param index operation index
3447       * @return If true continue visiting remaining entries, break otherwise
3448       */
3449      boolean visit(int index) throws IOException;
3450    }
3451
3452    /**
3453     * Helper method for visiting pending/ all batch operations
3454     */
3455    public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor)
3456      throws IOException {
3457      assert lastIndexExclusive <= this.size();
3458      for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) {
3459        if (!pendingOnly || isOperationPending(i)) {
3460          if (!visitor.visit(i)) {
3461            break;
3462          }
3463        }
3464      }
3465    }
3466
3467    public abstract Mutation getMutation(int index);
3468
3469    public abstract long getNonceGroup(int index);
3470
3471    public abstract long getNonce(int index);
3472
3473    /**
3474     * This method is potentially expensive and useful mostly for non-replay CP path.
3475     */
3476    public abstract Mutation[] getMutationsForCoprocs();
3477
3478    public abstract boolean isInReplay();
3479
3480    public abstract long getOrigLogSeqNum();
3481
3482    public abstract void startRegionOperation() throws IOException;
3483
3484    public abstract void closeRegionOperation() throws IOException;
3485
3486    /**
3487     * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs
3488     * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This
3489     * is intended to operate on entire batch and will be called from outside of class to check and
3490     * prepare batch. This can be implemented by calling helper method
3491     * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations.
3492     */
3493    public abstract void checkAndPrepare() throws IOException;
3494
3495    /**
3496     * Implement any Put request specific check and prepare logic here. Please refer to
3497     * {@link #checkAndPrepareMutation(Mutation, long)} for how its used.
3498     */
3499    protected abstract void checkAndPreparePut(final Put p) throws IOException;
3500
3501    /**
3502     * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell
3503     * count, tags and timestamp for all cells of all operations in a mini-batch.
3504     */
3505    public abstract void prepareMiniBatchOperations(
3506      MiniBatchOperationInProgress<Mutation> miniBatchOp, long timestamp,
3507      final List<RowLock> acquiredRowLocks) throws IOException;
3508
3509    /**
3510     * Write mini-batch operations to MemStore
3511     */
3512    public abstract WriteEntry writeMiniBatchOperationsToMemStore(
3513      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry,
3514      long now) throws IOException;
3515
3516    protected void writeMiniBatchOperationsToMemStore(
3517      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber)
3518      throws IOException {
3519      MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing();
3520      visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3521        // We need to update the sequence id for following reasons.
3522        // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id.
3523        // 2) If no WAL, FSWALEntry won't be used
3524        // we use durability of the original mutation for the mutation passed by CP.
3525        if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) {
3526          region.updateSequenceId(familyCellMaps[index].values(), writeNumber);
3527        }
3528        applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting);
3529        return true;
3530      });
3531      // update memStore size
3532      region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(),
3533        memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount());
3534    }
3535
3536    public boolean isDone() {
3537      return nextIndexToProcess == operations.length;
3538    }
3539
3540    public int size() {
3541      return operations.length;
3542    }
3543
3544    public boolean isOperationPending(int index) {
3545      return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN;
3546    }
3547
3548    public List<UUID> getClusterIds() {
3549      assert size() != 0;
3550      return getMutation(0).getClusterIds();
3551    }
3552
3553    boolean isAtomic() {
3554      return atomic;
3555    }
3556
3557    /**
3558     * Helper method that checks and prepares only one mutation. This can be used to implement
3559     * {@link #checkAndPrepare()} for entire Batch. NOTE: As CP
3560     * prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations, this method
3561     * should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks are run for
3562     * the mutation
3563     */
3564    protected void checkAndPrepareMutation(Mutation mutation, final long timestamp)
3565      throws IOException {
3566      region.checkRow(mutation.getRow(), "batchMutate");
3567      if (mutation instanceof Put) {
3568        // Check the families in the put. If bad, skip this one.
3569        checkAndPreparePut((Put) mutation);
3570        region.checkTimestamps(mutation.getFamilyCellMap(), timestamp);
3571      } else if (mutation instanceof Delete) {
3572        region.prepareDelete((Delete) mutation);
3573      } else if (mutation instanceof Increment || mutation instanceof Append) {
3574        region.checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability());
3575      }
3576    }
3577
3578    protected void checkAndPrepareMutation(int index, long timestamp) throws IOException {
3579      Mutation mutation = getMutation(index);
3580      try {
3581        this.checkAndPrepareMutation(mutation, timestamp);
3582
3583        if (mutation instanceof Put || mutation instanceof Delete) {
3584          // store the family map reference to allow for mutations
3585          // we know that in mutation, only ExtendedCells are allow so here we do a fake cast, to
3586          // simplify later logic
3587          familyCellMaps[index] = ClientInternalHelper.getExtendedFamilyCellMap(mutation);
3588        }
3589
3590        // store durability for the batch (highest durability of all operations in the batch)
3591        Durability tmpDur = region.getEffectiveDurability(mutation.getDurability());
3592        if (tmpDur.ordinal() > durability.ordinal()) {
3593          durability = tmpDur;
3594        }
3595      } catch (NoSuchColumnFamilyException nscfe) {
3596        final String msg = "No such column family in batch mutation in region " + this;
3597        if (observedExceptions.hasSeenNoSuchFamily()) {
3598          LOG.warn(msg + nscfe.getMessage());
3599        } else {
3600          LOG.warn(msg, nscfe);
3601          observedExceptions.sawNoSuchFamily();
3602        }
3603        retCodeDetails[index] =
3604          new OperationStatus(OperationStatusCode.BAD_FAMILY, nscfe.getMessage());
3605        if (isAtomic()) { // fail, atomic means all or none
3606          throw nscfe;
3607        }
3608      } catch (FailedSanityCheckException fsce) {
3609        final String msg = "Batch Mutation did not pass sanity check in region " + this;
3610        if (observedExceptions.hasSeenFailedSanityCheck()) {
3611          LOG.warn(msg + fsce.getMessage());
3612        } else {
3613          LOG.warn(msg, fsce);
3614          observedExceptions.sawFailedSanityCheck();
3615        }
3616        retCodeDetails[index] =
3617          new OperationStatus(OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3618        if (isAtomic()) {
3619          throw fsce;
3620        }
3621      } catch (WrongRegionException we) {
3622        final String msg = "Batch mutation had a row that does not belong to this region " + this;
3623        if (observedExceptions.hasSeenWrongRegion()) {
3624          LOG.warn(msg + we.getMessage());
3625        } else {
3626          LOG.warn(msg, we);
3627          observedExceptions.sawWrongRegion();
3628        }
3629        retCodeDetails[index] =
3630          new OperationStatus(OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3631        if (isAtomic()) {
3632          throw we;
3633        }
3634      }
3635    }
3636
3637    /**
3638     * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which a row
3639     * lock can be acquired. All mutations with locked rows are considered to be In-progress
3640     * operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch is window over
3641     * {@link BatchOperation} and contains contiguous pending operations.
3642     * @param acquiredRowLocks keeps track of rowLocks acquired.
3643     */
3644    public MiniBatchOperationInProgress<Mutation>
3645      lockRowsAndBuildMiniBatch(List<RowLock> acquiredRowLocks) throws IOException {
3646      int readyToWriteCount = 0;
3647      int lastIndexExclusive = 0;
3648      RowLock prevRowLock = null;
3649      for (; lastIndexExclusive < size(); lastIndexExclusive++) {
3650        // It reaches the miniBatchSize, stop here and process the miniBatch
3651        // This only applies to non-atomic batch operations.
3652        if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) {
3653          break;
3654        }
3655
3656        if (!isOperationPending(lastIndexExclusive)) {
3657          continue;
3658        }
3659
3660        // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting
3661        // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation
3662        // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't
3663        // pass the isOperationPending check
3664        Map<byte[], List<Cell>> curFamilyCellMap =
3665          getMutation(lastIndexExclusive).getFamilyCellMap();
3666        try {
3667          // start the protector before acquiring row lock considering performance, and will finish
3668          // it when encountering exception
3669          region.storeHotnessProtector.start(curFamilyCellMap);
3670        } catch (RegionTooBusyException rtbe) {
3671          region.storeHotnessProtector.finish(curFamilyCellMap);
3672          if (isAtomic()) {
3673            throw rtbe;
3674          }
3675          retCodeDetails[lastIndexExclusive] =
3676            new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage());
3677          continue;
3678        }
3679
3680        Mutation mutation = getMutation(lastIndexExclusive);
3681        // If we haven't got any rows in our batch, we should block to get the next one.
3682        RowLock rowLock = null;
3683        boolean throwException = false;
3684        try {
3685          // if atomic then get exclusive lock, else shared lock
3686          rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock);
3687        } catch (TimeoutIOException | InterruptedIOException e) {
3688          // NOTE: We will retry when other exceptions, but we should stop if we receive
3689          // TimeoutIOException or InterruptedIOException as operation has timed out or
3690          // interrupted respectively.
3691          throwException = true;
3692          throw e;
3693        } catch (IOException ioe) {
3694          LOG.warn("Failed getting lock, row={}, in region {}",
3695            Bytes.toStringBinary(mutation.getRow()), this, ioe);
3696          if (isAtomic()) { // fail, atomic means all or none
3697            throwException = true;
3698            throw ioe;
3699          }
3700        } catch (Throwable throwable) {
3701          throwException = true;
3702          throw throwable;
3703        } finally {
3704          if (throwException) {
3705            region.storeHotnessProtector.finish(curFamilyCellMap);
3706          }
3707        }
3708        if (rowLock == null) {
3709          // We failed to grab another lock
3710          if (isAtomic()) {
3711            region.storeHotnessProtector.finish(curFamilyCellMap);
3712            throw new IOException("Can't apply all operations atomically!");
3713          }
3714          break; // Stop acquiring more rows for this batch
3715        } else {
3716          if (rowLock != prevRowLock) {
3717            // It is a different row now, add this to the acquiredRowLocks and
3718            // set prevRowLock to the new returned rowLock
3719            acquiredRowLocks.add(rowLock);
3720            prevRowLock = rowLock;
3721          }
3722        }
3723
3724        readyToWriteCount++;
3725      }
3726      return createMiniBatch(lastIndexExclusive, readyToWriteCount);
3727    }
3728
3729    protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive,
3730      final int readyToWriteCount) {
3731      return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails,
3732        walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount);
3733    }
3734
3735    protected WALEdit createWALEdit(final MiniBatchOperationInProgress<Mutation> miniBatchOp) {
3736      return new WALEdit(miniBatchOp.getCellCount(), isInReplay());
3737    }
3738
3739    /**
3740     * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are
3741     * present, they are merged to result WALEdit.
3742     */
3743    public List<Pair<NonceKey, WALEdit>>
3744      buildWALEdits(final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
3745      List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>();
3746
3747      visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() {
3748        private Pair<NonceKey, WALEdit> curWALEditForNonce;
3749
3750        @Override
3751        public boolean visit(int index) throws IOException {
3752          Mutation m = getMutation(index);
3753          // we use durability of the original mutation for the mutation passed by CP.
3754          if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) {
3755            region.recordMutationWithoutWal(m.getFamilyCellMap());
3756            /**
3757             * Here is for HBASE-26993,in order to make the new framework for region replication
3758             * could work for SKIP_WAL, we save the {@link Mutation} which
3759             * {@link Mutation#getDurability} is {@link Durability#SKIP_WAL} in miniBatchOp.
3760             */
3761            cacheSkipWALMutationForRegionReplication(miniBatchOp, walEdits, familyCellMaps[index]);
3762            return true;
3763          }
3764
3765          // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each.
3766          // Given how nonce keys are originally written, these should be contiguous.
3767          // They don't have to be, it will still work, just write more WALEdits than needed.
3768          long nonceGroup = getNonceGroup(index);
3769          long nonce = getNonce(index);
3770          if (
3771            curWALEditForNonce == null
3772              || curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup
3773              || curWALEditForNonce.getFirst().getNonce() != nonce
3774          ) {
3775            curWALEditForNonce =
3776              new Pair<>(new NonceKey(nonceGroup, nonce), createWALEdit(miniBatchOp));
3777            walEdits.add(curWALEditForNonce);
3778          }
3779          WALEdit walEdit = curWALEditForNonce.getSecond();
3780
3781          // Add WAL edits from CPs.
3782          WALEdit fromCP = walEditsFromCoprocessors[index];
3783          List<ExtendedCell> cellsFromCP = fromCP == null
3784            ? Collections.emptyList()
3785            : WALEditInternalHelper.getExtendedCells(fromCP);
3786          addNonSkipWALMutationsToWALEdit(miniBatchOp, walEdit, cellsFromCP, familyCellMaps[index]);
3787          return true;
3788        }
3789      });
3790      return walEdits;
3791    }
3792
3793    protected void addNonSkipWALMutationsToWALEdit(
3794      final MiniBatchOperationInProgress<Mutation> miniBatchOp, WALEdit walEdit,
3795      List<ExtendedCell> cellsFromCP, Map<byte[], List<ExtendedCell>> familyCellMap) {
3796      doAddCellsToWALEdit(walEdit, cellsFromCP, familyCellMap);
3797    }
3798
3799    protected static void doAddCellsToWALEdit(WALEdit walEdit, List<ExtendedCell> cellsFromCP,
3800      Map<byte[], List<ExtendedCell>> familyCellMap) {
3801      WALEditInternalHelper.addExtendedCell(walEdit, cellsFromCP);
3802      WALEditInternalHelper.addMap(walEdit, familyCellMap);
3803    }
3804
3805    protected abstract void cacheSkipWALMutationForRegionReplication(
3806      final MiniBatchOperationInProgress<Mutation> miniBatchOp,
3807      List<Pair<NonceKey, WALEdit>> walEdits, Map<byte[], List<ExtendedCell>> familyCellMap);
3808
3809    /**
3810     * This method completes mini-batch operations by calling postBatchMutate() CP hook (if
3811     * required) and completing mvcc.
3812     */
3813    public void completeMiniBatchOperations(
3814      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3815      throws IOException {
3816      if (writeEntry != null) {
3817        region.mvcc.completeAndWait(writeEntry);
3818      }
3819    }
3820
3821    public void doPostOpCleanupForMiniBatch(
3822      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit,
3823      boolean success) throws IOException {
3824      doFinishHotnessProtector(miniBatchOp);
3825    }
3826
3827    private void
3828      doFinishHotnessProtector(final MiniBatchOperationInProgress<Mutation> miniBatchOp) {
3829      // check and return if the protector is not enabled
3830      if (!region.storeHotnessProtector.isEnable()) {
3831        return;
3832      }
3833      // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception.
3834      // This case was handled.
3835      if (miniBatchOp == null) {
3836        return;
3837      }
3838
3839      final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive();
3840
3841      for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) {
3842        switch (retCodeDetails[i].getOperationStatusCode()) {
3843          case SUCCESS:
3844          case FAILURE:
3845            region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap());
3846            break;
3847          default:
3848            // do nothing
3849            // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the
3850            // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start
3851            break;
3852        }
3853      }
3854    }
3855
3856    /**
3857     * Atomically apply the given map of family->edits to the memstore. This handles the consistency
3858     * control on its own, but the caller should already have locked updatesLock.readLock(). This
3859     * also does <b>not</b> check the families for validity.
3860     * @param familyMap Map of Cells by family
3861     */
3862    protected void applyFamilyMapToMemStore(Map<byte[], List<ExtendedCell>> familyMap,
3863      MemStoreSizing memstoreAccounting) {
3864      for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) {
3865        byte[] family = e.getKey();
3866        List<ExtendedCell> cells = e.getValue();
3867        assert cells instanceof RandomAccess;
3868        region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting);
3869      }
3870    }
3871  }
3872
3873  /**
3874   * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most of
3875   * the logic is same.
3876   */
3877  private static class MutationBatchOperation extends BatchOperation<Mutation> {
3878
3879    // For nonce operations
3880    private long nonceGroup;
3881    private long nonce;
3882    protected boolean canProceed;
3883    private boolean regionReplicateEnable;
3884
3885    public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic,
3886      long nonceGroup, long nonce) {
3887      super(region, operations);
3888      this.atomic = atomic;
3889      this.nonceGroup = nonceGroup;
3890      this.nonce = nonce;
3891      this.regionReplicateEnable = region.regionReplicationSink.isPresent();
3892    }
3893
3894    @Override
3895    public Mutation getMutation(int index) {
3896      return this.operations[index];
3897    }
3898
3899    @Override
3900    public long getNonceGroup(int index) {
3901      return nonceGroup;
3902    }
3903
3904    @Override
3905    public long getNonce(int index) {
3906      return nonce;
3907    }
3908
3909    @Override
3910    public Mutation[] getMutationsForCoprocs() {
3911      return this.operations;
3912    }
3913
3914    @Override
3915    public boolean isInReplay() {
3916      return false;
3917    }
3918
3919    @Override
3920    public long getOrigLogSeqNum() {
3921      return SequenceId.NO_SEQUENCE_ID;
3922    }
3923
3924    @Override
3925    public void startRegionOperation() throws IOException {
3926      region.startRegionOperation(Operation.BATCH_MUTATE);
3927    }
3928
3929    @Override
3930    public void closeRegionOperation() throws IOException {
3931      region.closeRegionOperation(Operation.BATCH_MUTATE);
3932    }
3933
3934    @Override
3935    public void checkAndPreparePut(Put p) throws IOException {
3936      region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability());
3937    }
3938
3939    @Override
3940    public void checkAndPrepare() throws IOException {
3941      // index 0: puts, index 1: deletes, index 2: increments, index 3: append
3942      final int[] metrics = { 0, 0, 0, 0 };
3943
3944      visitBatchOperations(true, this.size(), new Visitor() {
3945        private long now = EnvironmentEdgeManager.currentTime();
3946        private WALEdit walEdit;
3947
3948        @Override
3949        public boolean visit(int index) throws IOException {
3950          // Run coprocessor pre hook outside of locks to avoid deadlock
3951          if (region.coprocessorHost != null) {
3952            if (walEdit == null) {
3953              walEdit = new WALEdit();
3954            }
3955            callPreMutateCPHook(index, walEdit, metrics);
3956            if (!walEdit.isEmpty()) {
3957              walEditsFromCoprocessors[index] = walEdit;
3958              walEdit = null;
3959            }
3960          }
3961          if (isOperationPending(index)) {
3962            // TODO: Currently validation is done with current time before acquiring locks and
3963            // updates are done with different timestamps after acquiring locks. This behavior is
3964            // inherited from the code prior to this change. Can this be changed?
3965            checkAndPrepareMutation(index, now);
3966          }
3967          return true;
3968        }
3969      });
3970
3971      // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in
3972      // normal processing.
3973      // Update metrics in same way as it is done when we go the normal processing route (we now
3974      // update general metrics though a Coprocessor did the work).
3975      if (region.metricsRegion != null) {
3976        if (metrics[0] > 0) {
3977          // There were some Puts in the batch.
3978          region.metricsRegion.updatePut();
3979        }
3980        if (metrics[1] > 0) {
3981          // There were some Deletes in the batch.
3982          region.metricsRegion.updateDelete();
3983        }
3984        if (metrics[2] > 0) {
3985          // There were some Increment in the batch.
3986          region.metricsRegion.updateIncrement();
3987        }
3988        if (metrics[3] > 0) {
3989          // There were some Append in the batch.
3990          region.metricsRegion.updateAppend();
3991        }
3992      }
3993    }
3994
3995    @Override
3996    public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
3997      long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
3998      // For nonce operations
3999      canProceed = startNonceOperation();
4000
4001      visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
4002        Mutation mutation = getMutation(index);
4003        if (mutation instanceof Put) {
4004          HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp));
4005          miniBatchOp.incrementNumOfPuts();
4006        } else if (mutation instanceof Delete) {
4007          region.prepareDeleteTimestamps(mutation, familyCellMaps[index], Bytes.toBytes(timestamp));
4008          miniBatchOp.incrementNumOfDeletes();
4009        } else if (mutation instanceof Increment || mutation instanceof Append) {
4010          boolean returnResults;
4011          if (mutation instanceof Increment) {
4012            returnResults = ((Increment) mutation).isReturnResults();
4013          } else {
4014            returnResults = ((Append) mutation).isReturnResults();
4015          }
4016
4017          // For nonce operations
4018          if (!canProceed) {
4019            Result result;
4020            if (returnResults) {
4021              // convert duplicate increment/append to get
4022              List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce);
4023              result = Result.create(results);
4024            } else {
4025              result = Result.EMPTY_RESULT;
4026            }
4027            retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4028            return true;
4029          }
4030
4031          Result result = null;
4032          if (region.coprocessorHost != null) {
4033            if (mutation instanceof Increment) {
4034              result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation);
4035            } else {
4036              result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation);
4037            }
4038          }
4039          if (result != null) {
4040            retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS,
4041              returnResults ? result : Result.EMPTY_RESULT);
4042            return true;
4043          }
4044
4045          List<ExtendedCell> results = returnResults ? new ArrayList<>(mutation.size()) : null;
4046          familyCellMaps[index] = reckonDeltas(mutation, results, timestamp);
4047          this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT;
4048
4049          if (mutation instanceof Increment) {
4050            miniBatchOp.incrementNumOfIncrements();
4051          } else {
4052            miniBatchOp.incrementNumOfAppends();
4053          }
4054        }
4055        region.rewriteCellTags(familyCellMaps[index], mutation);
4056
4057        // update cell count
4058        if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
4059          for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
4060            miniBatchOp.addCellCount(cells.size());
4061          }
4062        }
4063
4064        WALEdit fromCP = walEditsFromCoprocessors[index];
4065        if (fromCP != null) {
4066          miniBatchOp.addCellCount(fromCP.size());
4067        }
4068        return true;
4069      });
4070
4071      if (region.coprocessorHost != null) {
4072        // calling the pre CP hook for batch mutation
4073        region.coprocessorHost.preBatchMutate(miniBatchOp);
4074        checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp);
4075      }
4076    }
4077
4078    /**
4079     * Starts the nonce operation for a mutation, if needed.
4080     * @return whether to proceed this mutation.
4081     */
4082    private boolean startNonceOperation() throws IOException {
4083      if (
4084        region.rsServices == null || region.rsServices.getNonceManager() == null
4085          || nonce == HConstants.NO_NONCE
4086      ) {
4087        return true;
4088      }
4089      boolean canProceed;
4090      try {
4091        canProceed =
4092          region.rsServices.getNonceManager().startOperation(nonceGroup, nonce, region.rsServices);
4093      } catch (InterruptedException ex) {
4094        throw new InterruptedIOException("Nonce start operation interrupted");
4095      }
4096      return canProceed;
4097    }
4098
4099    /**
4100     * Ends nonce operation for a mutation, if needed.
4101     * @param success Whether the operation for this nonce has succeeded.
4102     */
4103    private void endNonceOperation(boolean success) {
4104      if (
4105        region.rsServices != null && region.rsServices.getNonceManager() != null
4106          && nonce != HConstants.NO_NONCE
4107      ) {
4108        region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success);
4109      }
4110    }
4111
4112    private static Get toGet(final Mutation mutation) throws IOException {
4113      assert mutation instanceof Increment || mutation instanceof Append;
4114      Get get = new Get(mutation.getRow());
4115      CellScanner cellScanner = mutation.cellScanner();
4116      while (cellScanner.advance()) {
4117        Cell cell = cellScanner.current();
4118        get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell));
4119      }
4120      if (mutation instanceof Increment) {
4121        // Increment
4122        Increment increment = (Increment) mutation;
4123        get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax());
4124      } else {
4125        // Append
4126        Append append = (Append) mutation;
4127        get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax());
4128      }
4129      for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) {
4130        get.setAttribute(entry.getKey(), entry.getValue());
4131      }
4132      return get;
4133    }
4134
4135    private Map<byte[], List<ExtendedCell>> reckonDeltas(Mutation mutation,
4136      List<ExtendedCell> results, long now) throws IOException {
4137      assert mutation instanceof Increment || mutation instanceof Append;
4138      Map<byte[], List<ExtendedCell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
4139      // Process a Store/family at a time.
4140      for (Map.Entry<byte[], List<ExtendedCell>> entry : ClientInternalHelper
4141        .getExtendedFamilyCellMap(mutation).entrySet()) {
4142        final byte[] columnFamilyName = entry.getKey();
4143        List<ExtendedCell> deltas = entry.getValue();
4144        // Reckon for the Store what to apply to WAL and MemStore.
4145        List<ExtendedCell> toApply =
4146          reckonDeltasByStore(region.stores.get(columnFamilyName), mutation, now, deltas, results);
4147        if (!toApply.isEmpty()) {
4148          for (ExtendedCell cell : toApply) {
4149            HStore store = region.getStore(cell);
4150            if (store == null) {
4151              region.checkFamily(CellUtil.cloneFamily(cell));
4152            } else {
4153              ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(),
4154                key -> new ArrayList<>()).add(cell);
4155            }
4156          }
4157        }
4158      }
4159      return ret;
4160    }
4161
4162    /**
4163     * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed column
4164     * family/Store. Does Get of current value and then adds passed in deltas for this Store
4165     * returning the result.
4166     * @param mutation The encompassing Mutation object
4167     * @param deltas   Changes to apply to this Store; either increment amount or data to append
4168     * @param results  In here we accumulate all the Cells we are to return to the client. If null,
4169     *                 client doesn't want results returned.
4170     * @return Resulting Cells after <code>deltas</code> have been applied to current values. Side
4171     *         effect is our filling out of the <code>results</code> List.
4172     */
4173    private List<ExtendedCell> reckonDeltasByStore(HStore store, Mutation mutation, long now,
4174      List<ExtendedCell> deltas, List<ExtendedCell> results) throws IOException {
4175      assert mutation instanceof Increment || mutation instanceof Append;
4176      byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
4177      List<Pair<ExtendedCell, ExtendedCell>> cellPairs = new ArrayList<>(deltas.size());
4178
4179      // Sort the cells so that they match the order that they appear in the Get results.
4180      // Otherwise, we won't be able to find the existing values if the cells are not specified
4181      // in order by the client since cells are in an array list.
4182      deltas.sort(store.getComparator());
4183
4184      // Get previous values for all columns in this family.
4185      Get get = new Get(mutation.getRow());
4186      for (ExtendedCell cell : deltas) {
4187        get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
4188      }
4189      TimeRange tr;
4190      if (mutation instanceof Increment) {
4191        tr = ((Increment) mutation).getTimeRange();
4192      } else {
4193        tr = ((Append) mutation).getTimeRange();
4194      }
4195
4196      if (tr != null) {
4197        get.setTimeRange(tr.getMin(), tr.getMax());
4198      }
4199
4200      try (RegionScanner scanner = region.getScanner(new Scan(get))) {
4201        // NOTE: Please don't use HRegion.get() instead,
4202        // because it will copy cells to heap. See HBASE-26036
4203        List<ExtendedCell> currentValues = new ArrayList<>();
4204        scanner.next(currentValues);
4205        // Iterate the input columns and update existing values if they were found, otherwise
4206        // add new column initialized to the delta amount
4207        int currentValuesIndex = 0;
4208        for (int i = 0; i < deltas.size(); i++) {
4209          ExtendedCell delta = deltas.get(i);
4210          ExtendedCell currentValue = null;
4211          if (
4212            currentValuesIndex < currentValues.size()
4213              && CellUtil.matchingQualifier(currentValues.get(currentValuesIndex), delta)
4214          ) {
4215            currentValue = currentValues.get(currentValuesIndex);
4216            if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
4217              currentValuesIndex++;
4218            }
4219          }
4220          // Switch on whether this an increment or an append building the new Cell to apply.
4221          ExtendedCell newCell;
4222          if (mutation instanceof Increment) {
4223            long deltaAmount = getLongValue(delta);
4224            final long newValue =
4225              currentValue == null ? deltaAmount : getLongValue(currentValue) + deltaAmount;
4226            newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4227              (oldCell) -> Bytes.toBytes(newValue));
4228          } else {
4229            newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4230              (oldCell) -> ByteBuffer
4231                .wrap(new byte[delta.getValueLength() + oldCell.getValueLength()])
4232                .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
4233                .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength())
4234                .array());
4235          }
4236          if (region.maxCellSize > 0) {
4237            int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell);
4238            if (newCellSize > region.maxCellSize) {
4239              String msg = "Cell with size " + newCellSize + " exceeds limit of "
4240                + region.maxCellSize + " bytes in region " + this;
4241              LOG.debug(msg);
4242              throw new DoNotRetryIOException(msg);
4243            }
4244          }
4245          cellPairs.add(new Pair<>(currentValue, newCell));
4246          // Add to results to get returned to the Client. If null, cilent does not want results.
4247          if (results != null) {
4248            results.add(newCell);
4249          }
4250        }
4251        // Give coprocessors a chance to update the new cells before apply to WAL or memstore
4252        if (region.coprocessorHost != null) {
4253          // Here the operation must be increment or append.
4254          cellPairs = mutation instanceof Increment
4255            ? region.coprocessorHost.postIncrementBeforeWAL(mutation, (List) cellPairs)
4256            : region.coprocessorHost.postAppendBeforeWAL(mutation, (List) cellPairs);
4257        }
4258      }
4259      return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList());
4260    }
4261
4262    private static ExtendedCell reckonDelta(final ExtendedCell delta,
4263      final ExtendedCell currentCell, final byte[] columnFamily, final long now, Mutation mutation,
4264      Function<ExtendedCell, byte[]> supplier) throws IOException {
4265      // Forward any tags found on the delta.
4266      List<Tag> tags = TagUtil.carryForwardTags(delta);
4267      if (currentCell != null) {
4268        tags = TagUtil.carryForwardTags(tags, currentCell);
4269        tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4270        byte[] newValue = supplier.apply(currentCell);
4271        return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
4272          .setRow(mutation.getRow(), 0, mutation.getRow().length)
4273          .setFamily(columnFamily, 0, columnFamily.length)
4274          // copy the qualifier if the cell is located in shared memory.
4275          .setQualifier(CellUtil.cloneQualifier(delta))
4276          .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now))
4277          .setType(KeyValue.Type.Put.getCode()).setValue(newValue, 0, newValue.length)
4278          .setTags(TagUtil.fromList(tags)).build();
4279      } else {
4280        tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4281        PrivateCellUtil.updateLatestStamp(delta, now);
4282        ExtendedCell deltaCell = (ExtendedCell) delta;
4283        return CollectionUtils.isEmpty(tags)
4284          ? deltaCell
4285          : PrivateCellUtil.createCell(deltaCell, tags);
4286      }
4287    }
4288
4289    /** Returns Get the long out of the passed in Cell */
4290    private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
4291      int len = cell.getValueLength();
4292      if (len != Bytes.SIZEOF_LONG) {
4293        // throw DoNotRetryIOException instead of IllegalArgumentException
4294        throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
4295      }
4296      return PrivateCellUtil.getValueAsLong(cell);
4297    }
4298
4299    @Override
4300    public List<Pair<NonceKey, WALEdit>>
4301      buildWALEdits(final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
4302      List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp);
4303      // for MutationBatchOperation, more than one nonce is not allowed
4304      if (walEdits.size() > 1) {
4305        throw new IOException("Found multiple nonce keys per batch!");
4306      }
4307      return walEdits;
4308    }
4309
4310    /**
4311     * Here is for HBASE-26993,in order to make the new framework for region replication could work
4312     * for SKIP_WAL, we save the {@link Mutation} which {@link Mutation#getDurability} is
4313     * {@link Durability#SKIP_WAL} in miniBatchOp.
4314     */
4315    @Override
4316    protected void cacheSkipWALMutationForRegionReplication(
4317      MiniBatchOperationInProgress<Mutation> miniBatchOp,
4318      List<Pair<NonceKey, WALEdit>> nonceKeyAndWALEdits,
4319      Map<byte[], List<ExtendedCell>> familyCellMap) {
4320      if (!this.regionReplicateEnable) {
4321        return;
4322      }
4323
4324      WALEdit walEditForReplicateIfExistsSkipWAL =
4325        miniBatchOp.getWalEditForReplicateIfExistsSkipWAL();
4326      /**
4327       * When there is a SKIP_WAL {@link Mutation},we create a new {@link WALEdit} for replicating
4328       * to region replica,first we fill the existing {@link WALEdit} to it and then add the
4329       * {@link Mutation} which is SKIP_WAL to it.
4330       */
4331      if (walEditForReplicateIfExistsSkipWAL == null) {
4332        walEditForReplicateIfExistsSkipWAL =
4333          this.createWALEditForReplicateSkipWAL(miniBatchOp, nonceKeyAndWALEdits);
4334        miniBatchOp.setWalEditForReplicateIfExistsSkipWAL(walEditForReplicateIfExistsSkipWAL);
4335      }
4336      WALEditInternalHelper.addMap(walEditForReplicateIfExistsSkipWAL, familyCellMap);
4337
4338    }
4339
4340    private WALEdit createWALEditForReplicateSkipWAL(
4341      MiniBatchOperationInProgress<Mutation> miniBatchOp,
4342      List<Pair<NonceKey, WALEdit>> nonceKeyAndWALEdits) {
4343      if (nonceKeyAndWALEdits.isEmpty()) {
4344        return this.createWALEdit(miniBatchOp);
4345      }
4346      // for MutationBatchOperation, more than one nonce is not allowed
4347      assert nonceKeyAndWALEdits.size() == 1;
4348      WALEdit currentWALEdit = nonceKeyAndWALEdits.get(0).getSecond();
4349      return new WALEdit(currentWALEdit);
4350    }
4351
4352    @Override
4353    protected void addNonSkipWALMutationsToWALEdit(
4354      final MiniBatchOperationInProgress<Mutation> miniBatchOp, WALEdit walEdit,
4355      List<ExtendedCell> cellsFromCP, Map<byte[], List<ExtendedCell>> familyCellMap) {
4356      super.addNonSkipWALMutationsToWALEdit(miniBatchOp, walEdit, cellsFromCP, familyCellMap);
4357      WALEdit walEditForReplicateIfExistsSkipWAL =
4358        miniBatchOp.getWalEditForReplicateIfExistsSkipWAL();
4359      if (walEditForReplicateIfExistsSkipWAL == null) {
4360        return;
4361      }
4362      /**
4363       * When walEditForReplicateIfExistsSkipWAL is not null,it means there exists SKIP_WAL
4364       * {@link Mutation} and we create a new {@link WALEdit} in
4365       * {@link MutationBatchOperation#cacheSkipWALMutationForReplicateRegionReplica} for
4366       * replicating to region replica, so here we also add non SKIP_WAL{@link Mutation}s to
4367       * walEditForReplicateIfExistsSkipWAL.
4368       */
4369      doAddCellsToWALEdit(walEditForReplicateIfExistsSkipWAL, cellsFromCP, familyCellMap);
4370    }
4371
4372    @Override
4373    public WriteEntry writeMiniBatchOperationsToMemStore(
4374      final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry,
4375      long now) throws IOException {
4376      boolean newWriteEntry = false;
4377      if (writeEntry == null) {
4378        writeEntry = region.mvcc.begin();
4379        newWriteEntry = true;
4380      }
4381      super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber());
4382      if (newWriteEntry) {
4383        /**
4384         * Here is for HBASE-26993 case 2,all {@link Mutation}s are {@link Durability#SKIP_WAL}. In
4385         * order to make the new framework for region replication could work for SKIP_WAL,because
4386         * there is no {@link RegionReplicationSink#add} attached in {@link HRegion#doWALAppend},so
4387         * here we get {@link WALEdit} from
4388         * {@link MiniBatchOperationInProgress#getWalEditForReplicateIfExistsSkipWAL} and attach
4389         * {@link RegionReplicationSink#add} to the new mvcc writeEntry.
4390         */
4391        attachRegionReplicationToMVCCEntry(miniBatchOp, writeEntry, now);
4392      }
4393      return writeEntry;
4394    }
4395
4396    private WALKeyImpl createWALKey(long now) {
4397      // for MutationBatchOperation,isReplay is false.
4398      return this.region.createWALKeyForWALAppend(false, this, now, this.nonceGroup, this.nonce);
4399    }
4400
4401    /**
4402     * Create {@link WALKeyImpl} and get {@link WALEdit} from miniBatchOp and attach
4403     * {@link RegionReplicationSink#add} to the mvccWriteEntry.
4404     */
4405    private void attachRegionReplicationToMVCCEntry(
4406      final MiniBatchOperationInProgress<Mutation> miniBatchOp, WriteEntry mvccWriteEntry, long now)
4407      throws IOException {
4408      if (!this.regionReplicateEnable) {
4409        return;
4410      }
4411      assert !mvccWriteEntry.getCompletionAction().isPresent();
4412
4413      final WALKeyImpl walKey = this.createWALKey(now);
4414      walKey.setWriteEntry(mvccWriteEntry);
4415      region.doAttachReplicateRegionReplicaAction(walKey,
4416        miniBatchOp.getWalEditForReplicateIfExistsSkipWAL(), mvccWriteEntry);
4417    }
4418
4419    @Override
4420    public void completeMiniBatchOperations(
4421      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4422      throws IOException {
4423      // TODO: can it be done after completing mvcc?
4424      // calling the post CP hook for batch mutation
4425      if (region.coprocessorHost != null) {
4426        region.coprocessorHost.postBatchMutate(miniBatchOp);
4427      }
4428      super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4429
4430      if (nonce != HConstants.NO_NONCE) {
4431        if (region.rsServices != null && region.rsServices.getNonceManager() != null) {
4432          region.rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
4433            writeEntry.getWriteNumber());
4434        }
4435      }
4436    }
4437
4438    @Override
4439    public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4440      final WALEdit walEdit, boolean success) throws IOException {
4441
4442      super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success);
4443      if (miniBatchOp != null) {
4444        // synced so that the coprocessor contract is adhered to.
4445        if (region.coprocessorHost != null) {
4446          visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4447            // only for successful puts/deletes/increments/appends
4448            if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) {
4449              Mutation m = getMutation(i);
4450              if (m instanceof Put) {
4451                region.coprocessorHost.postPut((Put) m, walEdit);
4452              } else if (m instanceof Delete) {
4453                region.coprocessorHost.postDelete((Delete) m, walEdit);
4454              } else if (m instanceof Increment) {
4455                Result result =
4456                  region.getCoprocessorHost().postIncrement((Increment) m, results[i], walEdit);
4457                if (result != results[i]) {
4458                  retCodeDetails[i] =
4459                    new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4460                }
4461              } else if (m instanceof Append) {
4462                Result result =
4463                  region.getCoprocessorHost().postAppend((Append) m, results[i], walEdit);
4464                if (result != results[i]) {
4465                  retCodeDetails[i] =
4466                    new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4467                }
4468              }
4469            }
4470            return true;
4471          });
4472        }
4473
4474        // For nonce operations
4475        if (canProceed && nonce != HConstants.NO_NONCE) {
4476          boolean[] areAllIncrementsAndAppendsSuccessful = new boolean[] { true };
4477          visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4478            Mutation mutation = getMutation(i);
4479            if (mutation instanceof Increment || mutation instanceof Append) {
4480              if (retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
4481                areAllIncrementsAndAppendsSuccessful[0] = false;
4482                return false;
4483              }
4484            }
4485            return true;
4486          });
4487          endNonceOperation(areAllIncrementsAndAppendsSuccessful[0]);
4488        }
4489
4490        // See if the column families were consistent through the whole thing.
4491        // if they were then keep them. If they were not then pass a null.
4492        // null will be treated as unknown.
4493        // Total time taken might be involving Puts, Deletes, Increments and Appends.
4494        // Split the time for puts and deletes based on the total number of Puts, Deletes,
4495        // Increments and Appends.
4496        if (region.metricsRegion != null) {
4497          if (miniBatchOp.getNumOfPuts() > 0) {
4498            // There were some Puts in the batch.
4499            region.metricsRegion.updatePut();
4500          }
4501          if (miniBatchOp.getNumOfDeletes() > 0) {
4502            // There were some Deletes in the batch.
4503            region.metricsRegion.updateDelete();
4504          }
4505          if (miniBatchOp.getNumOfIncrements() > 0) {
4506            // There were some Increments in the batch.
4507            region.metricsRegion.updateIncrement();
4508          }
4509          if (miniBatchOp.getNumOfAppends() > 0) {
4510            // There were some Appends in the batch.
4511            region.metricsRegion.updateAppend();
4512          }
4513        }
4514      }
4515
4516      if (region.coprocessorHost != null) {
4517        // call the coprocessor hook to do any finalization steps after the put is done
4518        region.coprocessorHost.postBatchMutateIndispensably(
4519          miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success);
4520      }
4521    }
4522
4523    /**
4524     * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch
4525     * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2:
4526     *                count of increments and 3: count of appends
4527     */
4528    private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics)
4529      throws IOException {
4530      Mutation m = getMutation(index);
4531      if (m instanceof Put) {
4532        if (region.coprocessorHost.prePut((Put) m, walEdit)) {
4533          // pre hook says skip this Put
4534          // mark as success and skip in doMiniBatchMutation
4535          metrics[0]++;
4536          retCodeDetails[index] = OperationStatus.SUCCESS;
4537        }
4538      } else if (m instanceof Delete) {
4539        Delete curDel = (Delete) m;
4540        if (curDel.getFamilyCellMap().isEmpty()) {
4541          // handle deleting a row case
4542          // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook.
4543          // Can this be avoided?
4544          region.prepareDelete(curDel);
4545        }
4546        if (region.coprocessorHost.preDelete(curDel, walEdit)) {
4547          // pre hook says skip this Delete
4548          // mark as success and skip in doMiniBatchMutation
4549          metrics[1]++;
4550          retCodeDetails[index] = OperationStatus.SUCCESS;
4551        }
4552      } else if (m instanceof Increment) {
4553        Increment increment = (Increment) m;
4554        Result result = region.coprocessorHost.preIncrement(increment, walEdit);
4555        if (result != null) {
4556          // pre hook says skip this Increment
4557          // mark as success and skip in doMiniBatchMutation
4558          metrics[2]++;
4559          retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4560        }
4561      } else if (m instanceof Append) {
4562        Append append = (Append) m;
4563        Result result = region.coprocessorHost.preAppend(append, walEdit);
4564        if (result != null) {
4565          // pre hook says skip this Append
4566          // mark as success and skip in doMiniBatchMutation
4567          metrics[3]++;
4568          retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4569        }
4570      } else {
4571        String msg = "Put/Delete/Increment/Append mutations only supported in a batch";
4572        retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg);
4573        if (isAtomic()) { // fail, atomic means all or none
4574          throw new IOException(msg);
4575        }
4576      }
4577    }
4578
4579    // TODO Support Increment/Append operations
4580    private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp,
4581      final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException {
4582      visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> {
4583        // we pass (i - firstIndex) below since the call expects a relative index
4584        Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess);
4585        if (cpMutations == null) {
4586          return true;
4587        }
4588        // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
4589        Mutation mutation = getMutation(i);
4590        for (Mutation cpMutation : cpMutations) {
4591          this.checkAndPrepareMutation(cpMutation, timestamp);
4592
4593          // Acquire row locks. If not, the whole batch will fail.
4594          acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null));
4595
4596          // Returned mutations from coprocessor correspond to the Mutation at index i. We can
4597          // directly add the cells from those mutations to the familyMaps of this mutation.
4598          Map<byte[], List<ExtendedCell>> cpFamilyMap =
4599            ClientInternalHelper.getExtendedFamilyCellMap(cpMutation);
4600          region.rewriteCellTags(cpFamilyMap, mutation);
4601          // will get added to the memStore later
4602          mergeFamilyMaps(familyCellMaps[i], cpFamilyMap);
4603
4604          // The durability of returned mutation is replaced by the corresponding mutation.
4605          // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the
4606          // cells of returned mutation.
4607          if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
4608            for (List<ExtendedCell> cells : cpFamilyMap.values()) {
4609              miniBatchOp.addCellCount(cells.size());
4610            }
4611          }
4612        }
4613        return true;
4614      });
4615    }
4616
4617    private void mergeFamilyMaps(Map<byte[], List<ExtendedCell>> familyMap,
4618      Map<byte[], List<ExtendedCell>> toBeMerged) {
4619      for (Map.Entry<byte[], List<ExtendedCell>> entry : toBeMerged.entrySet()) {
4620        List<ExtendedCell> cells = familyMap.get(entry.getKey());
4621        if (cells == null) {
4622          familyMap.put(entry.getKey(), entry.getValue());
4623        } else {
4624          cells.addAll(entry.getValue());
4625        }
4626      }
4627    }
4628  }
4629
4630  /**
4631   * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most
4632   * of the logic is same.
4633   * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we will not use this operation to apply
4634   *             edits at secondary replica side.
4635   */
4636  @Deprecated
4637  private static final class ReplayBatchOperation extends BatchOperation<MutationReplay> {
4638
4639    private long origLogSeqNum = 0;
4640
4641    public ReplayBatchOperation(final HRegion region, MutationReplay[] operations,
4642      long origLogSeqNum) {
4643      super(region, operations);
4644      this.origLogSeqNum = origLogSeqNum;
4645    }
4646
4647    @Override
4648    public Mutation getMutation(int index) {
4649      return this.operations[index].mutation;
4650    }
4651
4652    @Override
4653    public long getNonceGroup(int index) {
4654      return this.operations[index].nonceGroup;
4655    }
4656
4657    @Override
4658    public long getNonce(int index) {
4659      return this.operations[index].nonce;
4660    }
4661
4662    @Override
4663    public Mutation[] getMutationsForCoprocs() {
4664      return null;
4665    }
4666
4667    @Override
4668    public boolean isInReplay() {
4669      return true;
4670    }
4671
4672    @Override
4673    public long getOrigLogSeqNum() {
4674      return this.origLogSeqNum;
4675    }
4676
4677    @Override
4678    public void startRegionOperation() throws IOException {
4679      region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4680    }
4681
4682    @Override
4683    public void closeRegionOperation() throws IOException {
4684      region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4685    }
4686
4687    /**
4688     * During replay, there could exist column families which are removed between region server
4689     * failure and replay
4690     */
4691    @Override
4692    protected void checkAndPreparePut(Put p) throws IOException {
4693      Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap();
4694      List<byte[]> nonExistentList = null;
4695      for (byte[] family : familyCellMap.keySet()) {
4696        if (!region.htableDescriptor.hasColumnFamily(family)) {
4697          if (nonExistentList == null) {
4698            nonExistentList = new ArrayList<>();
4699          }
4700          nonExistentList.add(family);
4701        }
4702      }
4703      if (nonExistentList != null) {
4704        for (byte[] family : nonExistentList) {
4705          // Perhaps schema was changed between crash and replay
4706          LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this);
4707          familyCellMap.remove(family);
4708        }
4709      }
4710    }
4711
4712    @Override
4713    public void checkAndPrepare() throws IOException {
4714      long now = EnvironmentEdgeManager.currentTime();
4715      visitBatchOperations(true, this.size(), (int index) -> {
4716        checkAndPrepareMutation(index, now);
4717        return true;
4718      });
4719    }
4720
4721    @Override
4722    public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4723      long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
4724      visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
4725        // update cell count
4726        for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) {
4727          miniBatchOp.addCellCount(cells.size());
4728        }
4729        return true;
4730      });
4731    }
4732
4733    @Override
4734    public WriteEntry writeMiniBatchOperationsToMemStore(
4735      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry,
4736      long now) throws IOException {
4737      super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum());
4738      return writeEntry;
4739    }
4740
4741    @Override
4742    public void completeMiniBatchOperations(
4743      final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4744      throws IOException {
4745      super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4746      region.mvcc.advanceTo(getOrigLogSeqNum());
4747    }
4748
4749    @Override
4750    protected void cacheSkipWALMutationForRegionReplication(
4751      MiniBatchOperationInProgress<Mutation> miniBatchOp, List<Pair<NonceKey, WALEdit>> walEdits,
4752      Map<byte[], List<ExtendedCell>> familyCellMap) {
4753      // There is no action to do if current region is secondary replica
4754    }
4755
4756  }
4757
4758  public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup,
4759    long nonce) throws IOException {
4760    // As it stands, this is used for 3 things
4761    // * batchMutate with single mutation - put/delete/increment/append, separate or from
4762    // checkAndMutate.
4763    // * coprocessor calls (see ex. BulkDeleteEndpoint).
4764    // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
4765    return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce));
4766  }
4767
4768  @Override
4769  public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
4770    // If the mutations has any Increment/Append operations, we need to do batchMutate atomically
4771    boolean atomic =
4772      Arrays.stream(mutations).anyMatch(m -> m instanceof Increment || m instanceof Append);
4773    return batchMutate(mutations, atomic);
4774  }
4775
4776  OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException {
4777    return TraceUtil.trace(
4778      () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
4779      () -> createRegionSpan("Region.batchMutate"));
4780  }
4781
4782  /**
4783   * @deprecated Since 3.0.0, will be removed in 4.0.0. Now we use
4784   *             {@link #replayWALEntry(WALEntry, CellScanner)} for replaying edits at secondary
4785   *             replica side.
4786   */
4787  @Deprecated
4788  OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId) throws IOException {
4789    if (
4790      !RegionReplicaUtil.isDefaultReplica(getRegionInfo())
4791        && replaySeqId < lastReplayedOpenRegionSeqId
4792    ) {
4793      // if it is a secondary replica we should ignore these entries silently
4794      // since they are coming out of order
4795      if (LOG.isTraceEnabled()) {
4796        LOG.trace(getRegionInfo().getEncodedName() + " : " + "Skipping " + mutations.length
4797          + " mutations with replaySeqId=" + replaySeqId
4798          + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
4799        for (MutationReplay mut : mutations) {
4800          LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
4801        }
4802      }
4803
4804      OperationStatus[] statuses = new OperationStatus[mutations.length];
4805      for (int i = 0; i < statuses.length; i++) {
4806        statuses[i] = OperationStatus.SUCCESS;
4807      }
4808      return statuses;
4809    }
4810    return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId));
4811  }
4812
4813  /**
4814   * Perform a batch of mutations.
4815   * <p/>
4816   * Operations in a batch are stored with highest durability specified of for all operations in a
4817   * batch, except for {@link Durability#SKIP_WAL}.
4818   * <p/>
4819   * This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with
4820   * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with
4821   * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch and
4822   * mutation batch is very similar, lot of code is shared by providing generic methods in base
4823   * class {@link BatchOperation}. The logic for this method and
4824   * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which are
4825   * overridden by derived classes to implement special behavior.
4826   * @param batchOp contains the list of mutations
4827   * @return an array of OperationStatus which internally contains the OperationStatusCode and the
4828   *         exceptionMessage if any.
4829   * @throws IOException if an IO problem is encountered
4830   */
4831  private OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException {
4832    boolean initialized = false;
4833    batchOp.startRegionOperation();
4834    try {
4835      while (!batchOp.isDone()) {
4836        if (!batchOp.isInReplay()) {
4837          checkReadOnly();
4838        }
4839        checkResources();
4840
4841        if (!initialized) {
4842          this.writeRequestsCount.add(batchOp.size());
4843          // validate and prepare batch for write, for MutationBatchOperation it also calls CP
4844          // prePut()/preDelete()/preIncrement()/preAppend() hooks
4845          batchOp.checkAndPrepare();
4846          initialized = true;
4847        }
4848        doMiniBatchMutate(batchOp);
4849        requestFlushIfNeeded();
4850      }
4851    } finally {
4852      if (rsServices != null && rsServices.getMetrics() != null) {
4853        rsServices.getMetrics().updateWriteQueryMeter(this, batchOp.size());
4854      }
4855      batchOp.closeRegionOperation();
4856    }
4857    return batchOp.retCodeDetails;
4858  }
4859
4860  /**
4861   * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])} In here we
4862   * also handle replay of edits on region recover. Also gets change in size brought about by
4863   * applying {@code batchOp}.
4864   */
4865  private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
4866    boolean success = false;
4867    WALEdit walEdit = null;
4868    WriteEntry writeEntry = null;
4869    boolean locked = false;
4870    // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive)
4871    MiniBatchOperationInProgress<Mutation> miniBatchOp = null;
4872    /** Keep track of the locks we hold so we can release them in finally clause */
4873    List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size());
4874
4875    // Check for thread interrupt status in case we have been signaled from
4876    // #interruptRegionOperation.
4877    checkInterrupt();
4878
4879    try {
4880      // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with
4881      // locked rows
4882      miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks);
4883
4884      // We've now grabbed as many mutations off the list as we can
4885      // Ensure we acquire at least one.
4886      if (miniBatchOp.getReadyToWriteCount() <= 0) {
4887        // Nothing to put/delete/increment/append -- an exception in the above such as
4888        // NoSuchColumnFamily?
4889        return;
4890      }
4891
4892      // Check for thread interrupt status in case we have been signaled from
4893      // #interruptRegionOperation. Do it before we take the lock and disable interrupts for
4894      // the WAL append.
4895      checkInterrupt();
4896
4897      lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount());
4898      locked = true;
4899
4900      // From this point until memstore update this operation should not be interrupted.
4901      disableInterrupts();
4902
4903      // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp
4904      // We should record the timestamp only after we have acquired the rowLock,
4905      // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer
4906      // timestamp
4907
4908      long now = EnvironmentEdgeManager.currentTime();
4909      batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks);
4910
4911      // STEP 3. Build WAL edit
4912
4913      List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp);
4914
4915      // STEP 4. Append the WALEdits to WAL and sync.
4916
4917      for (Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) {
4918        Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next();
4919        walEdit = nonceKeyWALEditPair.getSecond();
4920        NonceKey nonceKey = nonceKeyWALEditPair.getFirst();
4921
4922        if (walEdit != null && !walEdit.isEmpty()) {
4923          writeEntry = doWALAppend(walEdit, batchOp, miniBatchOp, now, nonceKey);
4924        }
4925
4926        // Complete mvcc for all but last writeEntry (for replay case)
4927        if (it.hasNext() && writeEntry != null) {
4928          mvcc.complete(writeEntry);
4929          writeEntry = null;
4930        }
4931      }
4932
4933      // STEP 5. Write back to memStore
4934      // NOTE: writeEntry can be null here
4935      writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry, now);
4936
4937      // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and
4938      // complete mvcc for last writeEntry
4939      batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry);
4940      writeEntry = null;
4941      success = true;
4942    } finally {
4943      // Call complete rather than completeAndWait because we probably had error if walKey != null
4944      if (writeEntry != null) mvcc.complete(writeEntry);
4945
4946      if (locked) {
4947        this.updatesLock.readLock().unlock();
4948      }
4949      releaseRowLocks(acquiredRowLocks);
4950
4951      enableInterrupts();
4952
4953      final int finalLastIndexExclusive =
4954        miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size();
4955      final boolean finalSuccess = success;
4956      batchOp.visitBatchOperations(true, finalLastIndexExclusive, (int i) -> {
4957        Mutation mutation = batchOp.getMutation(i);
4958        if (mutation instanceof Increment || mutation instanceof Append) {
4959          if (finalSuccess) {
4960            batchOp.retCodeDetails[i] =
4961              new OperationStatus(OperationStatusCode.SUCCESS, batchOp.results[i]);
4962          } else {
4963            batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
4964          }
4965        } else {
4966          batchOp.retCodeDetails[i] =
4967            finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE;
4968        }
4969        return true;
4970      });
4971
4972      batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess);
4973
4974      batchOp.nextIndexToProcess = finalLastIndexExclusive;
4975    }
4976  }
4977
4978  /**
4979   * Returns effective durability from the passed durability and the table descriptor.
4980   */
4981  private Durability getEffectiveDurability(Durability d) {
4982    return d == Durability.USE_DEFAULT ? this.regionDurability : d;
4983  }
4984
4985  @Override
4986  @Deprecated
4987  public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4988    ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException {
4989    CheckAndMutate checkAndMutate;
4990    try {
4991      CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row)
4992        .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange);
4993      if (mutation instanceof Put) {
4994        checkAndMutate = builder.build((Put) mutation);
4995      } else if (mutation instanceof Delete) {
4996        checkAndMutate = builder.build((Delete) mutation);
4997      } else {
4998        throw new DoNotRetryIOException(
4999          "Unsupported mutate type: " + mutation.getClass().getSimpleName().toUpperCase());
5000      }
5001    } catch (IllegalArgumentException e) {
5002      throw new DoNotRetryIOException(e.getMessage());
5003    }
5004    return checkAndMutate(checkAndMutate).isSuccess();
5005  }
5006
5007  @Override
5008  @Deprecated
5009  public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation)
5010    throws IOException {
5011    CheckAndMutate checkAndMutate;
5012    try {
5013      CheckAndMutate.Builder builder =
5014        CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange);
5015      if (mutation instanceof Put) {
5016        checkAndMutate = builder.build((Put) mutation);
5017      } else if (mutation instanceof Delete) {
5018        checkAndMutate = builder.build((Delete) mutation);
5019      } else {
5020        throw new DoNotRetryIOException(
5021          "Unsupported mutate type: " + mutation.getClass().getSimpleName().toUpperCase());
5022      }
5023    } catch (IllegalArgumentException e) {
5024      throw new DoNotRetryIOException(e.getMessage());
5025    }
5026    return checkAndMutate(checkAndMutate).isSuccess();
5027  }
5028
5029  @Override
5030  @Deprecated
5031  public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
5032    ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException {
5033    CheckAndMutate checkAndMutate;
5034    try {
5035      checkAndMutate = CheckAndMutate.newBuilder(row)
5036        .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm);
5037    } catch (IllegalArgumentException e) {
5038      throw new DoNotRetryIOException(e.getMessage());
5039    }
5040    return checkAndMutate(checkAndMutate).isSuccess();
5041  }
5042
5043  @Override
5044  @Deprecated
5045  public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm)
5046    throws IOException {
5047    CheckAndMutate checkAndMutate;
5048    try {
5049      checkAndMutate =
5050        CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange).build(rm);
5051    } catch (IllegalArgumentException e) {
5052      throw new DoNotRetryIOException(e.getMessage());
5053    }
5054    return checkAndMutate(checkAndMutate).isSuccess();
5055  }
5056
5057  @Override
5058  public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {
5059    return checkAndMutate(checkAndMutate, HConstants.NO_NONCE, HConstants.NO_NONCE);
5060  }
5061
5062  public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup,
5063    long nonce) throws IOException {
5064    return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce),
5065      () -> createRegionSpan("Region.checkAndMutate"));
5066  }
5067
5068  private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate,
5069    long nonceGroup, long nonce) throws IOException {
5070    byte[] row = checkAndMutate.getRow();
5071    Filter filter = null;
5072    byte[] family = null;
5073    byte[] qualifier = null;
5074    CompareOperator op = null;
5075    ByteArrayComparable comparator = null;
5076    if (checkAndMutate.hasFilter()) {
5077      filter = checkAndMutate.getFilter();
5078    } else {
5079      family = checkAndMutate.getFamily();
5080      qualifier = checkAndMutate.getQualifier();
5081      op = checkAndMutate.getCompareOp();
5082      comparator = new BinaryComparator(checkAndMutate.getValue());
5083    }
5084    TimeRange timeRange = checkAndMutate.getTimeRange();
5085
5086    Mutation mutation = null;
5087    RowMutations rowMutations = null;
5088    if (checkAndMutate.getAction() instanceof Mutation) {
5089      mutation = (Mutation) checkAndMutate.getAction();
5090    } else {
5091      rowMutations = (RowMutations) checkAndMutate.getAction();
5092    }
5093
5094    if (mutation != null) {
5095      checkMutationType(mutation);
5096      checkRow(mutation, row);
5097    } else {
5098      checkRow(rowMutations, row);
5099    }
5100    checkReadOnly();
5101    // TODO, add check for value length also move this check to the client
5102    checkResources();
5103    startRegionOperation();
5104    try {
5105      Get get = new Get(row);
5106      if (family != null) {
5107        checkFamily(family);
5108        get.addColumn(family, qualifier);
5109      }
5110      if (filter != null) {
5111        get.setFilter(filter);
5112      }
5113      if (timeRange != null) {
5114        get.setTimeRange(timeRange.getMin(), timeRange.getMax());
5115      }
5116      // Lock row - note that doBatchMutate will relock this row if called
5117      checkRow(row, "doCheckAndRowMutate");
5118      RowLock rowLock = getRowLock(get.getRow(), false, null);
5119      try {
5120        if (this.getCoprocessorHost() != null) {
5121          CheckAndMutateResult result =
5122            getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate);
5123          if (result != null) {
5124            return result;
5125          }
5126        }
5127
5128        // NOTE: We used to wait here until mvcc caught up: mvcc.await();
5129        // Supposition is that now all changes are done under row locks, then when we go to read,
5130        // we'll get the latest on this row.
5131        boolean matches = false;
5132        long cellTs = 0;
5133        QueryMetrics metrics = null;
5134        try (RegionScannerImpl scanner = getScanner(new Scan(get))) {
5135          // NOTE: Please don't use HRegion.get() instead,
5136          // because it will copy cells to heap. See HBASE-26036
5137          List<ExtendedCell> result = new ArrayList<>(1);
5138          scanner.next(result);
5139          if (filter != null) {
5140            if (!result.isEmpty()) {
5141              matches = true;
5142              cellTs = result.get(0).getTimestamp();
5143            }
5144          } else {
5145            boolean valueIsNull =
5146              comparator.getValue() == null || comparator.getValue().length == 0;
5147            if (result.isEmpty() && valueIsNull) {
5148              matches = op != CompareOperator.NOT_EQUAL;
5149            } else if (result.size() > 0 && valueIsNull) {
5150              matches = (result.get(0).getValueLength() == 0) == (op != CompareOperator.NOT_EQUAL);
5151              cellTs = result.get(0).getTimestamp();
5152            } else if (result.size() == 1) {
5153              ExtendedCell kv = result.get(0);
5154              cellTs = kv.getTimestamp();
5155              int compareResult = PrivateCellUtil.compareValue(kv, comparator);
5156              matches = matches(op, compareResult);
5157            }
5158          }
5159          if (checkAndMutate.isQueryMetricsEnabled()) {
5160            metrics = new QueryMetrics(scanner.getContext().getBlockSizeProgress());
5161          }
5162        }
5163
5164        // If matches, perform the mutation or the rowMutations
5165        if (matches) {
5166          // We have acquired the row lock already. If the system clock is NOT monotonically
5167          // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
5168          // larger timestamp than what was observed via Get. doBatchMutate already does this, but
5169          // there is no way to pass the cellTs. See HBASE-14054.
5170          long now = EnvironmentEdgeManager.currentTime();
5171          long ts = Math.max(now, cellTs); // ensure write is not eclipsed
5172          byte[] byteTs = Bytes.toBytes(ts);
5173          if (mutation != null) {
5174            if (mutation instanceof Put) {
5175              updateCellTimestamps(ClientInternalHelper.getExtendedFamilyCellMap(mutation).values(),
5176                byteTs);
5177            }
5178            // And else 'delete' is not needed since it already does a second get, and sets the
5179            // timestamp from get (see prepareDeleteTimestamps).
5180          } else {
5181            for (Mutation m : rowMutations.getMutations()) {
5182              if (m instanceof Put) {
5183                updateCellTimestamps(ClientInternalHelper.getExtendedFamilyCellMap(m).values(),
5184                  byteTs);
5185              }
5186            }
5187            // And else 'delete' is not needed since it already does a second get, and sets the
5188            // timestamp from get (see prepareDeleteTimestamps).
5189          }
5190          // All edits for the given row (across all column families) must happen atomically.
5191          Result r;
5192          if (mutation != null) {
5193            r = mutate(mutation, true, nonceGroup, nonce).getResult();
5194          } else {
5195            r = mutateRow(rowMutations, nonceGroup, nonce);
5196          }
5197          this.checkAndMutateChecksPassed.increment();
5198          return new CheckAndMutateResult(true, r).setMetrics(metrics);
5199        }
5200        this.checkAndMutateChecksFailed.increment();
5201        return new CheckAndMutateResult(false, null).setMetrics(metrics);
5202      } finally {
5203        rowLock.release();
5204      }
5205    } finally {
5206      closeRegionOperation();
5207    }
5208  }
5209
5210  private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException {
5211    if (
5212      !(mutation instanceof Put) && !(mutation instanceof Delete)
5213        && !(mutation instanceof Increment) && !(mutation instanceof Append)
5214    ) {
5215      throw new org.apache.hadoop.hbase.DoNotRetryIOException(
5216        "Action must be Put or Delete or Increment or Delete");
5217    }
5218  }
5219
5220  private void checkRow(final Row action, final byte[] row) throws DoNotRetryIOException {
5221    if (!Bytes.equals(row, action.getRow())) {
5222      throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match");
5223    }
5224  }
5225
5226  private boolean matches(final CompareOperator op, final int compareResult) {
5227    boolean matches = false;
5228    switch (op) {
5229      case LESS:
5230        matches = compareResult < 0;
5231        break;
5232      case LESS_OR_EQUAL:
5233        matches = compareResult <= 0;
5234        break;
5235      case EQUAL:
5236        matches = compareResult == 0;
5237        break;
5238      case NOT_EQUAL:
5239        matches = compareResult != 0;
5240        break;
5241      case GREATER_OR_EQUAL:
5242        matches = compareResult >= 0;
5243        break;
5244      case GREATER:
5245        matches = compareResult > 0;
5246        break;
5247      default:
5248        throw new RuntimeException("Unknown Compare op " + op.name());
5249    }
5250    return matches;
5251  }
5252
5253  private OperationStatus mutate(Mutation mutation) throws IOException {
5254    return mutate(mutation, false);
5255  }
5256
5257  private OperationStatus mutate(Mutation mutation, boolean atomic) throws IOException {
5258    return mutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE);
5259  }
5260
5261  private OperationStatus mutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce)
5262    throws IOException {
5263    OperationStatus[] status =
5264      this.batchMutate(new Mutation[] { mutation }, atomic, nonceGroup, nonce);
5265    if (status[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
5266      throw new FailedSanityCheckException(status[0].getExceptionMsg());
5267    } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
5268      throw new NoSuchColumnFamilyException(status[0].getExceptionMsg());
5269    } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.STORE_TOO_BUSY)) {
5270      throw new RegionTooBusyException(status[0].getExceptionMsg());
5271    }
5272    return status[0];
5273  }
5274
5275  /**
5276   * Complete taking the snapshot on the region. Writes the region info and adds references to the
5277   * working snapshot directory. TODO for api consistency, consider adding another version with no
5278   * {@link ForeignExceptionSnare} arg. (In the future other cancellable HRegion methods could
5279   * eventually add a {@link ForeignExceptionSnare}, or we could do something fancier).
5280   * @param desc     snapshot description object
5281   * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to bail
5282   *                 out. This is allowed to be null and will just be ignored in that case.
5283   * @throws IOException if there is an external or internal error causing the snapshot to fail
5284   */
5285  public void addRegionToSnapshot(SnapshotDescription desc, ForeignExceptionSnare exnSnare)
5286    throws IOException {
5287    Path rootDir = CommonFSUtils.getRootDir(conf);
5288    Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
5289
5290    SnapshotManifest manifest =
5291      SnapshotManifest.create(conf, getFilesystem(), snapshotDir, desc, exnSnare);
5292    manifest.addRegion(this);
5293  }
5294
5295  private void updateSequenceId(final Iterable<List<ExtendedCell>> cellItr, final long sequenceId)
5296    throws IOException {
5297    for (List<ExtendedCell> cells : cellItr) {
5298      if (cells == null) {
5299        return;
5300      }
5301      for (ExtendedCell cell : cells) {
5302        cell.setSequenceId(sequenceId);
5303      }
5304    }
5305  }
5306
5307  /**
5308   * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP}
5309   * provided current timestamp.
5310   */
5311  private static void updateCellTimestamps(final Iterable<List<ExtendedCell>> cellItr,
5312    final byte[] now) throws IOException {
5313    for (List<ExtendedCell> cells : cellItr) {
5314      if (cells == null) {
5315        continue;
5316      }
5317      // Optimization: 'foreach' loop is not used. See:
5318      // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5319      assert cells instanceof RandomAccess;
5320      int listSize = cells.size();
5321      for (int i = 0; i < listSize; i++) {
5322        PrivateCellUtil.updateLatestStamp(cells.get(i), now);
5323      }
5324    }
5325  }
5326
5327  /**
5328   * Possibly rewrite incoming cell tags.
5329   */
5330  private void rewriteCellTags(Map<byte[], List<ExtendedCell>> familyMap, final Mutation m) {
5331    // Check if we have any work to do and early out otherwise
5332    // Update these checks as more logic is added here
5333    if (m.getTTL() == Long.MAX_VALUE) {
5334      return;
5335    }
5336
5337    // From this point we know we have some work to do
5338    for (Map.Entry<byte[], List<ExtendedCell>> e : familyMap.entrySet()) {
5339      List<ExtendedCell> cells = e.getValue();
5340      assert cells instanceof RandomAccess;
5341      int listSize = cells.size();
5342      for (int i = 0; i < listSize; i++) {
5343        ExtendedCell cell = cells.get(i);
5344        List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
5345        newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
5346        // Rewrite the cell with the updated set of tags
5347        cells.set(i, PrivateCellUtil.createCell(cell, newTags));
5348      }
5349    }
5350  }
5351
5352  /**
5353   * Check if resources to support an update.
5354   * <p/>
5355   * We throw RegionTooBusyException if above memstore limit and expect client to retry using some
5356   * kind of backoff
5357   */
5358  private void checkResources() throws RegionTooBusyException {
5359    // If catalog region, do not impose resource constraints or block updates.
5360    if (this.getRegionInfo().isMetaRegion()) {
5361      return;
5362    }
5363
5364    MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
5365    if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) {
5366      blockedRequestsCount.increment();
5367      requestFlush();
5368      // Don't print current limit because it will vary too much. The message is used as a key
5369      // over in RetriesExhaustedWithDetailsException processing.
5370      final String regionName =
5371        this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName();
5372      final String serverName = this.getRegionServerServices() == null
5373        ? "unknown"
5374        : (this.getRegionServerServices().getServerName() == null
5375          ? "unknown"
5376          : this.getRegionServerServices().getServerName().toString());
5377      RegionTooBusyException rtbe = new RegionTooBusyException("Over memstore limit="
5378        + org.apache.hadoop.hbase.procedure2.util.StringUtils.humanSize(this.blockingMemStoreSize)
5379        + ", regionName=" + regionName + ", server=" + serverName);
5380      LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
5381      throw rtbe;
5382    }
5383  }
5384
5385  /**
5386   * @throws IOException Throws exception if region is in read-only mode.
5387   */
5388  private void checkReadOnly() throws IOException {
5389    if (isReadOnly()) {
5390      throw new DoNotRetryIOException("region is read only");
5391    }
5392  }
5393
5394  private void checkReadsEnabled() throws IOException {
5395    if (!this.writestate.readsEnabled) {
5396      throw new IOException(getRegionInfo().getEncodedName()
5397        + ": The region's reads are disabled. Cannot serve the request");
5398    }
5399  }
5400
5401  public void setReadsEnabled(boolean readsEnabled) {
5402    if (readsEnabled && !this.writestate.readsEnabled) {
5403      LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName());
5404    }
5405    this.writestate.setReadsEnabled(readsEnabled);
5406  }
5407
5408  /**
5409   * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
5410   *              set; when set we will run operations that make sense in the increment/append
5411   *              scenario but that do not make sense otherwise.
5412   */
5413  private void applyToMemStore(HStore store, List<ExtendedCell> cells, boolean delta,
5414    MemStoreSizing memstoreAccounting) {
5415    // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!!
5416    boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
5417    if (upsert) {
5418      store.upsert(cells, getSmallestReadPoint(), memstoreAccounting);
5419    } else {
5420      store.add(cells, memstoreAccounting);
5421    }
5422  }
5423
5424  private void checkFamilies(Collection<byte[]> families, Durability durability)
5425    throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5426    for (byte[] family : families) {
5427      checkFamily(family, durability);
5428    }
5429  }
5430
5431  private void checkFamily(final byte[] family, Durability durability)
5432    throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5433    checkFamily(family);
5434    if (
5435      durability.equals(Durability.SKIP_WAL)
5436        && htableDescriptor.getColumnFamily(family).getScope() != HConstants.REPLICATION_SCOPE_LOCAL
5437    ) {
5438      throw new InvalidMutationDurabilityException(
5439        "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family)
5440          + " need replication");
5441    }
5442  }
5443
5444  private void checkFamily(final byte[] family) throws NoSuchColumnFamilyException {
5445    if (!this.htableDescriptor.hasColumnFamily(family)) {
5446      throw new NoSuchColumnFamilyException("Column family " + Bytes.toString(family)
5447        + " does not exist in region " + this + " in table " + this.htableDescriptor);
5448    }
5449  }
5450
5451  /**
5452   * Check the collection of families for valid timestamps
5453   * @param now current timestamp
5454   */
5455  public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
5456    throws FailedSanityCheckException {
5457    if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
5458      return;
5459    }
5460    long maxTs = now + timestampSlop;
5461    for (List<Cell> kvs : familyMap.values()) {
5462      // Optimization: 'foreach' loop is not used. See:
5463      // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5464      assert kvs instanceof RandomAccess;
5465      int listSize = kvs.size();
5466      for (int i = 0; i < listSize; i++) {
5467        Cell cell = kvs.get(i);
5468        // see if the user-side TS is out of range. latest = server-side
5469        long ts = cell.getTimestamp();
5470        if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
5471          throw new FailedSanityCheckException(
5472            "Timestamp for KV out of range " + cell + " (too.new=" + timestampSlop + ")");
5473        }
5474      }
5475    }
5476  }
5477
5478  /*
5479   * @return True if size is over the flush threshold
5480   */
5481  private boolean isFlushSize(MemStoreSize size) {
5482    return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize();
5483  }
5484
5485  private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException {
5486    for (Path file : files) {
5487      if (!fs.delete(file, false)) {
5488        LOG.error("Failed delete of {}", file);
5489      } else {
5490        LOG.debug("Deleted recovered.edits file={}", file);
5491      }
5492    }
5493  }
5494
5495  /**
5496   * Read the edits put under this region by wal splitting process. Put the recovered edits back up
5497   * into this region.
5498   * <p>
5499   * We can ignore any wal message that has a sequence ID that's equal to or lower than minSeqId.
5500   * (Because we know such messages are already reflected in the HFiles.)
5501   * <p>
5502   * While this is running we are putting pressure on memory yet we are outside of our usual
5503   * accounting because we are not yet an onlined region (this stuff is being run as part of Region
5504   * initialization). This means that if we're up against global memory limits, we'll not be flagged
5505   * to flush because we are not online. We can't be flushed by usual mechanisms anyways; we're not
5506   * yet online so our relative sequenceids are not yet aligned with WAL sequenceids -- not till we
5507   * come up online, post processing of split edits.
5508   * <p>
5509   * But to help relieve memory pressure, at least manage our own heap size flushing if are in
5510   * excess of per-region limits. Flushing, though, we have to be careful and avoid using the
5511   * regionserver/wal sequenceid. Its running on a different line to whats going on in here in this
5512   * region context so if we crashed replaying these edits, but in the midst had a flush that used
5513   * the regionserver wal with a sequenceid in excess of whats going on in here in this region and
5514   * with its split editlogs, then we could miss edits the next time we go to recover. So, we have
5515   * to flush inline, using seqids that make sense in a this single region context only -- until we
5516   * online.
5517   * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of the maxSeqId
5518   *                         for the store to be applied, else its skipped.
5519   * @return the sequence id of the last edit added to this region out of the recovered edits log or
5520   *         <code>minSeqId</code> if nothing added from editlogs.
5521   */
5522  long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
5523    final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
5524    long minSeqIdForTheRegion = -1;
5525    for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
5526      if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
5527        minSeqIdForTheRegion = maxSeqIdInStore;
5528      }
5529    }
5530    long seqId = minSeqIdForTheRegion;
5531    String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR);
5532    if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
5533      FileSystem walFS = getWalFileSystem();
5534      FileSystem rootFS = getFilesystem();
5535      Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
5536        getRegionInfo().getEncodedName());
5537      Path regionWALDir = getWALRegionDir();
5538      Path regionDir =
5539        FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
5540
5541      // We made a mistake in HBASE-20734 so we need to do this dirty hack...
5542      NavigableSet<Path> filesUnderWrongRegionWALDir =
5543        WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
5544      seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
5545        filesUnderWrongRegionWALDir, reporter, regionDir));
5546      // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
5547      // under the root dir even if walDir is set.
5548      NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet();
5549      if (!regionWALDir.equals(regionDir)) {
5550        filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir);
5551        seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
5552          filesUnderRootDir, reporter, regionDir));
5553      }
5554
5555      NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir);
5556      seqId = Math.max(seqId,
5557        replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir));
5558      if (seqId > minSeqIdForTheRegion) {
5559        // Then we added some edits to memory. Flush and cleanup split edit files.
5560        internalFlushcache(null, seqId, stores.values(), status, false,
5561          FlushLifeCycleTracker.DUMMY);
5562      }
5563      // Now delete the content of recovered edits. We're done w/ them.
5564      if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
5565        // For debugging data loss issues!
5566        // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
5567        // column family. Have to fake out file type too by casting our recovered.edits as
5568        // storefiles
5569        String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName();
5570        StoreContext storeContext =
5571          StoreContext.getBuilder().withRegionFileSystem(getRegionFileSystem()).build();
5572        StoreFileTracker sft = StoreFileTrackerFactory.create(this.conf, true, storeContext);
5573        Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
5574        for (Path file : files) {
5575          fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true, sft));
5576        }
5577        getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles);
5578      } else {
5579        deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir));
5580        deleteRecoveredEdits(rootFS, filesUnderRootDir);
5581      }
5582    } else {
5583      Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr);
5584      FileSystem fs = recoveredEditsDir.getFileSystem(conf);
5585      FileStatus[] files = fs.listStatus(recoveredEditsDir);
5586      LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length,
5587        recoveredEditsDir);
5588      if (files != null) {
5589        for (FileStatus file : files) {
5590          // it is safe to trust the zero-length in this case because we've been through rename and
5591          // lease recovery in the above.
5592          if (isZeroLengthThenDelete(fs, file, file.getPath())) {
5593            continue;
5594          }
5595          seqId =
5596            Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs));
5597        }
5598      }
5599      if (seqId > minSeqIdForTheRegion) {
5600        // Then we added some edits to memory. Flush and cleanup split edit files.
5601        internalFlushcache(null, seqId, stores.values(), status, false,
5602          FlushLifeCycleTracker.DUMMY);
5603      }
5604      deleteRecoveredEdits(fs,
5605        Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList()));
5606    }
5607
5608    return seqId;
5609  }
5610
5611  private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
5612    final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
5613    throws IOException {
5614    long seqid = minSeqIdForTheRegion;
5615    if (LOG.isDebugEnabled()) {
5616      LOG.debug("Found " + (files == null ? 0 : files.size()) + " recovered edits file(s) under "
5617        + regionDir);
5618    }
5619
5620    if (files == null || files.isEmpty()) {
5621      return minSeqIdForTheRegion;
5622    }
5623
5624    for (Path edits : files) {
5625      if (edits == null || !fs.exists(edits)) {
5626        LOG.warn("Null or non-existent edits file: " + edits);
5627        continue;
5628      }
5629      if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) {
5630        continue;
5631      }
5632
5633      long maxSeqId;
5634      String fileName = edits.getName();
5635      maxSeqId = Math.abs(Long.parseLong(fileName));
5636      if (maxSeqId <= minSeqIdForTheRegion) {
5637        if (LOG.isDebugEnabled()) {
5638          String msg = "Maximum sequenceid for this wal is " + maxSeqId
5639            + " and minimum sequenceid for the region " + this + "  is " + minSeqIdForTheRegion
5640            + ", skipped the whole file, path=" + edits;
5641          LOG.debug(msg);
5642        }
5643        continue;
5644      }
5645
5646      try {
5647        // replay the edits. Replay can return -1 if everything is skipped, only update
5648        // if seqId is greater
5649        seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
5650      } catch (IOException e) {
5651        handleException(fs, edits, e);
5652      }
5653    }
5654    return seqid;
5655  }
5656
5657  private void handleException(FileSystem fs, Path edits, IOException e) throws IOException {
5658    boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
5659      conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
5660    if (conf.get("hbase.skip.errors") != null) {
5661      LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use "
5662        + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
5663    }
5664    if (skipErrors) {
5665      Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits);
5666      LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed "
5667        + edits + " as " + p, e);
5668    } else {
5669      throw e;
5670    }
5671  }
5672
5673  /**
5674   * @param edits            File of recovered edits.
5675   * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger
5676   *                         than this to be replayed for each store.
5677   * @return the sequence id of the last edit added to this region out of the recovered edits log or
5678   *         <code>minSeqId</code> if nothing added from editlogs.
5679   */
5680  private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
5681    final CancelableProgressable reporter, FileSystem fs) throws IOException {
5682    String msg = "Replaying edits from " + edits;
5683    LOG.info(msg);
5684    MonitoredTask status = TaskMonitor.get().createStatus(msg);
5685
5686    status.setStatus("Opening recovered edits");
5687    try (WALStreamReader reader = WALFactory.createStreamReader(fs, edits, conf)) {
5688      long currentEditSeqId = -1;
5689      long currentReplaySeqId = -1;
5690      long firstSeqIdInLog = -1;
5691      long skippedEdits = 0;
5692      long editsCount = 0;
5693      long intervalEdits = 0;
5694      WAL.Entry entry;
5695      HStore store = null;
5696      boolean reported_once = false;
5697      ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
5698
5699      try {
5700        // How many edits seen before we check elapsed time
5701        int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
5702        // How often to send a progress report (default 1/2 master timeout)
5703        int period = this.conf.getInt("hbase.hstore.report.period", 300000);
5704        long lastReport = EnvironmentEdgeManager.currentTime();
5705
5706        if (coprocessorHost != null) {
5707          coprocessorHost.preReplayWALs(this.getRegionInfo(), edits);
5708        }
5709
5710        while ((entry = reader.next()) != null) {
5711          WALKey key = entry.getKey();
5712          WALEdit val = entry.getEdit();
5713
5714          if (ng != null) { // some test, or nonces disabled
5715            ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
5716          }
5717
5718          if (reporter != null) {
5719            intervalEdits += val.size();
5720            if (intervalEdits >= interval) {
5721              // Number of edits interval reached
5722              intervalEdits = 0;
5723              long cur = EnvironmentEdgeManager.currentTime();
5724              if (lastReport + period <= cur) {
5725                status.setStatus(
5726                  "Replaying edits..." + " skipped=" + skippedEdits + " edits=" + editsCount);
5727                // Timeout reached
5728                if (!reporter.progress()) {
5729                  msg = "Progressable reporter failed, stopping replay for region " + this;
5730                  LOG.warn(msg);
5731                  status.abort(msg);
5732                  throw new IOException(msg);
5733                }
5734                reported_once = true;
5735                lastReport = cur;
5736              }
5737            }
5738          }
5739
5740          if (firstSeqIdInLog == -1) {
5741            firstSeqIdInLog = key.getSequenceId();
5742          }
5743          if (currentEditSeqId > key.getSequenceId()) {
5744            // when this condition is true, it means we have a serious defect because we need to
5745            // maintain increasing SeqId for WAL edits per region
5746            LOG.error(getRegionInfo().getEncodedName() + " : " + "Found decreasing SeqId. PreId="
5747              + currentEditSeqId + " key=" + key + "; edit=" + val);
5748          } else {
5749            currentEditSeqId = key.getSequenceId();
5750          }
5751          currentReplaySeqId =
5752            (key.getOrigLogSeqNum() > 0) ? key.getOrigLogSeqNum() : currentEditSeqId;
5753
5754          boolean checkRowWithinBoundary = false;
5755          // Check this edit is for this region.
5756          if (
5757            !Bytes.equals(key.getEncodedRegionName(), this.getRegionInfo().getEncodedNameAsBytes())
5758          ) {
5759            checkRowWithinBoundary = true;
5760          }
5761
5762          boolean flush = false;
5763          MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
5764          for (Cell c : val.getCells()) {
5765            assert c instanceof ExtendedCell;
5766            ExtendedCell cell = (ExtendedCell) c;
5767            // Check this edit is for me. Also, guard against writing the special
5768            // METACOLUMN info such as HBASE::CACHEFLUSH entries
5769            if (WALEdit.isMetaEditFamily(cell)) {
5770              // if region names don't match, skipp replaying compaction marker
5771              if (!checkRowWithinBoundary) {
5772                // this is a special edit, we should handle it
5773                CompactionDescriptor compaction = WALEdit.getCompaction(cell);
5774                if (compaction != null) {
5775                  // replay the compaction
5776                  replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
5777                }
5778              }
5779              skippedEdits++;
5780              continue;
5781            }
5782            // Figure which store the edit is meant for.
5783            if (
5784              store == null
5785                || !CellUtil.matchingFamily(cell, store.getColumnFamilyDescriptor().getName())
5786            ) {
5787              store = getStore(cell);
5788            }
5789            if (store == null) {
5790              // This should never happen. Perhaps schema was changed between
5791              // crash and redeploy?
5792              LOG.warn("No family for cell {} in region {}", cell, this);
5793              skippedEdits++;
5794              continue;
5795            }
5796            if (
5797              checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(), cell.getRowArray(),
5798                cell.getRowOffset(), cell.getRowLength())
5799            ) {
5800              LOG.warn("Row of {} is not within region boundary for region {}", cell, this);
5801              skippedEdits++;
5802              continue;
5803            }
5804            // Now, figure if we should skip this edit.
5805            if (
5806              key.getSequenceId()
5807                  <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor().getName())
5808            ) {
5809              skippedEdits++;
5810              continue;
5811            }
5812            PrivateCellUtil.setSequenceId(cell, currentReplaySeqId);
5813
5814            restoreEdit(store, cell, memStoreSizing);
5815            editsCount++;
5816          }
5817          MemStoreSize mss = memStoreSizing.getMemStoreSize();
5818          incMemStoreSize(mss);
5819          flush = isFlushSize(this.memStoreSizing.getMemStoreSize());
5820          if (flush) {
5821            internalFlushcache(null, currentEditSeqId, stores.values(), status, false,
5822              FlushLifeCycleTracker.DUMMY);
5823          }
5824        }
5825
5826        if (coprocessorHost != null) {
5827          coprocessorHost.postReplayWALs(this.getRegionInfo(), edits);
5828        }
5829      } catch (EOFException eof) {
5830        if (!conf.getBoolean(RECOVERED_EDITS_IGNORE_EOF, false)) {
5831          Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5832          msg = "EnLongAddered EOF. Most likely due to Master failure during "
5833            + "wal splitting, so we have this data in another edit. Continuing, but renaming "
5834            + edits + " as " + p + " for region " + this;
5835          LOG.warn(msg, eof);
5836          status.abort(msg);
5837        } else {
5838          LOG.warn("EOF while replaying recover edits and config '{}' is true so "
5839            + "we will ignore it and continue", RECOVERED_EDITS_IGNORE_EOF, eof);
5840        }
5841      } catch (IOException ioe) {
5842        // If the IOE resulted from bad file format,
5843        // then this problem is idempotent and retrying won't help
5844        if (ioe.getCause() instanceof ParseException) {
5845          Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5846          msg =
5847            "File corruption enLongAddered!  " + "Continuing, but renaming " + edits + " as " + p;
5848          LOG.warn(msg, ioe);
5849          status.setStatus(msg);
5850        } else {
5851          status.abort(StringUtils.stringifyException(ioe));
5852          // other IO errors may be transient (bad network connection,
5853          // checksum exception on one datanode, etc). throw & retry
5854          throw ioe;
5855        }
5856      }
5857      if (reporter != null && !reported_once) {
5858        reporter.progress();
5859      }
5860      msg = "Applied " + editsCount + ", skipped " + skippedEdits + ", firstSequenceIdInLog="
5861        + firstSeqIdInLog + ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
5862      status.markComplete(msg);
5863      LOG.debug(msg);
5864      return currentEditSeqId;
5865    } finally {
5866      status.cleanup();
5867    }
5868  }
5869
5870  /**
5871   * Call to complete a compaction. Its for the case where we find in the WAL a compaction that was
5872   * not finished. We could find one recovering a WAL after a regionserver crash. See HBASE-2331.
5873   */
5874  void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
5875    boolean removeFiles, long replaySeqId) throws IOException {
5876    try {
5877      checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
5878        "Compaction marker from WAL ", compaction);
5879    } catch (WrongRegionException wre) {
5880      if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5881        // skip the compaction marker since it is not for this region
5882        return;
5883      }
5884      throw wre;
5885    }
5886
5887    synchronized (writestate) {
5888      if (replaySeqId < lastReplayedOpenRegionSeqId) {
5889        LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying compaction event :"
5890          + TextFormat.shortDebugString(compaction) + " because its sequence id " + replaySeqId
5891          + " is smaller than this regions " + "lastReplayedOpenRegionSeqId of "
5892          + lastReplayedOpenRegionSeqId);
5893        return;
5894      }
5895      if (replaySeqId < lastReplayedCompactionSeqId) {
5896        LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying compaction event :"
5897          + TextFormat.shortDebugString(compaction) + " because its sequence id " + replaySeqId
5898          + " is smaller than this regions " + "lastReplayedCompactionSeqId of "
5899          + lastReplayedCompactionSeqId);
5900        return;
5901      } else {
5902        lastReplayedCompactionSeqId = replaySeqId;
5903      }
5904
5905      if (LOG.isDebugEnabled()) {
5906        LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying compaction marker "
5907          + TextFormat.shortDebugString(compaction) + " with seqId=" + replaySeqId
5908          + " and lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
5909      }
5910
5911      startRegionOperation(Operation.REPLAY_EVENT);
5912      try {
5913        HStore store = this.getStore(compaction.getFamilyName().toByteArray());
5914        if (store == null) {
5915          LOG.warn(getRegionInfo().getEncodedName() + " : "
5916            + "Found Compaction WAL edit for deleted family:"
5917            + Bytes.toString(compaction.getFamilyName().toByteArray()));
5918          return;
5919        }
5920        store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
5921        logRegionFiles();
5922      } catch (FileNotFoundException ex) {
5923        LOG.warn(getRegionInfo().getEncodedName() + " : "
5924          + "At least one of the store files in compaction: "
5925          + TextFormat.shortDebugString(compaction)
5926          + " doesn't exist any more. Skip loading the file(s)", ex);
5927      } finally {
5928        closeRegionOperation(Operation.REPLAY_EVENT);
5929      }
5930    }
5931  }
5932
5933  /**
5934   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
5935   *             replica implementation.
5936   */
5937  @Deprecated
5938  void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
5939    checkTargetRegion(flush.getEncodedRegionName().toByteArray(), "Flush marker from WAL ", flush);
5940
5941    if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5942      return; // if primary nothing to do
5943    }
5944
5945    if (LOG.isDebugEnabled()) {
5946      LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying flush marker "
5947        + TextFormat.shortDebugString(flush));
5948    }
5949
5950    startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
5951    try {
5952      FlushAction action = flush.getAction();
5953      switch (action) {
5954        case START_FLUSH:
5955          replayWALFlushStartMarker(flush);
5956          break;
5957        case COMMIT_FLUSH:
5958          replayWALFlushCommitMarker(flush);
5959          break;
5960        case ABORT_FLUSH:
5961          replayWALFlushAbortMarker(flush);
5962          break;
5963        case CANNOT_FLUSH:
5964          replayWALFlushCannotFlushMarker(flush, replaySeqId);
5965          break;
5966        default:
5967          LOG.warn(getRegionInfo().getEncodedName() + " : "
5968            + "Received a flush event with unknown action, ignoring. "
5969            + TextFormat.shortDebugString(flush));
5970          break;
5971      }
5972
5973      logRegionFiles();
5974    } finally {
5975      closeRegionOperation(Operation.REPLAY_EVENT);
5976    }
5977  }
5978
5979  private Collection<HStore> getStoresToFlush(FlushDescriptor flushDesc) {
5980    List<HStore> storesToFlush = new ArrayList<>();
5981    for (StoreFlushDescriptor storeFlush : flushDesc.getStoreFlushesList()) {
5982      byte[] family = storeFlush.getFamilyName().toByteArray();
5983      HStore store = getStore(family);
5984      if (store == null) {
5985        LOG.warn(getRegionInfo().getEncodedName() + " : "
5986          + "Received a flush start marker from primary, but the family is not found. Ignoring"
5987          + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
5988        continue;
5989      }
5990      storesToFlush.add(store);
5991    }
5992    return storesToFlush;
5993  }
5994
5995  /**
5996   * Replay the flush marker from primary region by creating a corresponding snapshot of the store
5997   * memstores, only if the memstores do not have a higher seqId from an earlier wal edit (because
5998   * the events may be coming out of order).
5999   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6000   *             replica implementation.
6001   */
6002  @Deprecated
6003  PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
6004    long flushSeqId = flush.getFlushSequenceNumber();
6005
6006    Collection<HStore> storesToFlush = getStoresToFlush(flush);
6007
6008    MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
6009
6010    // we will use writestate as a coarse-grain lock for all the replay events
6011    // (flush, compaction, region open etc)
6012    synchronized (writestate) {
6013      try {
6014        if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
6015          LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :"
6016            + TextFormat.shortDebugString(flush)
6017            + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6018            + " of " + lastReplayedOpenRegionSeqId);
6019          return null;
6020        }
6021        if (numMutationsWithoutWAL.sum() > 0) {
6022          numMutationsWithoutWAL.reset();
6023          dataInMemoryWithoutWAL.reset();
6024        }
6025
6026        if (!writestate.flushing) {
6027          // we do not have an active snapshot and corresponding this.prepareResult. This means
6028          // we can just snapshot our memstores and continue as normal.
6029
6030          // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
6031          PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId,
6032            storesToFlush, status, false, FlushLifeCycleTracker.DUMMY);
6033          if (prepareResult.result == null) {
6034            // save the PrepareFlushResult so that we can use it later from commit flush
6035            this.writestate.flushing = true;
6036            this.prepareFlushResult = prepareResult;
6037            status.markComplete("Flush prepare successful");
6038            if (LOG.isDebugEnabled()) {
6039              LOG.debug(getRegionInfo().getEncodedName() + " : " + " Prepared flush with seqId:"
6040                + flush.getFlushSequenceNumber());
6041            }
6042          } else {
6043            // special case empty memstore. We will still save the flush result in this case, since
6044            // our memstore ie empty, but the primary is still flushing
6045            if (
6046              prepareResult.getResult().getResult()
6047                  == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY
6048            ) {
6049              this.writestate.flushing = true;
6050              this.prepareFlushResult = prepareResult;
6051              if (LOG.isDebugEnabled()) {
6052                LOG.debug(getRegionInfo().getEncodedName() + " : "
6053                  + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
6054              }
6055            }
6056            status.abort("Flush prepare failed with " + prepareResult.result);
6057            // nothing much to do. prepare flush failed because of some reason.
6058          }
6059          return prepareResult;
6060        } else {
6061          // we already have an active snapshot.
6062          if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
6063            // They define the same flush. Log and continue.
6064            LOG.warn(getRegionInfo().getEncodedName() + " : "
6065              + "Received a flush prepare marker with the same seqId: "
6066              + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
6067              + prepareFlushResult.flushOpSeqId + ". Ignoring");
6068            // ignore
6069          } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
6070            // We received a flush with a smaller seqNum than what we have prepared. We can only
6071            // ignore this prepare flush request.
6072            LOG.warn(getRegionInfo().getEncodedName() + " : "
6073              + "Received a flush prepare marker with a smaller seqId: "
6074              + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
6075              + prepareFlushResult.flushOpSeqId + ". Ignoring");
6076            // ignore
6077          } else {
6078            // We received a flush with a larger seqNum than what we have prepared
6079            LOG.warn(getRegionInfo().getEncodedName() + " : "
6080              + "Received a flush prepare marker with a larger seqId: "
6081              + +flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
6082              + prepareFlushResult.flushOpSeqId + ". Ignoring");
6083            // We do not have multiple active snapshots in the memstore or a way to merge current
6084            // memstore snapshot with the contents and resnapshot for now. We cannot take
6085            // another snapshot and drop the previous one because that will cause temporary
6086            // data loss in the secondary. So we ignore this for now, deferring the resolution
6087            // to happen when we see the corresponding flush commit marker. If we have a memstore
6088            // snapshot with x, and later received another prepare snapshot with y (where x < y),
6089            // when we see flush commit for y, we will drop snapshot for x, and can also drop all
6090            // the memstore edits if everything in memstore is < y. This is the usual case for
6091            // RS crash + recovery where we might see consequtive prepare flush wal markers.
6092            // Otherwise, this will cause more memory to be used in secondary replica until a
6093            // further prapare + commit flush is seen and replayed.
6094          }
6095        }
6096      } finally {
6097        status.cleanup();
6098        writestate.notifyAll();
6099      }
6100    }
6101    return null;
6102  }
6103
6104  /**
6105   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6106   *             replica implementation.
6107   */
6108  @Deprecated
6109  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6110      justification = "Intentional; post memstore flush")
6111  void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
6112    MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
6113
6114    // check whether we have the memstore snapshot with the corresponding seqId. Replay to
6115    // secondary region replicas are in order, except for when the region moves or then the
6116    // region server crashes. In those cases, we may receive replay requests out of order from
6117    // the original seqIds.
6118    synchronized (writestate) {
6119      try {
6120        if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
6121          LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :"
6122            + TextFormat.shortDebugString(flush)
6123            + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6124            + " of " + lastReplayedOpenRegionSeqId);
6125          return;
6126        }
6127
6128        if (writestate.flushing) {
6129          PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
6130          if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
6131            if (LOG.isDebugEnabled()) {
6132              LOG.debug(getRegionInfo().getEncodedName() + " : "
6133                + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
6134                + " and a previous prepared snapshot was found");
6135            }
6136            // This is the regular case where we received commit flush after prepare flush
6137            // corresponding to the same seqId.
6138            replayFlushInStores(flush, prepareFlushResult, true);
6139
6140            // Set down the memstore size by amount of flush.
6141            this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
6142            this.prepareFlushResult = null;
6143            writestate.flushing = false;
6144          } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
6145            // This should not happen normally. However, lets be safe and guard against these cases
6146            // we received a flush commit with a smaller seqId than what we have prepared
6147            // we will pick the flush file up from this commit (if we have not seen it), but we
6148            // will not drop the memstore
6149            LOG.warn(getRegionInfo().getEncodedName() + " : "
6150              + "Received a flush commit marker with smaller seqId: "
6151              + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
6152              + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
6153              + "  prepared memstore snapshot");
6154            replayFlushInStores(flush, prepareFlushResult, false);
6155
6156            // snapshot is not dropped, so memstore sizes should not be decremented
6157            // we still have the prepared snapshot, flushing should still be true
6158          } else {
6159            // This should not happen normally. However, lets be safe and guard against these cases
6160            // we received a flush commit with a larger seqId than what we have prepared
6161            // we will pick the flush file for this. We will also obtain the updates lock and
6162            // look for contents of the memstore to see whether we have edits after this seqId.
6163            // If not, we will drop all the memstore edits and the snapshot as well.
6164            LOG.warn(getRegionInfo().getEncodedName() + " : "
6165              + "Received a flush commit marker with larger seqId: "
6166              + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
6167              + prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
6168              + " memstore snapshot");
6169
6170            replayFlushInStores(flush, prepareFlushResult, true);
6171
6172            // Set down the memstore size by amount of flush.
6173            this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
6174
6175            // Inspect the memstore contents to see whether the memstore contains only edits
6176            // with seqId smaller than the flush seqId. If so, we can discard those edits.
6177            dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
6178
6179            this.prepareFlushResult = null;
6180            writestate.flushing = false;
6181          }
6182          // If we were waiting for observing a flush or region opening event for not showing
6183          // partial data after a secondary region crash, we can allow reads now. We can only make
6184          // sure that we are not showing partial data (for example skipping some previous edits)
6185          // until we observe a full flush start and flush commit. So if we were not able to find
6186          // a previous flush we will not enable reads now.
6187          this.setReadsEnabled(true);
6188        } else {
6189          LOG.warn(
6190            getRegionInfo().getEncodedName() + " : " + "Received a flush commit marker with seqId:"
6191              + flush.getFlushSequenceNumber() + ", but no previous prepared snapshot was found");
6192          // There is no corresponding prepare snapshot from before.
6193          // We will pick up the new flushed file
6194          replayFlushInStores(flush, null, false);
6195
6196          // Inspect the memstore contents to see whether the memstore contains only edits
6197          // with seqId smaller than the flush seqId. If so, we can discard those edits.
6198          dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
6199        }
6200
6201        status.markComplete("Flush commit successful");
6202
6203        // Update the last flushed sequence id for region.
6204        this.maxFlushedSeqId = flush.getFlushSequenceNumber();
6205
6206        // advance the mvcc read point so that the new flushed file is visible.
6207        mvcc.advanceTo(flush.getFlushSequenceNumber());
6208
6209      } catch (FileNotFoundException ex) {
6210        LOG.warn(getRegionInfo().getEncodedName() + " : "
6211          + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
6212          + " doesn't exist any more. Skip loading the file(s)", ex);
6213      } finally {
6214        status.cleanup();
6215        writestate.notifyAll();
6216      }
6217    }
6218
6219    // C. Finally notify anyone waiting on memstore to clear:
6220    // e.g. checkResources().
6221    synchronized (this) {
6222      notifyAll(); // FindBugs NN_NAKED_NOTIFY
6223    }
6224  }
6225
6226  /**
6227   * Replays the given flush descriptor by opening the flush files in stores and dropping the
6228   * memstore snapshots if requested.
6229   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6230   *             replica implementation.
6231   */
6232  @Deprecated
6233  private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
6234    boolean dropMemstoreSnapshot) throws IOException {
6235    for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
6236      byte[] family = storeFlush.getFamilyName().toByteArray();
6237      HStore store = getStore(family);
6238      if (store == null) {
6239        LOG.warn(getRegionInfo().getEncodedName() + " : "
6240          + "Received a flush commit marker from primary, but the family is not found."
6241          + "Ignoring StoreFlushDescriptor:" + storeFlush);
6242        continue;
6243      }
6244      List<String> flushFiles = storeFlush.getFlushOutputList();
6245      StoreFlushContext ctx = null;
6246      long startTime = EnvironmentEdgeManager.currentTime();
6247      if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
6248        ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY);
6249      } else {
6250        ctx = prepareFlushResult.storeFlushCtxs.get(family);
6251        startTime = prepareFlushResult.startTime;
6252      }
6253
6254      if (ctx == null) {
6255        LOG.warn(getRegionInfo().getEncodedName() + " : "
6256          + "Unexpected: flush commit marker received from store " + Bytes.toString(family)
6257          + " but no associated flush context. Ignoring");
6258        continue;
6259      }
6260
6261      ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
6262
6263      // Record latest flush time
6264      this.lastStoreFlushTimeMap.put(store, startTime);
6265    }
6266  }
6267
6268  private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException {
6269    Path regionDir = fs.getRegionDir();
6270    long maxSeqId = -1;
6271    for (HStore store : stores) {
6272      String familyName = store.getColumnFamilyName();
6273      FileStatus[] files =
6274        WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName);
6275      if (files != null && files.length != 0) {
6276        for (FileStatus file : files) {
6277          Path filePath = file.getPath();
6278          // If file length is zero then delete it
6279          if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) {
6280            continue;
6281          }
6282          try {
6283            HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath());
6284            maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID());
6285          } catch (IOException e) {
6286            handleException(fs.getFileSystem(), filePath, e);
6287            continue;
6288          }
6289        }
6290        if (this.rsServices != null && store.needsCompaction()) {
6291          this.rsServices.getCompactionRequestor().requestCompaction(this, store,
6292            "load recovered hfiles request compaction", Store.PRIORITY_USER + 1,
6293            CompactionLifeCycleTracker.DUMMY, null);
6294        }
6295      }
6296    }
6297    return maxSeqId;
6298  }
6299
6300  /**
6301   * Be careful, this method will drop all data in the memstore of this region. Currently, this
6302   * method is used to drop memstore to prevent memory leak when replaying recovered.edits while
6303   * opening region.
6304   */
6305  private MemStoreSize dropMemStoreContents() throws IOException {
6306    MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6307    this.updatesLock.writeLock().lock();
6308    try {
6309      for (HStore s : stores.values()) {
6310        MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM);
6311        LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region "
6312          + this.getRegionInfo().getRegionNameAsString() + " , dropped memstoresize: ["
6313          + memStoreSize + " }");
6314        totalFreedSize.incMemStoreSize(memStoreSize);
6315      }
6316      return totalFreedSize.getMemStoreSize();
6317    } finally {
6318      this.updatesLock.writeLock().unlock();
6319    }
6320  }
6321
6322  /**
6323   * Drops the memstore contents after replaying a flush descriptor or region open event replay if
6324   * the memstore edits have seqNums smaller than the given seq id
6325   */
6326  private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException {
6327    MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6328    this.updatesLock.writeLock().lock();
6329    try {
6330
6331      long currentSeqId = mvcc.getReadPoint();
6332      if (seqId >= currentSeqId) {
6333        // then we can drop the memstore contents since everything is below this seqId
6334        LOG.info(getRegionInfo().getEncodedName() + " : "
6335          + "Dropping memstore contents as well since replayed flush seqId: " + seqId
6336          + " is greater than current seqId:" + currentSeqId);
6337
6338        // Prepare flush (take a snapshot) and then abort (drop the snapshot)
6339        if (store == null) {
6340          for (HStore s : stores.values()) {
6341            totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId));
6342          }
6343        } else {
6344          totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId));
6345        }
6346      } else {
6347        LOG.info(getRegionInfo().getEncodedName() + " : "
6348          + "Not dropping memstore contents since replayed flush seqId: " + seqId
6349          + " is smaller than current seqId:" + currentSeqId);
6350      }
6351    } finally {
6352      this.updatesLock.writeLock().unlock();
6353    }
6354    return totalFreedSize.getMemStoreSize();
6355  }
6356
6357  private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId)
6358    throws IOException {
6359    MemStoreSize flushableSize = s.getFlushableSize();
6360    this.decrMemStoreSize(flushableSize);
6361    StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY);
6362    ctx.prepare();
6363    ctx.abort();
6364    return flushableSize;
6365  }
6366
6367  private void replayWALFlushAbortMarker(FlushDescriptor flush) {
6368    // nothing to do for now. A flush abort will cause a RS abort which means that the region
6369    // will be opened somewhere else later. We will see the region open event soon, and replaying
6370    // that will drop the snapshot
6371  }
6372
6373  private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
6374    synchronized (writestate) {
6375      if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
6376        LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying flush event :"
6377          + TextFormat.shortDebugString(flush) + " because its sequence id " + replaySeqId
6378          + " is smaller than this regions " + "lastReplayedOpenRegionSeqId of "
6379          + lastReplayedOpenRegionSeqId);
6380        return;
6381      }
6382
6383      // If we were waiting for observing a flush or region opening event for not showing partial
6384      // data after a secondary region crash, we can allow reads now. This event means that the
6385      // primary was not able to flush because memstore is empty when we requested flush. By the
6386      // time we observe this, we are guaranteed to have up to date seqId with our previous
6387      // assignment.
6388      this.setReadsEnabled(true);
6389    }
6390  }
6391
6392  PrepareFlushResult getPrepareFlushResult() {
6393    return prepareFlushResult;
6394  }
6395
6396  /**
6397   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6398   *             replica implementation.
6399   */
6400  @Deprecated
6401  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6402      justification = "Intentional; cleared the memstore")
6403  void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
6404    checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
6405      "RegionEvent marker from WAL ", regionEvent);
6406
6407    startRegionOperation(Operation.REPLAY_EVENT);
6408    try {
6409      if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6410        return; // if primary nothing to do
6411      }
6412
6413      if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
6414        // nothing to do on REGION_CLOSE for now.
6415        return;
6416      }
6417      if (regionEvent.getEventType() != EventType.REGION_OPEN) {
6418        LOG.warn(getRegionInfo().getEncodedName() + " : "
6419          + "Unknown region event received, ignoring :" + TextFormat.shortDebugString(regionEvent));
6420        return;
6421      }
6422
6423      if (LOG.isDebugEnabled()) {
6424        LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying region open event marker "
6425          + TextFormat.shortDebugString(regionEvent));
6426      }
6427
6428      // we will use writestate as a coarse-grain lock for all the replay events
6429      synchronized (writestate) {
6430        // Replication can deliver events out of order when primary region moves or the region
6431        // server crashes, since there is no coordination between replication of different wal files
6432        // belonging to different region servers. We have to safe guard against this case by using
6433        // region open event's seqid. Since this is the first event that the region puts (after
6434        // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6435        // smaller than this seqId
6436        if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
6437          this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
6438        } else {
6439          LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying region event :"
6440            + TextFormat.shortDebugString(regionEvent)
6441            + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6442            + " of " + lastReplayedOpenRegionSeqId);
6443          return;
6444        }
6445
6446        // region open lists all the files that the region has at the time of the opening. Just pick
6447        // all the files and drop prepared flushes and empty memstores
6448        for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
6449          // stores of primary may be different now
6450          byte[] family = storeDescriptor.getFamilyName().toByteArray();
6451          HStore store = getStore(family);
6452          if (store == null) {
6453            LOG.warn(getRegionInfo().getEncodedName() + " : "
6454              + "Received a region open marker from primary, but the family is not found. "
6455              + "Ignoring. StoreDescriptor:" + storeDescriptor);
6456            continue;
6457          }
6458
6459          long storeSeqId = store.getMaxSequenceId().orElse(0L);
6460          List<String> storeFiles = storeDescriptor.getStoreFileList();
6461          try {
6462            store.refreshStoreFiles(storeFiles); // replace the files with the new ones
6463          } catch (FileNotFoundException ex) {
6464            LOG.warn(getRegionInfo().getEncodedName() + " : " + "At least one of the store files: "
6465              + storeFiles + " doesn't exist any more. Skip loading the file(s)", ex);
6466            continue;
6467          }
6468          if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
6469            // Record latest flush time if we picked up new files
6470            lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
6471          }
6472
6473          if (writestate.flushing) {
6474            // only drop memstore snapshots if they are smaller than last flush for the store
6475            if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
6476              StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null
6477                ? null
6478                : this.prepareFlushResult.storeFlushCtxs.get(family);
6479              if (ctx != null) {
6480                MemStoreSize mss = store.getFlushableSize();
6481                ctx.abort();
6482                this.decrMemStoreSize(mss);
6483                this.prepareFlushResult.storeFlushCtxs.remove(family);
6484              }
6485            }
6486          }
6487
6488          // Drop the memstore contents if they are now smaller than the latest seen flushed file
6489          dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
6490          if (storeSeqId > this.maxFlushedSeqId) {
6491            this.maxFlushedSeqId = storeSeqId;
6492          }
6493        }
6494
6495        // if all stores ended up dropping their snapshots, we can safely drop the
6496        // prepareFlushResult
6497        dropPrepareFlushIfPossible();
6498
6499        // advance the mvcc read point so that the new flushed file is visible.
6500        mvcc.await();
6501
6502        // If we were waiting for observing a flush or region opening event for not showing partial
6503        // data after a secondary region crash, we can allow reads now.
6504        this.setReadsEnabled(true);
6505
6506        // C. Finally notify anyone waiting on memstore to clear:
6507        // e.g. checkResources().
6508        synchronized (this) {
6509          notifyAll(); // FindBugs NN_NAKED_NOTIFY
6510        }
6511      }
6512      logRegionFiles();
6513    } finally {
6514      closeRegionOperation(Operation.REPLAY_EVENT);
6515    }
6516  }
6517
6518  /**
6519   * @deprecated Since 3.0.0, will be removed in 4.0.0. Only for keep compatibility for old region
6520   *             replica implementation.
6521   */
6522  @Deprecated
6523  void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
6524    checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
6525      "BulkLoad marker from WAL ", bulkLoadEvent);
6526
6527    if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6528      return; // if primary nothing to do
6529    }
6530
6531    if (LOG.isDebugEnabled()) {
6532      LOG.debug(getRegionInfo().getEncodedName() + " : " + "Replaying bulkload event marker "
6533        + TextFormat.shortDebugString(bulkLoadEvent));
6534    }
6535    // check if multiple families involved
6536    boolean multipleFamilies = false;
6537    byte[] family = null;
6538    for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6539      byte[] fam = storeDescriptor.getFamilyName().toByteArray();
6540      if (family == null) {
6541        family = fam;
6542      } else if (!Bytes.equals(family, fam)) {
6543        multipleFamilies = true;
6544        break;
6545      }
6546    }
6547
6548    startBulkRegionOperation(multipleFamilies);
6549    try {
6550      // we will use writestate as a coarse-grain lock for all the replay events
6551      synchronized (writestate) {
6552        // Replication can deliver events out of order when primary region moves or the region
6553        // server crashes, since there is no coordination between replication of different wal files
6554        // belonging to different region servers. We have to safe guard against this case by using
6555        // region open event's seqid. Since this is the first event that the region puts (after
6556        // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6557        // smaller than this seqId
6558        if (
6559          bulkLoadEvent.getBulkloadSeqNum() >= 0
6560            && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()
6561        ) {
6562          LOG.warn(getRegionInfo().getEncodedName() + " : " + "Skipping replaying bulkload event :"
6563            + TextFormat.shortDebugString(bulkLoadEvent)
6564            + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
6565            + " =" + lastReplayedOpenRegionSeqId);
6566
6567          return;
6568        }
6569
6570        for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6571          // stores of primary may be different now
6572          family = storeDescriptor.getFamilyName().toByteArray();
6573          HStore store = getStore(family);
6574          if (store == null) {
6575            LOG.warn(getRegionInfo().getEncodedName() + " : "
6576              + "Received a bulk load marker from primary, but the family is not found. "
6577              + "Ignoring. StoreDescriptor:" + storeDescriptor);
6578            continue;
6579          }
6580
6581          StoreContext storeContext = store.getStoreContext();
6582          StoreFileTracker sft = StoreFileTrackerFactory.create(conf, false, storeContext);
6583
6584          List<StoreFileInfo> storeFiles = sft.load();
6585          for (StoreFileInfo storeFileInfo : storeFiles) {
6586            try {
6587              store.bulkLoadHFile(storeFileInfo);
6588            } catch (FileNotFoundException ex) {
6589              LOG.warn(getRegionInfo().getEncodedName() + " : " + storeFileInfo.toString()
6590                + " doesn't exist any more. Skip loading the file");
6591            }
6592          }
6593        }
6594      }
6595      if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
6596        mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
6597      }
6598    } finally {
6599      closeBulkRegionOperation();
6600    }
6601  }
6602
6603  /**
6604   * Replay the batch mutate for secondary replica.
6605   * <p/>
6606   * We will directly apply the cells to the memstore. This is because:
6607   * <ol>
6608   * <li>All the cells are gotten from {@link WALEdit}, so we only have {@link Put} and
6609   * {@link Delete} here</li>
6610   * <li>The replay is single threaded, we do not need to acquire row lock, as the region is read
6611   * only so no one else can write it.</li>
6612   * <li>We do not need to write WAL.</li>
6613   * <li>We will advance MVCC in the caller directly.</li>
6614   * </ol>
6615   */
6616  private void replayWALBatchMutate(Map<byte[], List<ExtendedCell>> family2Cells)
6617    throws IOException {
6618    startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
6619    try {
6620      for (Map.Entry<byte[], List<ExtendedCell>> entry : family2Cells.entrySet()) {
6621        applyToMemStore(getStore(entry.getKey()), entry.getValue(), false, memStoreSizing);
6622      }
6623    } finally {
6624      closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
6625    }
6626  }
6627
6628  /**
6629   * Replay the meta edits, i.e, flush marker, compaction marker, bulk load marker, region event
6630   * marker, etc.
6631   * <p/>
6632   * For all events other than start flush, we will just call {@link #refreshStoreFiles()} as the
6633   * logic is straight-forward and robust. For start flush, we need to snapshot the memstore, so
6634   * later {@link #refreshStoreFiles()} call could drop the snapshot, otherwise we may run out of
6635   * memory.
6636   */
6637  private void replayWALMetaEdit(Cell cell) throws IOException {
6638    startRegionOperation(Operation.REPLAY_EVENT);
6639    try {
6640      FlushDescriptor flushDesc = WALEdit.getFlushDescriptor(cell);
6641      if (flushDesc != null) {
6642        switch (flushDesc.getAction()) {
6643          case START_FLUSH:
6644            // for start flush, we need to take a snapshot of the current memstore
6645            synchronized (writestate) {
6646              if (!writestate.flushing) {
6647                this.writestate.flushing = true;
6648              } else {
6649                // usually this should not happen but let's make the code more robust, it is not a
6650                // big deal to just ignore it, the refreshStoreFiles call should have the ability to
6651                // clean up the inconsistent state.
6652                LOG.debug("NOT flushing {} as already flushing", getRegionInfo());
6653                break;
6654              }
6655            }
6656            MonitoredTask status =
6657              TaskMonitor.get().createStatus("Preparing flush " + getRegionInfo());
6658            Collection<HStore> storesToFlush = getStoresToFlush(flushDesc);
6659            try {
6660              PrepareFlushResult prepareResult =
6661                internalPrepareFlushCache(null, flushDesc.getFlushSequenceNumber(), storesToFlush,
6662                  status, false, FlushLifeCycleTracker.DUMMY);
6663              if (prepareResult.result == null) {
6664                // save the PrepareFlushResult so that we can use it later from commit flush
6665                this.prepareFlushResult = prepareResult;
6666                status.markComplete("Flush prepare successful");
6667                if (LOG.isDebugEnabled()) {
6668                  LOG.debug("{} prepared flush with seqId: {}", getRegionInfo(),
6669                    flushDesc.getFlushSequenceNumber());
6670                }
6671              } else {
6672                // special case empty memstore. We will still save the flush result in this case,
6673                // since our memstore is empty, but the primary is still flushing
6674                if (
6675                  prepareResult.getResult().getResult()
6676                      == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY
6677                ) {
6678                  this.prepareFlushResult = prepareResult;
6679                  if (LOG.isDebugEnabled()) {
6680                    LOG.debug("{} prepared empty flush with seqId: {}", getRegionInfo(),
6681                      flushDesc.getFlushSequenceNumber());
6682                  }
6683                }
6684                status.abort("Flush prepare failed with " + prepareResult.result);
6685                // nothing much to do. prepare flush failed because of some reason.
6686              }
6687            } finally {
6688              status.cleanup();
6689            }
6690            break;
6691          case ABORT_FLUSH:
6692            // do nothing, an abort flush means the source region server will crash itself, after
6693            // the primary region online, it will send us an open region marker, then we can clean
6694            // up the memstore.
6695            synchronized (writestate) {
6696              writestate.flushing = false;
6697            }
6698            break;
6699          case COMMIT_FLUSH:
6700          case CANNOT_FLUSH:
6701            // just call refreshStoreFiles
6702            refreshStoreFiles();
6703            logRegionFiles();
6704            synchronized (writestate) {
6705              writestate.flushing = false;
6706            }
6707            break;
6708          default:
6709            LOG.warn("{} received a flush event with unknown action: {}", getRegionInfo(),
6710              TextFormat.shortDebugString(flushDesc));
6711        }
6712      } else {
6713        // for all other region events, we will do a refreshStoreFiles
6714        refreshStoreFiles();
6715        logRegionFiles();
6716      }
6717    } finally {
6718      closeRegionOperation(Operation.REPLAY_EVENT);
6719    }
6720  }
6721
6722  /**
6723   * Replay remote wal entry sent by primary replica.
6724   * <p/>
6725   * Should only call this method on secondary replicas.
6726   */
6727  void replayWALEntry(WALEntry entry, CellScanner cells) throws IOException {
6728    long timeout = -1L;
6729    Optional<RpcCall> call = RpcServer.getCurrentCall();
6730    if (call.isPresent()) {
6731      long deadline = call.get().getDeadline();
6732      if (deadline < Long.MAX_VALUE) {
6733        timeout = deadline - EnvironmentEdgeManager.currentTime();
6734        if (timeout <= 0) {
6735          throw new TimeoutIOException("Timeout while replaying edits for " + getRegionInfo());
6736        }
6737      }
6738    }
6739    if (timeout > 0) {
6740      try {
6741        if (!replayLock.tryLock(timeout, TimeUnit.MILLISECONDS)) {
6742          throw new TimeoutIOException(
6743            "Timeout while waiting for lock when replaying edits for " + getRegionInfo());
6744        }
6745      } catch (InterruptedException e) {
6746        throw throwOnInterrupt(e);
6747      }
6748    } else {
6749      replayLock.lock();
6750    }
6751    try {
6752      int count = entry.getAssociatedCellCount();
6753      long sequenceId = entry.getKey().getLogSequenceNumber();
6754      if (lastReplayedSequenceId >= sequenceId) {
6755        // we have already replayed this edit, skip
6756        // remember to advance the CellScanner, as we may have multiple WALEntries, we may still
6757        // need apply later WALEntries
6758        for (int i = 0; i < count; i++) {
6759          // Throw index out of bounds if our cell count is off
6760          if (!cells.advance()) {
6761            throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
6762          }
6763        }
6764        return;
6765      }
6766      Map<byte[], List<ExtendedCell>> family2Cells = new TreeMap<>(Bytes.BYTES_COMPARATOR);
6767      for (int i = 0; i < count; i++) {
6768        // Throw index out of bounds if our cell count is off
6769        if (!cells.advance()) {
6770          throw new ArrayIndexOutOfBoundsException("Expected=" + count + ", index=" + i);
6771        }
6772        Cell c = cells.current();
6773        assert c instanceof ExtendedCell;
6774        ExtendedCell cell = (ExtendedCell) c;
6775        if (WALEdit.isMetaEditFamily(cell)) {
6776          // If there is meta edit, i.e, we have done flush/compaction/open, then we need to apply
6777          // the previous cells first, and then replay the special meta edit. The meta edit is like
6778          // a barrier, We need to keep the order. For example, the flush marker will contain a
6779          // flush sequence number, which makes us possible to drop memstore content, but if we
6780          // apply some edits which have greater sequence id first, then we can not drop the
6781          // memstore content when replaying the flush marker, which is not good as we could run out
6782          // of memory.
6783          // And usually, a meta edit will have a special WALEntry for it, so this is just a safe
6784          // guard logic to make sure we do not break things in the worst case.
6785          if (!family2Cells.isEmpty()) {
6786            replayWALBatchMutate(family2Cells);
6787            family2Cells.clear();
6788          }
6789          replayWALMetaEdit(cell);
6790        } else {
6791          family2Cells.computeIfAbsent(CellUtil.cloneFamily(cell), k -> new ArrayList<>())
6792            .add(cell);
6793        }
6794      }
6795      // do not forget to apply the remaining cells
6796      if (!family2Cells.isEmpty()) {
6797        replayWALBatchMutate(family2Cells);
6798      }
6799      mvcc.advanceTo(sequenceId);
6800      lastReplayedSequenceId = sequenceId;
6801    } finally {
6802      replayLock.unlock();
6803    }
6804  }
6805
6806  /**
6807   * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
6808   */
6809  private void dropPrepareFlushIfPossible() {
6810    if (writestate.flushing) {
6811      boolean canDrop = true;
6812      if (prepareFlushResult.storeFlushCtxs != null) {
6813        for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs
6814          .entrySet()) {
6815          HStore store = getStore(entry.getKey());
6816          if (store == null) {
6817            continue;
6818          }
6819          if (store.getSnapshotSize().getDataSize() > 0) {
6820            canDrop = false;
6821            break;
6822          }
6823        }
6824      }
6825
6826      // this means that all the stores in the region has finished flushing, but the WAL marker
6827      // may not have been written or we did not receive it yet.
6828      if (canDrop) {
6829        writestate.flushing = false;
6830        this.prepareFlushResult = null;
6831      }
6832    }
6833  }
6834
6835  @Override
6836  public boolean refreshStoreFiles() throws IOException {
6837    return refreshStoreFiles(false);
6838  }
6839
6840  @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6841      justification = "Notify is about post replay. Intentional")
6842  protected boolean refreshStoreFiles(boolean force) throws IOException {
6843    if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6844      return false; // if primary nothing to do
6845    }
6846
6847    if (LOG.isDebugEnabled()) {
6848      LOG.debug(getRegionInfo().getEncodedName() + " : "
6849        + "Refreshing store files to see whether we can free up memstore");
6850    }
6851
6852    long totalFreedDataSize = 0;
6853
6854    long smallestSeqIdInStores = Long.MAX_VALUE;
6855
6856    startRegionOperation(); // obtain region close lock
6857    try {
6858      Map<HStore, Long> map = new HashMap<>();
6859      synchronized (writestate) {
6860        for (HStore store : stores.values()) {
6861          // TODO: some stores might see new data from flush, while others do not which
6862          // MIGHT break atomic edits across column families.
6863          long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L);
6864
6865          // refresh the store files. This is similar to observing a region open wal marker.
6866          store.refreshStoreFiles();
6867
6868          long storeSeqId = store.getMaxSequenceId().orElse(0L);
6869          if (storeSeqId < smallestSeqIdInStores) {
6870            smallestSeqIdInStores = storeSeqId;
6871          }
6872
6873          // see whether we can drop the memstore or the snapshot
6874          if (storeSeqId > maxSeqIdBefore) {
6875            if (writestate.flushing) {
6876              // only drop memstore snapshots if they are smaller than last flush for the store
6877              if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
6878                StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null
6879                  ? null
6880                  : this.prepareFlushResult.storeFlushCtxs
6881                    .get(store.getColumnFamilyDescriptor().getName());
6882                if (ctx != null) {
6883                  MemStoreSize mss = store.getFlushableSize();
6884                  ctx.abort();
6885                  this.decrMemStoreSize(mss);
6886                  this.prepareFlushResult.storeFlushCtxs
6887                    .remove(store.getColumnFamilyDescriptor().getName());
6888                  totalFreedDataSize += mss.getDataSize();
6889                }
6890              }
6891            }
6892
6893            map.put(store, storeSeqId);
6894          }
6895        }
6896
6897        // if all stores ended up dropping their snapshots, we can safely drop the
6898        // prepareFlushResult
6899        dropPrepareFlushIfPossible();
6900
6901        // advance the mvcc read point so that the new flushed files are visible.
6902        // either greater than flush seq number or they were already picked up via flush.
6903        for (HStore s : stores.values()) {
6904          mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L));
6905        }
6906
6907        // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
6908        // skip all edits that are to be replayed in the future with that has a smaller seqId
6909        // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
6910        // that we have picked the flush files for
6911        if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
6912          this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
6913        }
6914      }
6915      if (!map.isEmpty()) {
6916        for (Map.Entry<HStore, Long> entry : map.entrySet()) {
6917          // Drop the memstore contents if they are now smaller than the latest seen flushed file
6918          totalFreedDataSize +=
6919            dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey()).getDataSize();
6920        }
6921      }
6922      // C. Finally notify anyone waiting on memstore to clear:
6923      // e.g. checkResources().
6924      synchronized (this) {
6925        notifyAll(); // FindBugs NN_NAKED_NOTIFY
6926      }
6927      return totalFreedDataSize > 0;
6928    } finally {
6929      closeRegionOperation();
6930    }
6931  }
6932
6933  private void logRegionFiles() {
6934    if (LOG.isTraceEnabled()) {
6935      LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
6936      stores.values().stream().filter(s -> s.getStorefiles() != null)
6937        .flatMap(s -> s.getStorefiles().stream())
6938        .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf));
6939    }
6940  }
6941
6942  /**
6943   * Checks whether the given regionName is either equal to our region, or that the regionName is
6944   * the primary region to our corresponding range for the secondary replica.
6945   */
6946  private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
6947    throws WrongRegionException {
6948    if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
6949      return;
6950    }
6951
6952    if (
6953      !RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())
6954        && Bytes.equals(encodedRegionName, this.fs.getRegionInfoForFS().getEncodedNameAsBytes())
6955    ) {
6956      return;
6957    }
6958
6959    throw new WrongRegionException(
6960      exceptionMsg + payload + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
6961        + " does not match this region: " + this.getRegionInfo());
6962  }
6963
6964  /**
6965   * Used by tests
6966   * @param s    Store to add edit too.
6967   * @param cell Cell to add.
6968   */
6969  protected void restoreEdit(HStore s, ExtendedCell cell, MemStoreSizing memstoreAccounting) {
6970    s.add(cell, memstoreAccounting);
6971  }
6972
6973  /**
6974   * make sure have been through lease recovery before get file status, so the file length can be
6975   * trusted.
6976   * @param p File to check.
6977   * @return True if file was zero-length (and if so, we'll delete it in here).
6978   */
6979  private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat,
6980    final Path p) throws IOException {
6981    if (stat.getLen() > 0) {
6982      return false;
6983    }
6984    LOG.warn("File " + p + " is zero-length, deleting.");
6985    fs.delete(p, false);
6986    return true;
6987  }
6988
6989  protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup)
6990    throws IOException {
6991    if (family.isMobEnabled()) {
6992      if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
6993        throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS
6994          + " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY
6995          + " accordingly.");
6996      }
6997      return new HMobStore(this, family, this.conf, warmup);
6998    }
6999    return new HStore(this, family, this.conf, warmup);
7000  }
7001
7002  @Override
7003  public HStore getStore(byte[] column) {
7004    return this.stores.get(column);
7005  }
7006
7007  /**
7008   * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on
7009   * the list.
7010   */
7011  private HStore getStore(Cell cell) {
7012    return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey()))
7013      .map(e -> e.getValue()).findFirst().orElse(null);
7014  }
7015
7016  @Override
7017  public List<HStore> getStores() {
7018    return new ArrayList<>(stores.values());
7019  }
7020
7021  @Override
7022  public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException {
7023    List<String> storeFileNames = new ArrayList<>();
7024    synchronized (closeLock) {
7025      for (byte[] column : columns) {
7026        HStore store = this.stores.get(column);
7027        if (store == null) {
7028          throw new IllegalArgumentException(
7029            "No column family : " + new String(column, StandardCharsets.UTF_8) + " available");
7030        }
7031        Collection<HStoreFile> storeFiles = store.getStorefiles();
7032        if (storeFiles == null) {
7033          continue;
7034        }
7035        for (HStoreFile storeFile : storeFiles) {
7036          storeFileNames.add(storeFile.getPath().toString());
7037        }
7038
7039        logRegionFiles();
7040      }
7041    }
7042    return storeFileNames;
7043  }
7044
7045  //////////////////////////////////////////////////////////////////////////////
7046  // Support code
7047  //////////////////////////////////////////////////////////////////////////////
7048
7049  /** Make sure this is a valid row for the HRegion */
7050  void checkRow(byte[] row, String op) throws IOException {
7051    if (!rowIsInRange(getRegionInfo(), row)) {
7052      throw new WrongRegionException("Requested row out of range for " + op + " on HRegion " + this
7053        + ", startKey='" + Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='"
7054        + Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" + Bytes.toStringBinary(row)
7055        + "'");
7056    }
7057  }
7058
7059  /**
7060   * Get an exclusive ( write lock ) lock on a given row.
7061   * @param row Which row to lock.
7062   * @return A locked RowLock. The lock is exclusive and already aqquired.
7063   */
7064  public RowLock getRowLock(byte[] row) throws IOException {
7065    return getRowLock(row, false);
7066  }
7067
7068  @Override
7069  public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
7070    checkRow(row, "row lock");
7071    return getRowLock(row, readLock, null);
7072  }
7073
7074  Span createRegionSpan(String name) {
7075    return TraceUtil.createSpan(name).setAttribute(REGION_NAMES_KEY,
7076      Collections.singletonList(getRegionInfo().getRegionNameAsString()));
7077  }
7078
7079  // will be override in tests
7080  protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock)
7081    throws IOException {
7082    // create an object to use a a key in the row lock map
7083    HashedBytes rowKey = new HashedBytes(row);
7084
7085    RowLockContext rowLockContext = null;
7086    RowLockImpl result = null;
7087
7088    boolean success = false;
7089    try {
7090      // Keep trying until we have a lock or error out.
7091      // TODO: do we need to add a time component here?
7092      while (result == null) {
7093        rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey));
7094        // Now try an get the lock.
7095        // This can fail as
7096        if (readLock) {
7097          // For read lock, if the caller has locked the same row previously, it will not try
7098          // to acquire the same read lock. It simply returns the previous row lock.
7099          RowLockImpl prevRowLockImpl = (RowLockImpl) prevRowLock;
7100          if (
7101            (prevRowLockImpl != null)
7102              && (prevRowLockImpl.getLock() == rowLockContext.readWriteLock.readLock())
7103          ) {
7104            success = true;
7105            return prevRowLock;
7106          }
7107          result = rowLockContext.newReadLock();
7108        } else {
7109          result = rowLockContext.newWriteLock();
7110        }
7111      }
7112
7113      int timeout = rowLockWaitDuration;
7114      boolean reachDeadlineFirst = false;
7115      Optional<RpcCall> call = RpcServer.getCurrentCall();
7116      if (call.isPresent()) {
7117        long deadline = call.get().getDeadline();
7118        if (deadline < Long.MAX_VALUE) {
7119          int timeToDeadline = (int) (deadline - EnvironmentEdgeManager.currentTime());
7120          if (timeToDeadline <= this.rowLockWaitDuration) {
7121            reachDeadlineFirst = true;
7122            timeout = timeToDeadline;
7123          }
7124        }
7125      }
7126
7127      if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
7128        String message = "Timed out waiting for lock for row: " + rowKey + " in region "
7129          + getRegionInfo().getEncodedName();
7130        if (reachDeadlineFirst) {
7131          throw new TimeoutIOException(message);
7132        } else {
7133          // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request.
7134          throw new IOException(message);
7135        }
7136      }
7137      rowLockContext.setThreadName(Thread.currentThread().getName());
7138      success = true;
7139      return result;
7140    } catch (InterruptedException ie) {
7141      if (LOG.isDebugEnabled()) {
7142        LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
7143          getRegionInfo().getRegionNameAsString());
7144      }
7145      throw throwOnInterrupt(ie);
7146    } catch (Error error) {
7147      // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
7148      // is reached, it will throw out an Error. This Error needs to be caught so it can
7149      // go ahead to process the minibatch with lock acquired.
7150      LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
7151        getRegionInfo().getRegionNameAsString(), error);
7152      IOException ioe = new IOException(error);
7153      throw ioe;
7154    } finally {
7155      // Clean up the counts just in case this was the thing keeping the context alive.
7156      if (!success && rowLockContext != null) {
7157        rowLockContext.cleanUp();
7158      }
7159    }
7160  }
7161
7162  private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock)
7163    throws IOException {
7164    return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock),
7165      () -> createRegionSpan("Region.getRowLock").setAttribute(ROW_LOCK_READ_LOCK_KEY, readLock));
7166  }
7167
7168  private void releaseRowLocks(List<RowLock> rowLocks) {
7169    if (rowLocks != null) {
7170      for (RowLock rowLock : rowLocks) {
7171        rowLock.release();
7172      }
7173      rowLocks.clear();
7174    }
7175  }
7176
7177  public int getReadLockCount() {
7178    return lock.getReadLockCount();
7179  }
7180
7181  public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
7182    return lockedRows;
7183  }
7184
7185  class RowLockContext {
7186    private final HashedBytes row;
7187    final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
7188    final AtomicBoolean usable = new AtomicBoolean(true);
7189    final AtomicInteger count = new AtomicInteger(0);
7190    final Object lock = new Object();
7191    private String threadName;
7192
7193    RowLockContext(HashedBytes row) {
7194      this.row = row;
7195    }
7196
7197    RowLockImpl newWriteLock() {
7198      Lock l = readWriteLock.writeLock();
7199      return getRowLock(l);
7200    }
7201
7202    RowLockImpl newReadLock() {
7203      Lock l = readWriteLock.readLock();
7204      return getRowLock(l);
7205    }
7206
7207    private RowLockImpl getRowLock(Lock l) {
7208      count.incrementAndGet();
7209      synchronized (lock) {
7210        if (usable.get()) {
7211          return new RowLockImpl(this, l);
7212        } else {
7213          return null;
7214        }
7215      }
7216    }
7217
7218    void cleanUp() {
7219      long c = count.decrementAndGet();
7220      if (c <= 0) {
7221        synchronized (lock) {
7222          if (count.get() <= 0 && usable.get()) { // Don't attempt to remove row if already removed
7223            usable.set(false);
7224            RowLockContext removed = lockedRows.remove(row);
7225            assert removed == this : "we should never remove a different context";
7226          }
7227        }
7228      }
7229    }
7230
7231    public void setThreadName(String threadName) {
7232      this.threadName = threadName;
7233    }
7234
7235    @Override
7236    public String toString() {
7237      return "RowLockContext{" + "row=" + row + ", readWriteLock=" + readWriteLock + ", count="
7238        + count + ", threadName=" + threadName + '}';
7239    }
7240  }
7241
7242  /**
7243   * Class used to represent a lock on a row.
7244   */
7245  public static class RowLockImpl implements RowLock {
7246    private final RowLockContext context;
7247    private final Lock lock;
7248
7249    public RowLockImpl(RowLockContext context, Lock lock) {
7250      this.context = context;
7251      this.lock = lock;
7252    }
7253
7254    public Lock getLock() {
7255      return lock;
7256    }
7257
7258    public RowLockContext getContext() {
7259      return context;
7260    }
7261
7262    @Override
7263    public void release() {
7264      lock.unlock();
7265      context.cleanUp();
7266    }
7267
7268    @Override
7269    public String toString() {
7270      return "RowLockImpl{" + "context=" + context + ", lock=" + lock + '}';
7271    }
7272  }
7273
7274  /**
7275   * Determines whether multiple column families are present Precondition: familyPaths is not null
7276   * @param familyPaths List of (column family, hfilePath)
7277   */
7278  private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
7279    boolean multipleFamilies = false;
7280    byte[] family = null;
7281    for (Pair<byte[], String> pair : familyPaths) {
7282      byte[] fam = pair.getFirst();
7283      if (family == null) {
7284        family = fam;
7285      } else if (!Bytes.equals(family, fam)) {
7286        multipleFamilies = true;
7287        break;
7288      }
7289    }
7290    return multipleFamilies;
7291  }
7292
7293  /**
7294   * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
7295   * column families atomically.
7296   * @param familyPaths      List of Pair&lt;byte[] column family, String hfilePath&gt;
7297   * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be
7298   *                         bulk loaded
7299   * @return Map from family to List of store file paths if successful, null if failed recoverably
7300   * @throws IOException if failed unrecoverably.
7301   */
7302  public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
7303    boolean assignSeqId, BulkLoadListener bulkLoadListener) throws IOException {
7304    return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false, null, true);
7305  }
7306
7307  /**
7308   * Listener class to enable callers of bulkLoadHFile() to perform any necessary pre/post
7309   * processing of a given bulkload call
7310   */
7311  public interface BulkLoadListener {
7312    /**
7313     * Called before an HFile is actually loaded
7314     * @param family  family being loaded to
7315     * @param srcPath path of HFile
7316     * @return final path to be used for actual loading
7317     */
7318    String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile, String customStaging)
7319      throws IOException;
7320
7321    /**
7322     * Called after a successful HFile load
7323     * @param family  family being loaded to
7324     * @param srcPath path of HFile
7325     */
7326    void doneBulkLoad(byte[] family, String srcPath) throws IOException;
7327
7328    /**
7329     * Called after a failed HFile load
7330     * @param family  family being loaded to
7331     * @param srcPath path of HFile
7332     */
7333    void failedBulkLoad(byte[] family, String srcPath) throws IOException;
7334  }
7335
7336  /**
7337   * Attempts to atomically load a group of hfiles. This is critical for loading rows with multiple
7338   * column families atomically.
7339   * @param familyPaths      List of Pair&lt;byte[] column family, String hfilePath&gt;
7340   * @param bulkLoadListener Internal hooks enabling massaging/preparation of a file about to be
7341   *                         bulk loaded
7342   * @param copyFile         always copy hfiles if true
7343   * @param clusterIds       ids from clusters that had already handled the given bulkload event.
7344   * @return Map from family to List of store file paths if successful, null if failed recoverably
7345   * @throws IOException if failed unrecoverably.
7346   */
7347  public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
7348    boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile,
7349    List<String> clusterIds, boolean replicate) throws IOException {
7350    long seqId = -1;
7351    Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
7352    Map<String, Long> storeFilesSizes = new HashMap<>();
7353    Preconditions.checkNotNull(familyPaths);
7354    // we need writeLock for multi-family bulk load
7355    startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
7356    boolean isSuccessful = false;
7357    try {
7358      this.writeRequestsCount.increment();
7359
7360      // There possibly was a split that happened between when the split keys
7361      // were gathered and before the HRegion's write lock was taken. We need
7362      // to validate the HFile region before attempting to bulk load all of them
7363      IOException ioException = null;
7364      List<Pair<byte[], String>> failures = new ArrayList<>();
7365      for (Pair<byte[], String> p : familyPaths) {
7366        byte[] familyName = p.getFirst();
7367        String path = p.getSecond();
7368
7369        HStore store = getStore(familyName);
7370        if (store == null) {
7371          ioException = new org.apache.hadoop.hbase.DoNotRetryIOException(
7372            "No such column family " + Bytes.toStringBinary(familyName));
7373        } else {
7374          try {
7375            store.assertBulkLoadHFileOk(new Path(path));
7376          } catch (WrongRegionException wre) {
7377            // recoverable (file doesn't fit in region)
7378            failures.add(p);
7379          } catch (IOException ioe) {
7380            // unrecoverable (hdfs problem)
7381            ioException = ioe;
7382          }
7383        }
7384
7385        // validation failed because of some sort of IO problem.
7386        if (ioException != null) {
7387          LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this,
7388            ioException);
7389          throw ioException;
7390        }
7391      }
7392      // validation failed, bail out before doing anything permanent.
7393      if (failures.size() != 0) {
7394        StringBuilder list = new StringBuilder();
7395        for (Pair<byte[], String> p : failures) {
7396          list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
7397            .append(p.getSecond());
7398        }
7399        // problem when validating
7400        LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family,"
7401          + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this);
7402        return null;
7403      }
7404
7405      // We need to assign a sequential ID that's in between two memstores in order to preserve
7406      // the guarantee that all the edits lower than the highest sequential ID from all the
7407      // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is
7408      // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
7409      // a sequence id that we can be sure is beyond the last hfile written).
7410      if (assignSeqId) {
7411        FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY);
7412        if (fs.isFlushSucceeded()) {
7413          seqId = ((FlushResultImpl) fs).flushSequenceId;
7414        } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
7415          seqId = ((FlushResultImpl) fs).flushSequenceId;
7416        } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
7417          // CANNOT_FLUSH may mean that a flush is already on-going
7418          // we need to wait for that flush to complete
7419          waitForFlushes();
7420        } else {
7421          throw new IOException("Could not bulk load with an assigned sequential ID because the "
7422            + "flush didn't run. Reason for not flushing: " + ((FlushResultImpl) fs).failureReason);
7423        }
7424      }
7425
7426      Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
7427        new TreeMap<>(Bytes.BYTES_COMPARATOR);
7428      for (Pair<byte[], String> p : familyPaths) {
7429        byte[] familyName = p.getFirst();
7430        String path = p.getSecond();
7431        HStore store = getStore(familyName);
7432        if (!familyWithFinalPath.containsKey(familyName)) {
7433          familyWithFinalPath.put(familyName, new ArrayList<>());
7434        }
7435        List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
7436        String finalPath = path;
7437        try {
7438          boolean reqTmp = store.storeEngine.requireWritingToTmpDirFirst();
7439          if (bulkLoadListener != null) {
7440            finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile,
7441              reqTmp ? null : fs.getRegionDir().toString());
7442          }
7443          Pair<Path, Path> pair = null;
7444          if (reqTmp || !StoreFileInfo.isHFile(finalPath)) {
7445            pair = store.preBulkLoadHFile(finalPath, seqId);
7446          } else {
7447            Path livePath = new Path(finalPath);
7448            pair = new Pair<>(livePath, livePath);
7449          }
7450          lst.add(pair);
7451        } catch (IOException ioe) {
7452          // A failure here can cause an atomicity violation that we currently
7453          // cannot recover from since it is likely a failed HDFS operation.
7454
7455          LOG.error("There was a partial failure due to IO when attempting to" + " load "
7456            + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
7457          if (bulkLoadListener != null) {
7458            try {
7459              bulkLoadListener.failedBulkLoad(familyName, finalPath);
7460            } catch (Exception ex) {
7461              LOG.error("Error while calling failedBulkLoad for family "
7462                + Bytes.toString(familyName) + " with path " + path, ex);
7463            }
7464          }
7465          throw ioe;
7466        }
7467      }
7468
7469      if (this.getCoprocessorHost() != null) {
7470        for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7471          this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
7472        }
7473      }
7474      for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7475        byte[] familyName = entry.getKey();
7476        for (Pair<Path, Path> p : entry.getValue()) {
7477          String path = p.getFirst().toString();
7478          Path commitedStoreFile = p.getSecond();
7479          HStore store = getStore(familyName);
7480          try {
7481            store.bulkLoadHFile(familyName, path, commitedStoreFile);
7482            // Note the size of the store file
7483            try {
7484              FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
7485              storeFilesSizes.put(commitedStoreFile.getName(),
7486                fs.getFileStatus(commitedStoreFile).getLen());
7487            } catch (IOException e) {
7488              LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e);
7489              storeFilesSizes.put(commitedStoreFile.getName(), 0L);
7490            }
7491
7492            if (storeFiles.containsKey(familyName)) {
7493              storeFiles.get(familyName).add(commitedStoreFile);
7494            } else {
7495              List<Path> storeFileNames = new ArrayList<>();
7496              storeFileNames.add(commitedStoreFile);
7497              storeFiles.put(familyName, storeFileNames);
7498            }
7499            if (bulkLoadListener != null) {
7500              bulkLoadListener.doneBulkLoad(familyName, path);
7501            }
7502          } catch (IOException ioe) {
7503            // A failure here can cause an atomicity violation that we currently
7504            // cannot recover from since it is likely a failed HDFS operation.
7505
7506            // TODO Need a better story for reverting partial failures due to HDFS.
7507            LOG.error("There was a partial failure due to IO when attempting to" + " load "
7508              + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
7509            if (bulkLoadListener != null) {
7510              try {
7511                bulkLoadListener.failedBulkLoad(familyName, path);
7512              } catch (Exception ex) {
7513                LOG.error("Error while calling failedBulkLoad for family "
7514                  + Bytes.toString(familyName) + " with path " + path, ex);
7515              }
7516            }
7517            throw ioe;
7518          }
7519        }
7520      }
7521
7522      isSuccessful = true;
7523      if (conf.getBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, false)) {
7524        // request compaction
7525        familyWithFinalPath.keySet().forEach(family -> {
7526          HStore store = getStore(family);
7527          try {
7528            if (this.rsServices != null && store.needsCompaction()) {
7529              this.rsServices.getCompactionRequestor().requestSystemCompaction(this, store,
7530                "bulkload hfiles request compaction", true);
7531              LOG.info("Request compaction for region {} family {} after bulk load",
7532                this.getRegionInfo().getEncodedName(), store.getColumnFamilyName());
7533            }
7534          } catch (IOException e) {
7535            LOG.error("bulkload hfiles request compaction error ", e);
7536          }
7537        });
7538      }
7539    } finally {
7540      if (wal != null && !storeFiles.isEmpty()) {
7541        // Write a bulk load event for hfiles that are loaded
7542        try {
7543          WALProtos.BulkLoadDescriptor loadDescriptor =
7544            ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
7545              UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
7546              storeFiles, storeFilesSizes, seqId, clusterIds, replicate);
7547          WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
7548            loadDescriptor, mvcc, regionReplicationSink.orElse(null));
7549        } catch (IOException ioe) {
7550          if (this.rsServices != null) {
7551            // Have to abort region server because some hfiles has been loaded but we can't write
7552            // the event into WAL
7553            isSuccessful = false;
7554            this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
7555          }
7556        }
7557      }
7558
7559      closeBulkRegionOperation();
7560    }
7561    return isSuccessful ? storeFiles : null;
7562  }
7563
7564  @Override
7565  public boolean equals(Object o) {
7566    return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
7567      ((HRegion) o).getRegionInfo().getRegionName());
7568  }
7569
7570  @Override
7571  public int hashCode() {
7572    return Bytes.hashCode(getRegionInfo().getRegionName());
7573  }
7574
7575  @Override
7576  public String toString() {
7577    return getRegionInfo().getRegionNameAsString();
7578  }
7579
7580  // Utility methods
7581  /**
7582   * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL}
7583   * configuration property.
7584   * @param tableDir   qualified path of directory where region should be located, usually the table
7585   *                   directory.
7586   * @param wal        The WAL is the outbound log for any updates to the HRegion The wal file is a
7587   *                   logfile from the previous execution that's custom-computed for this HRegion.
7588   *                   The HRegionServer computes and sorts the appropriate wal info for this
7589   *                   HRegion. If there is a previous file (implying that the HRegion has been
7590   *                   written-to before), then read it from the supplied path.
7591   * @param fs         is the filesystem.
7592   * @param conf       is global configuration settings.
7593   * @param regionInfo - RegionInfo that describes the region is new), then read them from the
7594   *                   supplied path.
7595   * @param htd        the table descriptor
7596   * @return the new instance
7597   */
7598  public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs, Configuration conf,
7599    RegionInfo regionInfo, final TableDescriptor htd, RegionServerServices rsServices) {
7600    try {
7601      @SuppressWarnings("unchecked")
7602      Class<? extends HRegion> regionClass =
7603        (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
7604
7605      Constructor<? extends HRegion> c =
7606        regionClass.getConstructor(Path.class, WAL.class, FileSystem.class, Configuration.class,
7607          RegionInfo.class, TableDescriptor.class, RegionServerServices.class);
7608
7609      return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
7610    } catch (Throwable e) {
7611      // todo: what should I throw here?
7612      throw new IllegalStateException("Could not instantiate a region instance.", e);
7613    }
7614  }
7615
7616  /**
7617   * Convenience method creating new HRegions. Used by createTable.
7618   * @param info       Info for region to create.
7619   * @param rootDir    Root directory for HBase instance
7620   * @param wal        shared WAL
7621   * @param initialize - true to initialize the region
7622   * @return new HRegion
7623   */
7624  public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7625    final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7626    final boolean initialize) throws IOException {
7627    return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null);
7628  }
7629
7630  /**
7631   * Convenience method creating new HRegions. Used by createTable.
7632   * @param info          Info for region to create.
7633   * @param rootDir       Root directory for HBase instance
7634   * @param wal           shared WAL
7635   * @param initialize    - true to initialize the region
7636   * @param rsRpcServices An interface we can request flushes against.
7637   * @return new HRegion
7638   */
7639  public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7640    final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7641    final boolean initialize, RegionServerServices rsRpcServices) throws IOException {
7642    LOG.info("creating " + info + ", tableDescriptor="
7643      + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
7644    createRegionDir(conf, info, rootDir);
7645    FileSystem fs = rootDir.getFileSystem(conf);
7646    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7647    HRegion region =
7648      HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices);
7649    if (initialize) {
7650      region.initialize(null);
7651    }
7652    return region;
7653  }
7654
7655  /**
7656   * Create a region under the given table directory.
7657   */
7658  public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs,
7659    Path tableDir, TableDescriptor tableDesc) throws IOException {
7660    LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc,
7661      tableDir);
7662    HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo);
7663    HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null);
7664    return region;
7665  }
7666
7667  /**
7668   * Create the region directory in the filesystem.
7669   */
7670  public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri,
7671    Path rootDir) throws IOException {
7672    FileSystem fs = rootDir.getFileSystem(configuration);
7673    Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
7674    // If directory already exists, will log warning and keep going. Will try to create
7675    // .regioninfo. If one exists, will overwrite.
7676    return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
7677  }
7678
7679  public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7680    final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal)
7681    throws IOException {
7682    return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7683  }
7684
7685  /**
7686   * Open a Region.
7687   * @param info Info for region to be opened.
7688   * @param wal  WAL for region to use. This method will call WAL#setSequenceNumber(long) passing
7689   *             the result of the call to HRegion#getMinSequenceId() to ensure the wal id is
7690   *             properly kept up. HRegionStore does this every time it opens a new region.
7691   * @return new HRegion
7692   */
7693  public static HRegion openHRegion(final RegionInfo info, final TableDescriptor htd, final WAL wal,
7694    final Configuration conf) throws IOException {
7695    return openHRegion(info, htd, wal, conf, null, null);
7696  }
7697
7698  /**
7699   * Open a Region.
7700   * @param info       Info for region to be opened
7701   * @param htd        the table descriptor
7702   * @param wal        WAL for region to use. This method will call WAL#setSequenceNumber(long)
7703   *                   passing the result of the call to HRegion#getMinSequenceId() to ensure the
7704   *                   wal id is properly kept up. HRegionStore does this every time it opens a new
7705   *                   region.
7706   * @param conf       The Configuration object to use.
7707   * @param rsServices An interface we can request flushes against.
7708   * @param reporter   An interface we can report progress against.
7709   * @return new HRegion
7710   */
7711  public static HRegion openHRegion(final RegionInfo info, final TableDescriptor htd, final WAL wal,
7712    final Configuration conf, final RegionServerServices rsServices,
7713    final CancelableProgressable reporter) throws IOException {
7714    return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7715  }
7716
7717  /**
7718   * Open a Region.
7719   * @param rootDir Root directory for HBase instance
7720   * @param info    Info for region to be opened.
7721   * @param htd     the table descriptor
7722   * @param wal     WAL for region to use. This method will call WAL#setSequenceNumber(long) passing
7723   *                the result of the call to HRegion#getMinSequenceId() to ensure the wal id is
7724   *                properly kept up. HRegionStore does this every time it opens a new region.
7725   * @param conf    The Configuration object to use.
7726   * @return new HRegion
7727   */
7728  public static HRegion openHRegion(Path rootDir, final RegionInfo info, final TableDescriptor htd,
7729    final WAL wal, final Configuration conf) throws IOException {
7730    return openHRegion(rootDir, info, htd, wal, conf, null, null);
7731  }
7732
7733  /**
7734   * Open a Region.
7735   * @param rootDir    Root directory for HBase instance
7736   * @param info       Info for region to be opened.
7737   * @param htd        the table descriptor
7738   * @param wal        WAL for region to use. This method will call WAL#setSequenceNumber(long)
7739   *                   passing the result of the call to HRegion#getMinSequenceId() to ensure the
7740   *                   wal id is properly kept up. HRegionStore does this every time it opens a new
7741   *                   region.
7742   * @param conf       The Configuration object to use.
7743   * @param rsServices An interface we can request flushes against.
7744   * @param reporter   An interface we can report progress against.
7745   * @return new HRegion
7746   */
7747  public static HRegion openHRegion(final Path rootDir, final RegionInfo info,
7748    final TableDescriptor htd, final WAL wal, final Configuration conf,
7749    final RegionServerServices rsServices, final CancelableProgressable reporter)
7750    throws IOException {
7751    FileSystem fs = null;
7752    if (rsServices != null) {
7753      fs = rsServices.getFileSystem();
7754    }
7755    if (fs == null) {
7756      fs = rootDir.getFileSystem(conf);
7757    }
7758    return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7759  }
7760
7761  /**
7762   * Open a Region.
7763   * @param conf    The Configuration object to use.
7764   * @param fs      Filesystem to use
7765   * @param rootDir Root directory for HBase instance
7766   * @param info    Info for region to be opened.
7767   * @param htd     the table descriptor
7768   * @param wal     WAL for region to use. This method will call WAL#setSequenceNumber(long) passing
7769   *                the result of the call to HRegion#getMinSequenceId() to ensure the wal id is
7770   *                properly kept up. HRegionStore does this every time it opens a new region.
7771   * @return new HRegion
7772   */
7773  public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7774    final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal)
7775    throws IOException {
7776    return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7777  }
7778
7779  /**
7780   * Open a Region.
7781   * @param conf       The Configuration object to use.
7782   * @param fs         Filesystem to use
7783   * @param rootDir    Root directory for HBase instance
7784   * @param info       Info for region to be opened.
7785   * @param htd        the table descriptor
7786   * @param wal        WAL for region to use. This method will call WAL#setSequenceNumber(long)
7787   *                   passing the result of the call to HRegion#getMinSequenceId() to ensure the
7788   *                   wal id is properly kept up. HRegionStore does this every time it opens a new
7789   *                   region.
7790   * @param rsServices An interface we can request flushes against.
7791   * @param reporter   An interface we can report progress against.
7792   * @return new HRegion
7793   */
7794  public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7795    final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7796    final RegionServerServices rsServices, final CancelableProgressable reporter)
7797    throws IOException {
7798    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7799    return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
7800  }
7801
7802  /**
7803   * Open a Region.
7804   * @param conf       The Configuration object to use.
7805   * @param fs         Filesystem to use
7806   * @param info       Info for region to be opened.
7807   * @param htd        the table descriptor
7808   * @param wal        WAL for region to use. This method will call WAL#setSequenceNumber(long)
7809   *                   passing the result of the call to HRegion#getMinSequenceId() to ensure the
7810   *                   wal id is properly kept up. HRegionStore does this every time it opens a new
7811   *                   region.
7812   * @param rsServices An interface we can request flushes against.
7813   * @param reporter   An interface we can report progress against.
7814   * @return new HRegion
7815   * @throws NullPointerException if {@code info} is {@code null}
7816   */
7817  public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs,
7818    final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7819    final RegionServerServices rsServices, final CancelableProgressable reporter)
7820    throws IOException {
7821    Objects.requireNonNull(info, "RegionInfo cannot be null");
7822    LOG.debug("Opening region: {}", info);
7823    HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7824    return r.openHRegion(reporter);
7825  }
7826
7827  public NavigableMap<byte[], Integer> getReplicationScope() {
7828    return this.replicationScope;
7829  }
7830
7831  /**
7832   * Useful when reopening a closed region (normally for unit tests)
7833   * @param other    original object
7834   * @param reporter An interface we can report progress against.
7835   * @return new HRegion
7836   */
7837  public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7838    throws IOException {
7839    HRegionFileSystem regionFs = other.getRegionFileSystem();
7840    HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7841      other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null);
7842    return r.openHRegion(reporter);
7843  }
7844
7845  public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7846    throws IOException {
7847    return openHRegion((HRegion) other, reporter);
7848  }
7849
7850  /**
7851   * Open HRegion.
7852   * <p/>
7853   * Calls initialize and sets sequenceId.
7854   * @return Returns <code>this</code>
7855   */
7856  private HRegion openHRegion(final CancelableProgressable reporter) throws IOException {
7857    try {
7858      CompoundConfiguration cConfig =
7859        new CompoundConfiguration().add(conf).addBytesMap(htableDescriptor.getValues());
7860      // Refuse to open the region if we are missing local compression support
7861      TableDescriptorChecker.checkCompression(cConfig, htableDescriptor);
7862      // Refuse to open the region if encryption configuration is incorrect or
7863      // codec support is missing
7864      LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName());
7865      TableDescriptorChecker.checkEncryption(cConfig, htableDescriptor);
7866      // Refuse to open the region if a required class cannot be loaded
7867      LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName());
7868      TableDescriptorChecker.checkClassLoading(cConfig, htableDescriptor);
7869      this.openSeqNum = initialize(reporter);
7870      this.mvcc.advanceTo(openSeqNum);
7871      // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
7872      // determine whether a region has been successfully reopened. So here we always write open
7873      // marker, even if the table is read only.
7874      if (
7875        wal != null && getRegionServerServices() != null
7876          && RegionReplicaUtil.isDefaultReplica(getRegionInfo())
7877      ) {
7878        writeRegionOpenMarker(wal, openSeqNum);
7879      }
7880    } catch (Throwable t) {
7881      // By coprocessor path wrong region will open failed,
7882      // MetricsRegionWrapperImpl is already init and not close,
7883      // add region close when open failed
7884      try {
7885        // It is not required to write sequence id file when region open is failed.
7886        // Passing true to skip the sequence id file write.
7887        this.close(true);
7888      } catch (Throwable e) {
7889        LOG.warn("Open region: {} failed. Try close region but got exception ",
7890          this.getRegionInfo(), e);
7891      }
7892      throw t;
7893    }
7894    return this;
7895  }
7896
7897  /**
7898   * Open a Region on a read-only file-system (like hdfs snapshots)
7899   * @param conf The Configuration object to use.
7900   * @param fs   Filesystem to use
7901   * @param info Info for region to be opened.
7902   * @param htd  the table descriptor
7903   * @return new HRegion
7904   * @throws NullPointerException if {@code info} is {@code null}
7905   */
7906  public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7907    final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException {
7908    Objects.requireNonNull(info, "RegionInfo cannot be null");
7909    if (LOG.isDebugEnabled()) {
7910      LOG.debug("Opening region (readOnly filesystem): " + info);
7911    }
7912    if (info.getReplicaId() <= 0) {
7913      info = RegionReplicaUtil.getRegionInfoForReplica(info, 1);
7914    }
7915    HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7916    r.writestate.setReadOnly(true);
7917    return r.openHRegion(null);
7918  }
7919
7920  public static HRegion warmupHRegion(final RegionInfo info, final TableDescriptor htd,
7921    final WAL wal, final Configuration conf, final RegionServerServices rsServices,
7922    final CancelableProgressable reporter) throws IOException {
7923
7924    Objects.requireNonNull(info, "RegionInfo cannot be null");
7925    LOG.debug("Warmup {}", info);
7926    Path rootDir = CommonFSUtils.getRootDir(conf);
7927    Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7928    FileSystem fs = null;
7929    if (rsServices != null) {
7930      fs = rsServices.getFileSystem();
7931    }
7932    if (fs == null) {
7933      fs = rootDir.getFileSystem(conf);
7934    }
7935    HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7936    r.initializeWarmup(reporter);
7937    r.close();
7938    return r;
7939  }
7940
7941  /**
7942   * Computes the Path of the HRegion
7943   * @param tabledir qualified path for table
7944   * @param name     ENCODED region name
7945   * @return Path of HRegion directory
7946   * @deprecated For tests only; to be removed.
7947   */
7948  @Deprecated
7949  public static Path getRegionDir(final Path tabledir, final String name) {
7950    return new Path(tabledir, name);
7951  }
7952
7953  /**
7954   * Determines if the specified row is within the row range specified by the specified RegionInfo
7955   * @param info RegionInfo that specifies the row range
7956   * @param row  row to be checked
7957   * @return true if the row is within the range specified by the RegionInfo
7958   */
7959  public static boolean rowIsInRange(RegionInfo info, final byte[] row) {
7960    return ((info.getStartKey().length == 0) || (Bytes.compareTo(info.getStartKey(), row) <= 0))
7961      && ((info.getEndKey().length == 0) || (Bytes.compareTo(info.getEndKey(), row) > 0));
7962  }
7963
7964  public static boolean rowIsInRange(RegionInfo info, final byte[] row, final int offset,
7965    final short length) {
7966    return ((info.getStartKey().length == 0)
7967      || (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length, row, offset, length)
7968          <= 0))
7969      && ((info.getEndKey().length == 0)
7970        || (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length)
7971            > 0));
7972  }
7973
7974  @Override
7975  public Result get(final Get get) throws IOException {
7976    prepareGet(get);
7977    List<Cell> results = get(get, true);
7978    boolean stale = this.getRegionInfo().getReplicaId() != 0;
7979    return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7980  }
7981
7982  void prepareGet(final Get get) throws IOException {
7983    checkRow(get.getRow(), "Get");
7984    // Verify families are all valid
7985    if (get.hasFamilies()) {
7986      for (byte[] family : get.familySet()) {
7987        checkFamily(family);
7988      }
7989    } else { // Adding all families to scanner
7990      for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
7991        get.addFamily(family);
7992      }
7993    }
7994  }
7995
7996  @Override
7997  public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7998    return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7999  }
8000
8001  private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
8002    throws IOException {
8003    return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce),
8004      () -> createRegionSpan("Region.get"));
8005  }
8006
8007  private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
8008    throws IOException {
8009    List<Cell> results = new ArrayList<>();
8010
8011    // pre-get CP hook
8012    if (withCoprocessor && (coprocessorHost != null)) {
8013      if (coprocessorHost.preGet(get, results)) {
8014        metricsUpdateForGet();
8015        return results;
8016      }
8017    }
8018    Scan scan = new Scan(get);
8019    if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
8020      scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
8021    }
8022    try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) {
8023      List<Cell> tmp = new ArrayList<>();
8024      scanner.next(tmp);
8025      // Copy EC to heap, then close the scanner.
8026      // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers.
8027      // See more details in HBASE-26036.
8028      for (Cell cell : tmp) {
8029        results.add(CellUtil.cloneIfNecessary(cell));
8030      }
8031    }
8032
8033    // post-get CP hook
8034    if (withCoprocessor && (coprocessorHost != null)) {
8035      coprocessorHost.postGet(get, results);
8036    }
8037
8038    metricsUpdateForGet();
8039
8040    return results;
8041  }
8042
8043  void metricsUpdateForGet() {
8044    if (this.metricsRegion != null) {
8045      this.metricsRegion.updateGet();
8046    }
8047    if (this.rsServices != null && this.rsServices.getMetrics() != null) {
8048      rsServices.getMetrics().updateReadQueryMeter(this, 1);
8049    }
8050
8051  }
8052
8053  @Override
8054  public Result mutateRow(RowMutations rm) throws IOException {
8055    return mutateRow(rm, HConstants.NO_NONCE, HConstants.NO_NONCE);
8056  }
8057
8058  public Result mutateRow(RowMutations rm, long nonceGroup, long nonce) throws IOException {
8059    final List<Mutation> m = rm.getMutations();
8060    OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, nonceGroup, nonce);
8061
8062    List<Result> results = new ArrayList<>();
8063    for (OperationStatus status : statuses) {
8064      if (status.getResult() != null) {
8065        results.add(status.getResult());
8066      }
8067    }
8068
8069    if (results.isEmpty()) {
8070      return null;
8071    }
8072
8073    // Merge the results of the Increment/Append operations
8074    List<Cell> cells = new ArrayList<>();
8075    for (Result result : results) {
8076      if (result.rawCells() != null) {
8077        cells.addAll(Arrays.asList(result.rawCells()));
8078      }
8079    }
8080    return Result.create(cells);
8081  }
8082
8083  /**
8084   * Perform atomic (all or none) mutations within the region.
8085   * @param mutations  The list of mutations to perform. <code>mutations</code> can contain
8086   *                   operations for multiple rows. Caller has to ensure that all rows are
8087   *                   contained in this region.
8088   * @param rowsToLock Rows to lock
8089   * @param nonceGroup Optional nonce group of the operation (client Id)
8090   * @param nonce      Optional nonce of the operation (unique random id to ensure "more
8091   *                   idempotence") If multiple rows are locked care should be taken that
8092   *                   <code>rowsToLock</code> is sorted in order to avoid deadlocks.
8093   */
8094  @Override
8095  public void mutateRowsWithLocks(Collection<Mutation> mutations, Collection<byte[]> rowsToLock,
8096    long nonceGroup, long nonce) throws IOException {
8097    batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]),
8098      true, nonceGroup, nonce) {
8099      @Override
8100      public MiniBatchOperationInProgress<Mutation>
8101        lockRowsAndBuildMiniBatch(List<RowLock> acquiredRowLocks) throws IOException {
8102        RowLock prevRowLock = null;
8103        for (byte[] row : rowsToLock) {
8104          try {
8105            RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock
8106            if (rowLock != prevRowLock) {
8107              acquiredRowLocks.add(rowLock);
8108              prevRowLock = rowLock;
8109            }
8110          } catch (IOException ioe) {
8111            LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this,
8112              ioe);
8113            throw ioe;
8114          }
8115        }
8116        return createMiniBatch(size(), size());
8117      }
8118    });
8119  }
8120
8121  /** Returns statistics about the current load of the region */
8122  public ClientProtos.RegionLoadStats getLoadStatistics() {
8123    if (!regionStatsEnabled) {
8124      return null;
8125    }
8126    ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
8127    stats.setMemStoreLoad((int) (Math.min(100,
8128      (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize)));
8129    if (rsServices.getHeapMemoryManager() != null) {
8130      // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
8131      // so we could just do the calculation below and we'll get a 0.
8132      // treating it as a special case analogous to no HMM instead so that it can be
8133      // programatically treated different from using <1% of heap.
8134      final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
8135      if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
8136        stats.setHeapOccupancy((int) (occupancy * 100));
8137      }
8138    }
8139    stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100
8140      ? 100
8141      : rsServices.getCompactionPressure() * 100));
8142    return stats.build();
8143  }
8144
8145  @Override
8146  public Result append(Append append) throws IOException {
8147    return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
8148  }
8149
8150  public Result append(Append append, long nonceGroup, long nonce) throws IOException {
8151    return TraceUtil.trace(() -> {
8152      checkReadOnly();
8153      checkResources();
8154      startRegionOperation(Operation.APPEND);
8155      try {
8156        // All edits for the given row (across all column families) must happen atomically.
8157        return mutate(append, true, nonceGroup, nonce).getResult();
8158      } finally {
8159        closeRegionOperation(Operation.APPEND);
8160      }
8161    }, () -> createRegionSpan("Region.append"));
8162  }
8163
8164  @Override
8165  public Result increment(Increment increment) throws IOException {
8166    return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
8167  }
8168
8169  public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException {
8170    return TraceUtil.trace(() -> {
8171      checkReadOnly();
8172      checkResources();
8173      startRegionOperation(Operation.INCREMENT);
8174      try {
8175        // All edits for the given row (across all column families) must happen atomically.
8176        return mutate(increment, true, nonceGroup, nonce).getResult();
8177      } finally {
8178        closeRegionOperation(Operation.INCREMENT);
8179      }
8180    }, () -> createRegionSpan("Region.increment"));
8181  }
8182
8183  private WALKeyImpl createWALKeyForWALAppend(boolean isReplay, BatchOperation<?> batchOp, long now,
8184    long nonceGroup, long nonce) {
8185    WALKeyImpl walKey = isReplay
8186      ? new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
8187        this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now,
8188        batchOp.getClusterIds(), nonceGroup, nonce, mvcc)
8189      : new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
8190        this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now,
8191        batchOp.getClusterIds(), nonceGroup, nonce, mvcc, this.getReplicationScope());
8192    if (isReplay) {
8193      walKey.setOrigLogSeqNum(batchOp.getOrigLogSeqNum());
8194    }
8195    return walKey;
8196  }
8197
8198  /** Returns writeEntry associated with this append */
8199  private WriteEntry doWALAppend(WALEdit walEdit, BatchOperation<?> batchOp,
8200    MiniBatchOperationInProgress<Mutation> miniBatchOp, long now, NonceKey nonceKey)
8201    throws IOException {
8202    Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(), "WALEdit is null or empty!");
8203    Preconditions.checkArgument(
8204      !walEdit.isReplay() || batchOp.getOrigLogSeqNum() != SequenceId.NO_SEQUENCE_ID,
8205      "Invalid replay sequence Id for replay WALEdit!");
8206
8207    WALKeyImpl walKey = createWALKeyForWALAppend(walEdit.isReplay(), batchOp, now,
8208      nonceKey.getNonceGroup(), nonceKey.getNonce());
8209    // don't call the coproc hook for writes to the WAL caused by
8210    // system lifecycle events like flushes or compactions
8211    if (this.coprocessorHost != null && !walEdit.isMetaEdit()) {
8212      this.coprocessorHost.preWALAppend(walKey, walEdit);
8213    }
8214    try {
8215      long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
8216      WriteEntry writeEntry = walKey.getWriteEntry();
8217      // Call sync on our edit.
8218      if (txid != 0) {
8219        sync(txid, batchOp.durability);
8220      }
8221      /**
8222       * If above {@link HRegion#sync} throws Exception, the RegionServer should be aborted and
8223       * following {@link BatchOperation#writeMiniBatchOperationsToMemStore} will not be executed,
8224       * so there is no need to replicate to secondary replica, for this reason here we attach the
8225       * region replication action after the {@link HRegion#sync} is successful.
8226       */
8227      this.attachRegionReplicationInWALAppend(batchOp, miniBatchOp, walKey, walEdit, writeEntry);
8228      return writeEntry;
8229    } catch (IOException ioe) {
8230      if (walKey.getWriteEntry() != null) {
8231        mvcc.complete(walKey.getWriteEntry());
8232      }
8233
8234      /**
8235       * If {@link WAL#sync} get a timeout exception, the only correct way is to abort the region
8236       * server, as the design of {@link WAL#sync}, is to succeed or die, there is no 'failure'. It
8237       * is usually not a big deal is because we set a very large default value(5 minutes) for
8238       * {@link AbstractFSWAL#WAL_SYNC_TIMEOUT_MS}, usually the WAL system will abort the region
8239       * server if it can not finish the sync within 5 minutes.
8240       */
8241      if (ioe instanceof WALSyncTimeoutIOException) {
8242        if (rsServices != null) {
8243          rsServices.abort("WAL sync timeout,forcing server shutdown", ioe);
8244        }
8245      }
8246      throw ioe;
8247    }
8248  }
8249
8250  /**
8251   * Attach {@link RegionReplicationSink#add} to the mvcc writeEntry for replicating to region
8252   * replica.
8253   */
8254  private void attachRegionReplicationInWALAppend(BatchOperation<?> batchOp,
8255    MiniBatchOperationInProgress<Mutation> miniBatchOp, WALKeyImpl walKey, WALEdit walEdit,
8256    WriteEntry writeEntry) {
8257    if (!regionReplicationSink.isPresent()) {
8258      return;
8259    }
8260    /**
8261     * If {@link HRegion#regionReplicationSink} is present,only {@link MutationBatchOperation} is
8262     * used and {@link NonceKey} is all the same for {@link Mutation}s in
8263     * {@link MutationBatchOperation},so for HBASE-26993 case 1,if
8264     * {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} is not null and we could
8265     * enter {@link HRegion#doWALAppend},that means partial {@link Mutation}s are
8266     * {@link Durability#SKIP_WAL}, we use
8267     * {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} to replicate to region
8268     * replica,but if {@link MiniBatchOperationInProgress#getWalEditForReplicateSkipWAL} is
8269     * null,that means there is no {@link Mutation} is {@link Durability#SKIP_WAL},so we just use
8270     * walEdit to replicate.
8271     */
8272    assert batchOp instanceof MutationBatchOperation;
8273    WALEdit walEditToUse = miniBatchOp.getWalEditForReplicateIfExistsSkipWAL();
8274    if (walEditToUse == null) {
8275      walEditToUse = walEdit;
8276    }
8277    doAttachReplicateRegionReplicaAction(walKey, walEditToUse, writeEntry);
8278  }
8279
8280  /**
8281   * Attach {@link RegionReplicationSink#add} to the mvcc writeEntry for replicating to region
8282   * replica.
8283   */
8284  private void doAttachReplicateRegionReplicaAction(WALKeyImpl walKey, WALEdit walEdit,
8285    WriteEntry writeEntry) {
8286    if (walEdit == null || walEdit.isEmpty()) {
8287      return;
8288    }
8289    final ServerCall<?> rpcCall = RpcServer.getCurrentServerCallWithCellScanner().orElse(null);
8290    regionReplicationSink.ifPresent(sink -> writeEntry.attachCompletionAction(() -> {
8291      sink.add(walKey, walEdit, rpcCall);
8292    }));
8293  }
8294
8295  public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false);
8296
8297  // woefully out of date - currently missing:
8298  // 1 x HashMap - coprocessorServiceHandlers
8299  // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
8300  // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount,
8301  // writeRequestsCount, cpRequestsCount
8302  // 1 x HRegion$WriteState - writestate
8303  // 1 x RegionCoprocessorHost - coprocessorHost
8304  // 1 x RegionSplitPolicy - splitPolicy
8305  // 1 x MetricsRegion - metricsRegion
8306  // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper
8307  // 1 x ReadPointCalculationLock - smallestReadPointCalcLock
8308  public static final long DEEP_OVERHEAD = FIXED_OVERHEAD + ClassSize.OBJECT + // closeLock
8309    (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
8310    (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL,
8311                                  // compactionsFailed
8312    (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders
8313    WriteState.HEAP_SIZE + // writestate
8314    ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
8315    (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
8316    MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
8317    + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes
8318    + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
8319    + ClassSize.STORE_SERVICES // store services
8320    + StoreHotnessProtector.FIXED_SIZE;
8321
8322  @Override
8323  public long heapSize() {
8324    // this does not take into account row locks, recent flushes, mvcc entries, and more
8325    return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum();
8326  }
8327
8328  /**
8329   * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
8330   * available for handling {@link #execService(RpcController, CoprocessorServiceCall)} calls.
8331   * <p/>
8332   * Only a single instance may be registered per region for a given {@link Service} subclass (the
8333   * instances are keyed on {@link ServiceDescriptor#getFullName()}.. After the first registration,
8334   * subsequent calls with the same service name will fail with a return value of {@code false}.
8335   * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint
8336   * @return {@code true} if the registration was successful, {@code false} otherwise
8337   */
8338  public boolean registerService(Service instance) {
8339    // No stacking of instances is allowed for a single service name
8340    ServiceDescriptor serviceDesc = instance.getDescriptorForType();
8341    String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
8342    if (coprocessorServiceHandlers.containsKey(serviceName)) {
8343      LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}",
8344        serviceName, instance, this);
8345      return false;
8346    }
8347
8348    coprocessorServiceHandlers.put(serviceName, instance);
8349    if (LOG.isDebugEnabled()) {
8350      LOG.debug("Registered coprocessor service: region="
8351        + Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName);
8352    }
8353    return true;
8354  }
8355
8356  /**
8357   * Executes a single protocol buffer coprocessor endpoint {@link Service} method using the
8358   * registered protocol handlers. {@link Service} implementations must be registered via the
8359   * {@link #registerService(Service)} method before they are available.
8360   * @param controller an {@code RpcContoller} implementation to pass to the invoked service
8361   * @param call       a {@code CoprocessorServiceCall} instance identifying the service, method,
8362   *                   and parameters for the method invocation
8363   * @return a protocol buffer {@code Message} instance containing the method's result
8364   * @throws IOException if no registered service handler is found or an error occurs during the
8365   *                     invocation
8366   * @see #registerService(Service)
8367   */
8368  public Message execService(RpcController controller, CoprocessorServiceCall call)
8369    throws IOException {
8370    String serviceName = call.getServiceName();
8371    Service service = coprocessorServiceHandlers.get(serviceName);
8372    if (service == null) {
8373      throw new UnknownProtocolException(null, "No registered coprocessor service found for "
8374        + serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName()));
8375    }
8376    ServiceDescriptor serviceDesc = service.getDescriptorForType();
8377
8378    cpRequestsCount.increment();
8379    String methodName = call.getMethodName();
8380    MethodDescriptor methodDesc = CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
8381
8382    Message.Builder builder = service.getRequestPrototype(methodDesc).newBuilderForType();
8383
8384    ProtobufUtil.mergeFrom(builder, call.getRequest().toByteArray());
8385    Message request = CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
8386
8387    if (coprocessorHost != null) {
8388      request = coprocessorHost.preEndpointInvocation(service, methodName, request);
8389    }
8390
8391    final Message.Builder responseBuilder =
8392      service.getResponsePrototype(methodDesc).newBuilderForType();
8393    service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
8394      @Override
8395      public void run(Message message) {
8396        if (message != null) {
8397          responseBuilder.mergeFrom(message);
8398        }
8399      }
8400    });
8401
8402    if (coprocessorHost != null) {
8403      coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
8404    }
8405    IOException exception =
8406      org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller);
8407    if (exception != null) {
8408      throw exception;
8409    }
8410
8411    return responseBuilder.build();
8412  }
8413
8414  public Optional<byte[]> checkSplit() {
8415    return checkSplit(false);
8416  }
8417
8418  /**
8419   * Return the split point. An empty result indicates the region isn't splittable.
8420   */
8421  public Optional<byte[]> checkSplit(boolean force) {
8422    // Can't split META
8423    if (this.getRegionInfo().isMetaRegion()) {
8424      return Optional.empty();
8425    }
8426
8427    // Can't split a region that is closing.
8428    if (this.isClosing()) {
8429      return Optional.empty();
8430    }
8431
8432    if (!force && !splitPolicy.shouldSplit()) {
8433      return Optional.empty();
8434    }
8435
8436    byte[] ret = splitPolicy.getSplitPoint();
8437    if (ret != null && ret.length > 0) {
8438      ret = splitRestriction.getRestrictedSplitPoint(ret);
8439    }
8440
8441    if (ret != null) {
8442      try {
8443        checkRow(ret, "calculated split");
8444      } catch (IOException e) {
8445        LOG.error("Ignoring invalid split for region {}", this, e);
8446        return Optional.empty();
8447      }
8448      return Optional.of(ret);
8449    } else {
8450      return Optional.empty();
8451    }
8452  }
8453
8454  /** Returns The priority that this region should have in the compaction queue */
8455  public int getCompactPriority() {
8456    if (checkSplit().isPresent() && conf.getBoolean(SPLIT_IGNORE_BLOCKING_ENABLED_KEY, false)) {
8457      // if a region should split, split it before compact
8458      return Store.PRIORITY_USER;
8459    }
8460    return stores.values().stream().mapToInt(HStore::getCompactPriority).min()
8461      .orElse(Store.NO_PRIORITY);
8462  }
8463
8464  /** Returns the coprocessor host */
8465  public RegionCoprocessorHost getCoprocessorHost() {
8466    return coprocessorHost;
8467  }
8468
8469  /** @param coprocessorHost the new coprocessor host */
8470  public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
8471    this.coprocessorHost = coprocessorHost;
8472  }
8473
8474  @Override
8475  public void startRegionOperation() throws IOException {
8476    startRegionOperation(Operation.ANY);
8477  }
8478
8479  @Override
8480  public void startRegionOperation(Operation op) throws IOException {
8481    boolean isInterruptableOp = false;
8482    switch (op) {
8483      case GET: // interruptible read operations
8484      case SCAN:
8485        isInterruptableOp = true;
8486        checkReadsEnabled();
8487        break;
8488      case INCREMENT: // interruptible write operations
8489      case APPEND:
8490      case PUT:
8491      case DELETE:
8492      case BATCH_MUTATE:
8493      case CHECK_AND_MUTATE:
8494        isInterruptableOp = true;
8495        break;
8496      default: // all others
8497        break;
8498    }
8499    if (
8500      op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION || op == Operation.COMPACT_REGION
8501        || op == Operation.COMPACT_SWITCH
8502    ) {
8503      // split, merge or compact region doesn't need to check the closing/closed state or lock the
8504      // region
8505      return;
8506    }
8507    if (this.closing.get()) {
8508      throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8509    }
8510    lock(lock.readLock());
8511    // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from
8512    // an RPC handler
8513    Thread thisThread = Thread.currentThread();
8514    if (isInterruptableOp) {
8515      regionLockHolders.put(thisThread, true);
8516    }
8517    if (this.closed.get()) {
8518      lock.readLock().unlock();
8519      throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8520    }
8521    // The unit for snapshot is a region. So, all stores for this region must be
8522    // prepared for snapshot operation before proceeding.
8523    if (op == Operation.SNAPSHOT) {
8524      stores.values().forEach(HStore::preSnapshotOperation);
8525    }
8526    try {
8527      if (coprocessorHost != null) {
8528        coprocessorHost.postStartRegionOperation(op);
8529      }
8530    } catch (Exception e) {
8531      if (isInterruptableOp) {
8532        // would be harmless to remove what we didn't add but we know by 'isInterruptableOp'
8533        // if we added this thread to regionLockHolders
8534        regionLockHolders.remove(thisThread);
8535      }
8536      lock.readLock().unlock();
8537      throw new IOException(e);
8538    }
8539  }
8540
8541  @Override
8542  public void closeRegionOperation() throws IOException {
8543    closeRegionOperation(Operation.ANY);
8544  }
8545
8546  @Override
8547  public void closeRegionOperation(Operation operation) throws IOException {
8548    if (operation == Operation.SNAPSHOT) {
8549      stores.values().forEach(HStore::postSnapshotOperation);
8550    }
8551    Thread thisThread = Thread.currentThread();
8552    regionLockHolders.remove(thisThread);
8553    lock.readLock().unlock();
8554    if (coprocessorHost != null) {
8555      coprocessorHost.postCloseRegionOperation(operation);
8556    }
8557  }
8558
8559  /**
8560   * This method needs to be called before any public call that reads or modifies stores in bulk. It
8561   * has to be called just before a try. #closeBulkRegionOperation needs to be called in the try's
8562   * finally block Acquires a writelock and checks if the region is closing or closed.
8563   * @throws NotServingRegionException when the region is closing or closed
8564   * @throws RegionTooBusyException    if failed to get the lock in time
8565   * @throws InterruptedIOException    if interrupted while waiting for a lock
8566   */
8567  private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8568    if (this.closing.get()) {
8569      throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8570    }
8571    if (writeLockNeeded) lock(lock.writeLock());
8572    else lock(lock.readLock());
8573    if (this.closed.get()) {
8574      if (writeLockNeeded) lock.writeLock().unlock();
8575      else lock.readLock().unlock();
8576      throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8577    }
8578    regionLockHolders.put(Thread.currentThread(), true);
8579  }
8580
8581  /**
8582   * Closes the lock. This needs to be called in the finally block corresponding to the try block of
8583   * #startRegionOperation
8584   */
8585  private void closeBulkRegionOperation() {
8586    regionLockHolders.remove(Thread.currentThread());
8587    if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8588    else lock.readLock().unlock();
8589  }
8590
8591  /**
8592   * Update LongAdders for number of puts without wal and the size of possible data loss. These
8593   * information are exposed by the region server metrics.
8594   */
8595  private void recordMutationWithoutWal(final Map<byte[], List<Cell>> familyMap) {
8596    numMutationsWithoutWAL.increment();
8597    if (numMutationsWithoutWAL.sum() <= 1) {
8598      LOG.info("writing data to region " + this
8599        + " with WAL disabled. Data may be lost in the event of a crash.");
8600    }
8601
8602    long mutationSize = 0;
8603    for (List<Cell> cells : familyMap.values()) {
8604      // Optimization: 'foreach' loop is not used. See:
8605      // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
8606      assert cells instanceof RandomAccess;
8607      int listSize = cells.size();
8608      for (int i = 0; i < listSize; i++) {
8609        Cell cell = cells.get(i);
8610        mutationSize += cell.getSerializedSize();
8611      }
8612    }
8613
8614    dataInMemoryWithoutWAL.add(mutationSize);
8615  }
8616
8617  private void lock(final Lock lock) throws IOException {
8618    lock(lock, 1);
8619  }
8620
8621  /**
8622   * Try to acquire a lock. Throw RegionTooBusyException if failed to get the lock in time. Throw
8623   * InterruptedIOException if interrupted while waiting for the lock.
8624   */
8625  private void lock(final Lock lock, final int multiplier) throws IOException {
8626    try {
8627      final long waitTime = Math.min(maxBusyWaitDuration,
8628        busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
8629      if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
8630        // Don't print millis. Message is used as a key over in
8631        // RetriesExhaustedWithDetailsException processing.
8632        final String regionName =
8633          this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
8634        final String serverName = this.getRegionServerServices() == null
8635          ? "unknown"
8636          : (this.getRegionServerServices().getServerName() == null
8637            ? "unknown"
8638            : this.getRegionServerServices().getServerName().toString());
8639        RegionTooBusyException rtbe = new RegionTooBusyException(
8640          "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName);
8641        LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
8642        throw rtbe;
8643      }
8644    } catch (InterruptedException ie) {
8645      if (LOG.isDebugEnabled()) {
8646        LOG.debug("Interrupted while waiting for a lock in region {}", this);
8647      }
8648      throw throwOnInterrupt(ie);
8649    }
8650  }
8651
8652  /**
8653   * Calls sync with the given transaction ID
8654   * @param txid should sync up to which transaction
8655   * @throws IOException If anything goes wrong with DFS
8656   */
8657  private void sync(long txid, Durability durability) throws IOException {
8658    if (this.getRegionInfo().isMetaRegion()) {
8659      this.wal.sync(txid);
8660    } else {
8661      switch (durability) {
8662        case USE_DEFAULT:
8663          // do what table defaults to
8664          if (shouldSyncWAL()) {
8665            this.wal.sync(txid);
8666          }
8667          break;
8668        case SKIP_WAL:
8669          // nothing do to
8670          break;
8671        case ASYNC_WAL:
8672          // nothing do to
8673          break;
8674        case SYNC_WAL:
8675          this.wal.sync(txid, false);
8676          break;
8677        case FSYNC_WAL:
8678          this.wal.sync(txid, true);
8679          break;
8680        default:
8681          throw new RuntimeException("Unknown durability " + durability);
8682      }
8683    }
8684  }
8685
8686  /**
8687   * Check whether we should sync the wal from the table's durability settings
8688   */
8689  private boolean shouldSyncWAL() {
8690    return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal();
8691  }
8692
8693  /** Returns the latest sequence number that was read from storage when this region was opened */
8694  public long getOpenSeqNum() {
8695    return this.openSeqNum;
8696  }
8697
8698  @Override
8699  public Map<byte[], Long> getMaxStoreSeqId() {
8700    return this.maxSeqIdInStores;
8701  }
8702
8703  public long getOldestSeqIdOfStore(byte[] familyName) {
8704    return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
8705  }
8706
8707  @Override
8708  public CompactionState getCompactionState() {
8709    boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
8710    return (hasMajor
8711      ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
8712      : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
8713  }
8714
8715  public void reportCompactionRequestStart(boolean isMajor) {
8716    (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
8717  }
8718
8719  public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
8720    int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
8721
8722    // metrics
8723    compactionsFinished.increment();
8724    compactionNumFilesCompacted.add(numFiles);
8725    compactionNumBytesCompacted.add(filesSizeCompacted);
8726
8727    assert newValue >= 0;
8728  }
8729
8730  public void reportCompactionRequestFailure() {
8731    compactionsFailed.increment();
8732  }
8733
8734  public void incrementCompactionsQueuedCount() {
8735    compactionsQueued.increment();
8736  }
8737
8738  public void decrementCompactionsQueuedCount() {
8739    compactionsQueued.decrement();
8740  }
8741
8742  public void incrementFlushesQueuedCount() {
8743    flushesQueued.increment();
8744  }
8745
8746  protected void decrementFlushesQueuedCount() {
8747    flushesQueued.decrement();
8748  }
8749
8750  /**
8751   * If a handler thread is eligible for interrupt, make it ineligible. Should be paired with
8752   * {{@link #enableInterrupts()}.
8753   */
8754  void disableInterrupts() {
8755    regionLockHolders.computeIfPresent(Thread.currentThread(), (t, b) -> false);
8756  }
8757
8758  /**
8759   * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()}, make
8760   * it eligible again. No-op if interrupts are already enabled.
8761   */
8762  void enableInterrupts() {
8763    regionLockHolders.computeIfPresent(Thread.currentThread(), (t, b) -> true);
8764  }
8765
8766  /**
8767   * Interrupt any region options that have acquired the region lock via
8768   * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)}, or
8769   * {@link #startBulkRegionOperation(boolean)}.
8770   */
8771  private void interruptRegionOperations() {
8772    for (Map.Entry<Thread, Boolean> entry : regionLockHolders.entrySet()) {
8773      // An entry in this map will have a boolean value indicating if it is currently
8774      // eligible for interrupt; if so, we should interrupt it.
8775      if (entry.getValue().booleanValue()) {
8776        entry.getKey().interrupt();
8777      }
8778    }
8779  }
8780
8781  /**
8782   * Check thread interrupt status and throw an exception if interrupted.
8783   * @throws NotServingRegionException if region is closing
8784   * @throws InterruptedIOException    if interrupted but region is not closing
8785   */
8786  // Package scope for tests
8787  void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
8788    if (Thread.interrupted()) {
8789      if (this.closing.get()) {
8790        throw new NotServingRegionException(
8791          getRegionInfo().getRegionNameAsString() + " is closing");
8792      }
8793      throw new InterruptedIOException();
8794    }
8795  }
8796
8797  /**
8798   * Throw the correct exception upon interrupt
8799   * @param t cause
8800   */
8801  // Package scope for tests
8802  IOException throwOnInterrupt(Throwable t) {
8803    if (this.closing.get()) {
8804      return (NotServingRegionException) new NotServingRegionException(
8805        getRegionInfo().getRegionNameAsString() + " is closing").initCause(t);
8806    }
8807    return (InterruptedIOException) new InterruptedIOException().initCause(t);
8808  }
8809
8810  /**
8811   * {@inheritDoc}
8812   */
8813  @Override
8814  public void onConfigurationChange(Configuration conf) {
8815    this.storeHotnessProtector.update(conf);
8816    // update coprocessorHost if the configuration has changed.
8817    if (
8818      CoprocessorConfigurationUtil.checkConfigurationChange(getReadOnlyConfiguration(), conf,
8819        CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8820        CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY)
8821    ) {
8822      LOG.info("Update the system coprocessors because the configuration has changed");
8823      decorateRegionConfiguration(conf);
8824      this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
8825    }
8826  }
8827
8828  /**
8829   * {@inheritDoc}
8830   */
8831  @Override
8832  public void registerChildren(ConfigurationManager manager) {
8833    configurationManager = manager;
8834    stores.values().forEach(manager::registerObserver);
8835  }
8836
8837  /**
8838   * {@inheritDoc}
8839   */
8840  @Override
8841  public void deregisterChildren(ConfigurationManager manager) {
8842    stores.values().forEach(configurationManager::deregisterObserver);
8843  }
8844
8845  @Override
8846  public CellComparator getCellComparator() {
8847    return cellComparator;
8848  }
8849
8850  public long getMemStoreFlushSize() {
8851    return this.memstoreFlushSize;
8852  }
8853
8854  //// method for debugging tests
8855  void throwException(String title, String regionName) {
8856    StringBuilder buf = new StringBuilder();
8857    buf.append(title + ", ");
8858    buf.append(getRegionInfo().toString());
8859    buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
8860    buf.append("stores: ");
8861    for (HStore s : stores.values()) {
8862      buf.append(s.getColumnFamilyDescriptor().getNameAsString());
8863      buf.append(" size: ");
8864      buf.append(s.getMemStoreSize().getDataSize());
8865      buf.append(" ");
8866    }
8867    buf.append("end-of-stores");
8868    buf.append(", memstore size ");
8869    buf.append(getMemStoreDataSize());
8870    if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
8871      throw new RuntimeException(buf.toString());
8872    }
8873  }
8874
8875  @Override
8876  public void requestCompaction(String why, int priority, boolean major,
8877    CompactionLifeCycleTracker tracker) throws IOException {
8878    if (major) {
8879      stores.values().forEach(HStore::triggerMajorCompaction);
8880    }
8881    rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker,
8882      RpcServer.getRequestUser().orElse(null));
8883  }
8884
8885  @Override
8886  public void requestCompaction(byte[] family, String why, int priority, boolean major,
8887    CompactionLifeCycleTracker tracker) throws IOException {
8888    HStore store = stores.get(family);
8889    if (store == null) {
8890      throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family)
8891        + " does not exist in region " + getRegionInfo().getRegionNameAsString());
8892    }
8893    if (major) {
8894      store.triggerMajorCompaction();
8895    }
8896    rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker,
8897      RpcServer.getRequestUser().orElse(null));
8898  }
8899
8900  private void requestFlushIfNeeded() throws RegionTooBusyException {
8901    if (isFlushSize(this.memStoreSizing.getMemStoreSize())) {
8902      requestFlush();
8903    }
8904  }
8905
8906  private void requestFlush() {
8907    if (this.rsServices == null) {
8908      return;
8909    }
8910    requestFlush0(FlushLifeCycleTracker.DUMMY);
8911  }
8912
8913  private void requestFlush0(FlushLifeCycleTracker tracker) {
8914    boolean shouldFlush = false;
8915    synchronized (writestate) {
8916      if (!this.writestate.isFlushRequested()) {
8917        shouldFlush = true;
8918        writestate.flushRequested = true;
8919      }
8920    }
8921    if (shouldFlush) {
8922      // Make request outside of synchronize block; HBASE-818.
8923      this.rsServices.getFlushRequester().requestFlush(this, tracker);
8924      if (LOG.isDebugEnabled()) {
8925        LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
8926      }
8927    } else {
8928      tracker.notExecuted("Flush already requested on " + this);
8929    }
8930  }
8931
8932  @Override
8933  public void requestFlush(FlushLifeCycleTracker tracker) throws IOException {
8934    requestFlush0(tracker);
8935  }
8936
8937  /**
8938   * This method modifies the region's configuration in order to inject replication-related features
8939   * @param conf region configurations
8940   */
8941  private static void decorateRegionConfiguration(Configuration conf) {
8942    if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
8943      String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, "");
8944      String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName();
8945      if (!plugins.contains(replicationCoprocessorClass)) {
8946        conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8947          (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass);
8948      }
8949    }
8950  }
8951
8952  public Optional<RegionReplicationSink> getRegionReplicationSink() {
8953    return regionReplicationSink;
8954  }
8955
8956  public void addReadRequestsCount(long readRequestsCount) {
8957    this.readRequestsCount.add(readRequestsCount);
8958  }
8959
8960  public void addWriteRequestsCount(long writeRequestsCount) {
8961    this.writeRequestsCount.add(writeRequestsCount);
8962  }
8963
8964  @RestrictedApi(explanation = "Should only be called in tests", link = "",
8965      allowedOnPath = ".*/src/test/.*")
8966  boolean isReadsEnabled() {
8967    return this.writestate.readsEnabled;
8968  }
8969}