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