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