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