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