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