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