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