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