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