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