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