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