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