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