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