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