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