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