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