HBASE-26286: Add support for specifying store file tracker when restoring or cloning...
[hbase.git] / hbase-server / src / main / java / org / apache / hadoop / hbase / regionserver / HRegion.java
blob63992dc97a46d7d2c8e507cf968158c084770d09
1 /*
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements. See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership. The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License. You may obtain a copy of the License at
10 * http://www.apache.org/licenses/LICENSE-2.0
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
18 package org.apache.hadoop.hbase.regionserver;
20 import static org.apache.hadoop.hbase.HConstants.REPLICATION_SCOPE_LOCAL;
21 import static org.apache.hadoop.hbase.regionserver.HStoreFile.MAJOR_COMPACTION_KEY;
22 import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.REGION_NAMES_KEY;
23 import static org.apache.hadoop.hbase.trace.HBaseSemanticAttributes.ROW_LOCK_READ_LOCK_KEY;
24 import static org.apache.hadoop.hbase.util.ConcurrentMapUtils.computeIfAbsent;
26 import edu.umd.cs.findbugs.annotations.Nullable;
27 import io.opentelemetry.api.trace.Span;
28 import java.io.EOFException;
29 import java.io.FileNotFoundException;
30 import java.io.IOException;
31 import java.io.InterruptedIOException;
32 import java.lang.reflect.Constructor;
33 import java.nio.ByteBuffer;
34 import java.nio.charset.StandardCharsets;
35 import java.text.ParseException;
36 import java.util.ArrayList;
37 import java.util.Arrays;
38 import java.util.Collection;
39 import java.util.Collections;
40 import java.util.HashMap;
41 import java.util.HashSet;
42 import java.util.Iterator;
43 import java.util.List;
44 import java.util.Map;
45 import java.util.Map.Entry;
46 import java.util.NavigableMap;
47 import java.util.NavigableSet;
48 import java.util.Objects;
49 import java.util.Optional;
50 import java.util.RandomAccess;
51 import java.util.Set;
52 import java.util.TreeMap;
53 import java.util.UUID;
54 import java.util.concurrent.Callable;
55 import java.util.concurrent.CompletionService;
56 import java.util.concurrent.ConcurrentHashMap;
57 import java.util.concurrent.ConcurrentMap;
58 import java.util.concurrent.ConcurrentSkipListMap;
59 import java.util.concurrent.ExecutionException;
60 import java.util.concurrent.ExecutorCompletionService;
61 import java.util.concurrent.Future;
62 import java.util.concurrent.ThreadFactory;
63 import java.util.concurrent.ThreadPoolExecutor;
64 import java.util.concurrent.TimeUnit;
65 import java.util.concurrent.atomic.AtomicBoolean;
66 import java.util.concurrent.atomic.AtomicInteger;
67 import java.util.concurrent.atomic.LongAdder;
68 import java.util.concurrent.locks.Lock;
69 import java.util.concurrent.locks.ReadWriteLock;
70 import java.util.concurrent.locks.ReentrantReadWriteLock;
71 import java.util.function.Function;
72 import java.util.stream.Collectors;
73 import java.util.stream.Stream;
75 import org.apache.hadoop.conf.Configuration;
76 import org.apache.hadoop.fs.FileStatus;
77 import org.apache.hadoop.fs.FileSystem;
78 import org.apache.hadoop.fs.LocatedFileStatus;
79 import org.apache.hadoop.fs.Path;
80 import org.apache.hadoop.hbase.ByteBufferExtendedCell;
81 import org.apache.hadoop.hbase.Cell;
82 import org.apache.hadoop.hbase.CellBuilderType;
83 import org.apache.hadoop.hbase.CellComparator;
84 import org.apache.hadoop.hbase.CellComparatorImpl;
85 import org.apache.hadoop.hbase.CellScanner;
86 import org.apache.hadoop.hbase.CellUtil;
87 import org.apache.hadoop.hbase.CompareOperator;
88 import org.apache.hadoop.hbase.CompoundConfiguration;
89 import org.apache.hadoop.hbase.DoNotRetryIOException;
90 import org.apache.hadoop.hbase.DroppedSnapshotException;
91 import org.apache.hadoop.hbase.ExtendedCellBuilderFactory;
92 import org.apache.hadoop.hbase.HConstants;
93 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
94 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
95 import org.apache.hadoop.hbase.KeyValue;
96 import org.apache.hadoop.hbase.MetaCellComparator;
97 import org.apache.hadoop.hbase.NamespaceDescriptor;
98 import org.apache.hadoop.hbase.NotServingRegionException;
99 import org.apache.hadoop.hbase.PrivateCellUtil;
100 import org.apache.hadoop.hbase.RegionTooBusyException;
101 import org.apache.hadoop.hbase.Tag;
102 import org.apache.hadoop.hbase.TagUtil;
103 import org.apache.hadoop.hbase.client.Append;
104 import org.apache.hadoop.hbase.client.CheckAndMutate;
105 import org.apache.hadoop.hbase.client.CheckAndMutateResult;
106 import org.apache.hadoop.hbase.client.ColumnFamilyDescriptor;
107 import org.apache.hadoop.hbase.client.CompactionState;
108 import org.apache.hadoop.hbase.client.Delete;
109 import org.apache.hadoop.hbase.client.Durability;
110 import org.apache.hadoop.hbase.client.Get;
111 import org.apache.hadoop.hbase.client.Increment;
112 import org.apache.hadoop.hbase.client.IsolationLevel;
113 import org.apache.hadoop.hbase.client.Mutation;
114 import org.apache.hadoop.hbase.client.Put;
115 import org.apache.hadoop.hbase.client.RegionInfo;
116 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
117 import org.apache.hadoop.hbase.client.Result;
118 import org.apache.hadoop.hbase.client.Row;
119 import org.apache.hadoop.hbase.client.RowMutations;
120 import org.apache.hadoop.hbase.client.Scan;
121 import org.apache.hadoop.hbase.client.TableDescriptor;
122 import org.apache.hadoop.hbase.client.TableDescriptorBuilder;
123 import org.apache.hadoop.hbase.conf.ConfigurationManager;
124 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
125 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
126 import org.apache.hadoop.hbase.coprocessor.ReadOnlyConfiguration;
127 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
128 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
129 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
130 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
131 import org.apache.hadoop.hbase.filter.BinaryComparator;
132 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
133 import org.apache.hadoop.hbase.filter.Filter;
134 import org.apache.hadoop.hbase.io.HFileLink;
135 import org.apache.hadoop.hbase.io.HeapSize;
136 import org.apache.hadoop.hbase.io.TimeRange;
137 import org.apache.hadoop.hbase.io.hfile.BlockCache;
138 import org.apache.hadoop.hbase.io.hfile.HFile;
139 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
140 import org.apache.hadoop.hbase.ipc.RpcCall;
141 import org.apache.hadoop.hbase.ipc.RpcServer;
142 import org.apache.hadoop.hbase.mob.MobFileCache;
143 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
144 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
145 import org.apache.hadoop.hbase.quotas.RegionServerSpaceQuotaManager;
146 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
147 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
148 import org.apache.hadoop.hbase.regionserver.compactions.CompactionLifeCycleTracker;
149 import org.apache.hadoop.hbase.regionserver.compactions.ForbidMajorCompactionChecker;
150 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
151 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
152 import org.apache.hadoop.hbase.regionserver.throttle.StoreHotnessProtector;
153 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
154 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
155 import org.apache.hadoop.hbase.replication.ReplicationUtils;
156 import org.apache.hadoop.hbase.replication.regionserver.ReplicationObserver;
157 import org.apache.hadoop.hbase.security.User;
158 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
159 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
160 import org.apache.hadoop.hbase.trace.TraceUtil;
161 import org.apache.hadoop.hbase.util.Bytes;
162 import org.apache.hadoop.hbase.util.CancelableProgressable;
163 import org.apache.hadoop.hbase.util.ClassSize;
164 import org.apache.hadoop.hbase.util.CommonFSUtils;
165 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
166 import org.apache.hadoop.hbase.util.FSUtils;
167 import org.apache.hadoop.hbase.util.HashedBytes;
168 import org.apache.hadoop.hbase.util.NonceKey;
169 import org.apache.hadoop.hbase.util.Pair;
170 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
171 import org.apache.hadoop.hbase.util.TableDescriptorChecker;
172 import org.apache.hadoop.hbase.util.Threads;
173 import org.apache.hadoop.hbase.wal.WAL;
174 import org.apache.hadoop.hbase.wal.WALEdit;
175 import org.apache.hadoop.hbase.wal.WALFactory;
176 import org.apache.hadoop.hbase.wal.WALKey;
177 import org.apache.hadoop.hbase.wal.WALKeyImpl;
178 import org.apache.hadoop.hbase.wal.WALSplitUtil;
179 import org.apache.hadoop.hbase.wal.WALSplitUtil.MutationReplay;
180 import org.apache.hadoop.util.StringUtils;
181 import org.apache.yetus.audience.InterfaceAudience;
182 import org.slf4j.Logger;
183 import org.slf4j.LoggerFactory;
185 import org.apache.hbase.thirdparty.com.google.common.base.Preconditions;
186 import org.apache.hbase.thirdparty.com.google.common.collect.Iterables;
187 import org.apache.hbase.thirdparty.com.google.common.collect.Lists;
188 import org.apache.hbase.thirdparty.com.google.common.collect.Maps;
189 import org.apache.hbase.thirdparty.com.google.common.io.Closeables;
190 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.MethodDescriptor;
191 import org.apache.hbase.thirdparty.com.google.protobuf.Descriptors.ServiceDescriptor;
192 import org.apache.hbase.thirdparty.com.google.protobuf.Message;
193 import org.apache.hbase.thirdparty.com.google.protobuf.RpcCallback;
194 import org.apache.hbase.thirdparty.com.google.protobuf.RpcController;
195 import org.apache.hbase.thirdparty.com.google.protobuf.Service;
196 import org.apache.hbase.thirdparty.com.google.protobuf.TextFormat;
197 import org.apache.hbase.thirdparty.com.google.protobuf.UnsafeByteOperations;
198 import org.apache.hbase.thirdparty.org.apache.commons.collections4.CollectionUtils;
199 import org.apache.hadoop.hbase.shaded.protobuf.ProtobufUtil;
200 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos;
201 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClientProtos.CoprocessorServiceCall;
202 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.RegionLoad;
203 import org.apache.hadoop.hbase.shaded.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
204 import org.apache.hadoop.hbase.shaded.protobuf.generated.SnapshotProtos.SnapshotDescription;
205 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos;
206 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.CompactionDescriptor;
207 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor;
208 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
209 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
210 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor;
211 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
212 import org.apache.hadoop.hbase.shaded.protobuf.generated.WALProtos.StoreDescriptor;
215 * Regions store data for a certain region of a table. It stores all columns
216 * for each row. A given table consists of one or more Regions.
218 * <p>An Region is defined by its table and its key extent.
220 * <p>Locking at the Region level serves only one purpose: preventing the
221 * region from being closed (and consequently split) while other operations
222 * are ongoing. Each row level operation obtains both a row lock and a region
223 * read lock for the duration of the operation. While a scanner is being
224 * constructed, getScanner holds a read lock. If the scanner is successfully
225 * constructed, it holds a read lock until it is closed. A close takes out a
226 * write lock and consequently will block for ongoing operations and will block
227 * new operations from starting while the close is in progress.
229 @SuppressWarnings("deprecation")
230 @InterfaceAudience.Private
231 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
232 private static final Logger LOG = LoggerFactory.getLogger(HRegion.class);
234 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
235 "hbase.hregion.scan.loadColumnFamiliesOnDemand";
237 public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
238 public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
240 public static final String HBASE_REGIONSERVER_MINIBATCH_SIZE =
241 "hbase.regionserver.minibatch.size";
242 public static final int DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE = 20000;
244 public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
245 public static final boolean DEFAULT_WAL_HSYNC = false;
247 /** Parameter name for compaction after bulkload */
248 public static final String COMPACTION_AFTER_BULKLOAD_ENABLE =
249 "hbase.compaction.after.bulkload.enable";
252 * This is for for using HRegion as a local storage, where we may put the recovered edits in a
253 * special place. Once this is set, we will only replay the recovered edits under this directory
254 * and ignore the original replay directory configs.
256 public static final String SPECIAL_RECOVERED_EDITS_DIR =
257 "hbase.hregion.special.recovered.edits.dir";
260 * Whether to use {@link MetaCellComparator} even if we are not meta region. Used when creating
261 * master local region.
263 public static final String USE_META_CELL_COMPARATOR = "hbase.region.use.meta.cell.comparator";
265 public static final boolean DEFAULT_USE_META_CELL_COMPARATOR = false;
267 final AtomicBoolean closed = new AtomicBoolean(false);
269 /* Closing can take some time; use the closing flag if there is stuff we don't
270 * want to do while in closing state; e.g. like offer this region up to the
271 * master as a region to close if the carrying regionserver is overloaded.
272 * Once set, it is never cleared.
274 final AtomicBoolean closing = new AtomicBoolean(false);
277 * The max sequence id of flushed data on this region. There is no edit in memory that is
278 * less that this sequence id.
280 private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM;
283 * Record the sequence id of last flush operation. Can be in advance of
284 * {@link #maxFlushedSeqId} when flushing a single column family. In this case,
285 * {@link #maxFlushedSeqId} will be older than the oldest edit in memory.
287 private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM;
290 * The sequence id of the last replayed open region event from the primary region. This is used
291 * to skip entries before this due to the possibility of replay edits coming out of order from
292 * replication.
294 protected volatile long lastReplayedOpenRegionSeqId = -1L;
295 protected volatile long lastReplayedCompactionSeqId = -1L;
297 //////////////////////////////////////////////////////////////////////////////
298 // Members
299 //////////////////////////////////////////////////////////////////////////////
301 // map from a locked row to the context for that lock including:
302 // - CountDownLatch for threads waiting on that row
303 // - the thread that owns the lock (allow reentrancy)
304 // - reference count of (reentrant) locks held by the thread
305 // - the row itself
306 private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
307 new ConcurrentHashMap<>();
309 protected final Map<byte[], HStore> stores =
310 new ConcurrentSkipListMap<>(Bytes.BYTES_RAWCOMPARATOR);
312 // TODO: account for each registered handler in HeapSize computation
313 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
315 // Track data size in all memstores
316 private final MemStoreSizing memStoreSizing = new ThreadSafeMemStoreSizing();
317 RegionServicesForStores regionServicesForStores;
319 // Debug possible data loss due to WAL off
320 final LongAdder numMutationsWithoutWAL = new LongAdder();
321 final LongAdder dataInMemoryWithoutWAL = new LongAdder();
323 // Debug why CAS operations are taking a while.
324 final LongAdder checkAndMutateChecksPassed = new LongAdder();
325 final LongAdder checkAndMutateChecksFailed = new LongAdder();
327 // Number of requests
328 // Count rows for scan
329 final LongAdder readRequestsCount = new LongAdder();
330 final LongAdder cpRequestsCount = new LongAdder();
331 final LongAdder filteredReadRequestsCount = new LongAdder();
332 // Count rows for multi row mutations
333 final LongAdder writeRequestsCount = new LongAdder();
335 // Number of requests blocked by memstore size.
336 private final LongAdder blockedRequestsCount = new LongAdder();
338 // Compaction LongAdders
339 final LongAdder compactionsFinished = new LongAdder();
340 final LongAdder compactionsFailed = new LongAdder();
341 final LongAdder compactionNumFilesCompacted = new LongAdder();
342 final LongAdder compactionNumBytesCompacted = new LongAdder();
343 final LongAdder compactionsQueued = new LongAdder();
344 final LongAdder flushesQueued = new LongAdder();
346 private BlockCache blockCache;
347 private MobFileCache mobFileCache;
348 private final WAL wal;
349 private final HRegionFileSystem fs;
350 protected final Configuration conf;
351 private final Configuration baseConf;
352 private final int rowLockWaitDuration;
353 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
355 private Path regionDir;
356 private FileSystem walFS;
358 // set to true if the region is restored from snapshot
359 private boolean isRestoredRegion = false;
361 public void setRestoredRegion(boolean restoredRegion) {
362 isRestoredRegion = restoredRegion;
365 // The internal wait duration to acquire a lock before read/update
366 // from the region. It is not per row. The purpose of this wait time
367 // is to avoid waiting a long time while the region is busy, so that
368 // we can release the IPC handler soon enough to improve the
369 // availability of the region server. It can be adjusted by
370 // tuning configuration "hbase.busy.wait.duration".
371 final long busyWaitDuration;
372 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
374 // If updating multiple rows in one call, wait longer,
375 // i.e. waiting for busyWaitDuration * # of rows. However,
376 // we can limit the max multiplier.
377 final int maxBusyWaitMultiplier;
379 // Max busy wait duration. There is no point to wait longer than the RPC
380 // purge timeout, when a RPC call will be terminated by the RPC engine.
381 final long maxBusyWaitDuration;
383 // Max cell size. If nonzero, the maximum allowed size for any given cell
384 // in bytes
385 final long maxCellSize;
387 // Number of mutations for minibatch processing.
388 private final int miniBatchSize;
390 final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
393 * The sequence ID that was enLongAddered when this region was opened.
395 private long openSeqNum = HConstants.NO_SEQNUM;
398 * The default setting for whether to enable on-demand CF loading for
399 * scan requests to this region. Requests can override it.
401 private boolean isLoadingCfsOnDemandDefault = false;
403 private final AtomicInteger majorInProgress = new AtomicInteger(0);
404 private final AtomicInteger minorInProgress = new AtomicInteger(0);
407 // Context: During replay we want to ensure that we do not lose any data. So, we
408 // have to be conservative in how we replay wals. For each store, we calculate
409 // the maxSeqId up to which the store was flushed. And, skip the edits which
410 // are equal to or lower than maxSeqId for each store.
411 // The following map is populated when opening the region
412 Map<byte[], Long> maxSeqIdInStores = new TreeMap<>(Bytes.BYTES_COMPARATOR);
414 /** Saved state from replaying prepare flush cache */
415 private PrepareFlushResult prepareFlushResult = null;
417 private volatile ConfigurationManager configurationManager;
419 // Used for testing.
420 private volatile Long timeoutForWriteLock = null;
422 private final CellComparator cellComparator;
425 * @return The smallest mvcc readPoint across all the scanners in this
426 * region. Writes older than this readPoint, are included in every
427 * read operation.
429 public long getSmallestReadPoint() {
430 long minimumReadPoint;
431 // We need to ensure that while we are calculating the smallestReadPoint
432 // no new RegionScanners can grab a readPoint that we are unaware of.
433 // We achieve this by synchronizing on the scannerReadPoints object.
434 synchronized (scannerReadPoints) {
435 minimumReadPoint = mvcc.getReadPoint();
436 for (Long readPoint : this.scannerReadPoints.values()) {
437 if (readPoint < minimumReadPoint) {
438 minimumReadPoint = readPoint;
442 return minimumReadPoint;
446 * Data structure of write state flags used coordinating flushes,
447 * compactions and closes.
449 static class WriteState {
450 // Set while a memstore flush is happening.
451 volatile boolean flushing = false;
452 // Set when a flush has been requested.
453 volatile boolean flushRequested = false;
454 // Number of compactions running.
455 AtomicInteger compacting = new AtomicInteger(0);
456 // Gets set in close. If set, cannot compact or flush again.
457 volatile boolean writesEnabled = true;
458 // Set if region is read-only
459 volatile boolean readOnly = false;
460 // whether the reads are enabled. This is different than readOnly, because readOnly is
461 // static in the lifetime of the region, while readsEnabled is dynamic
462 volatile boolean readsEnabled = true;
465 * Set flags that make this region read-only.
467 * @param onOff flip value for region r/o setting
469 synchronized void setReadOnly(final boolean onOff) {
470 this.writesEnabled = !onOff;
471 this.readOnly = onOff;
474 boolean isReadOnly() {
475 return this.readOnly;
478 boolean isFlushRequested() {
479 return this.flushRequested;
482 void setReadsEnabled(boolean readsEnabled) {
483 this.readsEnabled = readsEnabled;
486 static final long HEAP_SIZE = ClassSize.align(
487 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
491 * Objects from this class are created when flushing to describe all the different states that
492 * that method ends up in. The Result enum describes those states. The sequence id should only
493 * be specified if the flush was successful, and the failure message should only be specified
494 * if it didn't flush.
496 public static class FlushResultImpl implements FlushResult {
497 final Result result;
498 final String failureReason;
499 final long flushSequenceId;
500 final boolean wroteFlushWalMarker;
503 * Convenience constructor to use when the flush is successful, the failure message is set to
504 * null.
505 * @param result Expecting FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_COMPACTION_NEEDED.
506 * @param flushSequenceId Generated sequence id that comes right after the edits in the
507 * memstores.
509 FlushResultImpl(Result result, long flushSequenceId) {
510 this(result, flushSequenceId, null, false);
511 assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
512 .FLUSHED_COMPACTION_NEEDED;
516 * Convenience constructor to use when we cannot flush.
517 * @param result Expecting CANNOT_FLUSH_MEMSTORE_EMPTY or CANNOT_FLUSH.
518 * @param failureReason Reason why we couldn't flush.
520 FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) {
521 this(result, -1, failureReason, wroteFlushMarker);
522 assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
526 * Constructor with all the parameters.
527 * @param result Any of the Result.
528 * @param flushSequenceId Generated sequence id if the memstores were flushed else -1.
529 * @param failureReason Reason why we couldn't flush, or null.
531 FlushResultImpl(Result result, long flushSequenceId, String failureReason,
532 boolean wroteFlushMarker) {
533 this.result = result;
534 this.flushSequenceId = flushSequenceId;
535 this.failureReason = failureReason;
536 this.wroteFlushWalMarker = wroteFlushMarker;
540 * Convenience method, the equivalent of checking if result is
541 * FLUSHED_NO_COMPACTION_NEEDED or FLUSHED_NO_COMPACTION_NEEDED.
542 * @return true if the memstores were flushed, else false.
544 @Override
545 public boolean isFlushSucceeded() {
546 return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
547 .FLUSHED_COMPACTION_NEEDED;
551 * Convenience method, the equivalent of checking if result is FLUSHED_COMPACTION_NEEDED.
552 * @return True if the flush requested a compaction, else false (doesn't even mean it flushed).
554 @Override
555 public boolean isCompactionNeeded() {
556 return result == Result.FLUSHED_COMPACTION_NEEDED;
559 @Override
560 public String toString() {
561 return new StringBuilder()
562 .append("flush result:").append(result).append(", ")
563 .append("failureReason:").append(failureReason).append(",")
564 .append("flush seq id").append(flushSequenceId).toString();
567 @Override
568 public Result getResult() {
569 return result;
573 /** A result object from prepare flush cache stage */
574 static class PrepareFlushResult {
575 final FlushResultImpl result; // indicating a failure result from prepare
576 final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
577 final TreeMap<byte[], List<Path>> committedFiles;
578 final TreeMap<byte[], MemStoreSize> storeFlushableSize;
579 final long startTime;
580 final long flushOpSeqId;
581 final long flushedSeqId;
582 final MemStoreSizing totalFlushableSize;
584 /** Constructs an early exit case */
585 PrepareFlushResult(FlushResultImpl result, long flushSeqId) {
586 this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, MemStoreSizing.DUD);
589 /** Constructs a successful prepare flush result */
590 PrepareFlushResult(
591 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
592 TreeMap<byte[], List<Path>> committedFiles,
593 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
594 long flushedSeqId, MemStoreSizing totalFlushableSize) {
595 this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
596 flushSeqId, flushedSeqId, totalFlushableSize);
599 private PrepareFlushResult(
600 FlushResultImpl result,
601 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
602 TreeMap<byte[], List<Path>> committedFiles,
603 TreeMap<byte[], MemStoreSize> storeFlushableSize, long startTime, long flushSeqId,
604 long flushedSeqId, MemStoreSizing totalFlushableSize) {
605 this.result = result;
606 this.storeFlushCtxs = storeFlushCtxs;
607 this.committedFiles = committedFiles;
608 this.storeFlushableSize = storeFlushableSize;
609 this.startTime = startTime;
610 this.flushOpSeqId = flushSeqId;
611 this.flushedSeqId = flushedSeqId;
612 this.totalFlushableSize = totalFlushableSize;
615 public FlushResult getResult() {
616 return this.result;
621 * A class that tracks exceptions that have been observed in one batch. Not thread safe.
623 static class ObservedExceptionsInBatch {
624 private boolean wrongRegion = false;
625 private boolean failedSanityCheck = false;
626 private boolean wrongFamily = false;
629 * @return If a {@link WrongRegionException} has been observed.
631 boolean hasSeenWrongRegion() {
632 return wrongRegion;
636 * Records that a {@link WrongRegionException} has been observed.
638 void sawWrongRegion() {
639 wrongRegion = true;
643 * @return If a {@link FailedSanityCheckException} has been observed.
645 boolean hasSeenFailedSanityCheck() {
646 return failedSanityCheck;
650 * Records that a {@link FailedSanityCheckException} has been observed.
652 void sawFailedSanityCheck() {
653 failedSanityCheck = true;
657 * @return If a {@link NoSuchColumnFamilyException} has been observed.
659 boolean hasSeenNoSuchFamily() {
660 return wrongFamily;
664 * Records that a {@link NoSuchColumnFamilyException} has been observed.
666 void sawNoSuchFamily() {
667 wrongFamily = true;
671 final WriteState writestate = new WriteState();
673 long memstoreFlushSize;
674 final long timestampSlop;
676 // Last flush time for each Store. Useful when we are flushing for each column
677 private final ConcurrentMap<HStore, Long> lastStoreFlushTimeMap = new ConcurrentHashMap<>();
679 protected RegionServerServices rsServices;
680 private RegionServerAccounting rsAccounting;
681 private long flushCheckInterval;
682 // flushPerChanges is to prevent too many changes in memstore
683 private long flushPerChanges;
684 private long blockingMemStoreSize;
685 // Used to guard closes
686 final ReentrantReadWriteLock lock;
687 // Used to track interruptible holders of the region lock. Currently that is only RPC handler
688 // threads. Boolean value in map determines if lock holder can be interrupted, normally true,
689 // but may be false when thread is transiting a critical section.
690 final ConcurrentHashMap<Thread, Boolean> regionLockHolders;
692 // Stop updates lock
693 private final ReentrantReadWriteLock updatesLock = new ReentrantReadWriteLock();
695 private final MultiVersionConcurrencyControl mvcc;
697 // Coprocessor host
698 private RegionCoprocessorHost coprocessorHost;
700 private TableDescriptor htableDescriptor = null;
701 private RegionSplitPolicy splitPolicy;
702 private RegionSplitRestriction splitRestriction;
703 private FlushPolicy flushPolicy;
705 private final MetricsRegion metricsRegion;
706 private final MetricsRegionWrapperImpl metricsRegionWrapper;
707 private final Durability regionDurability;
708 private final boolean regionStatsEnabled;
709 // Stores the replication scope of the various column families of the table
710 // that has non-default scope
711 private final NavigableMap<byte[], Integer> replicationScope = new TreeMap<>(
712 Bytes.BYTES_COMPARATOR);
714 private final StoreHotnessProtector storeHotnessProtector;
717 * HRegion constructor. This constructor should only be used for testing and
718 * extensions. Instances of HRegion should be instantiated with the
719 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
721 * @param tableDir qualified path of directory where region should be located,
722 * usually the table directory.
723 * @param wal The WAL is the outbound log for any updates to the HRegion
724 * The wal file is a logfile from the previous execution that's
725 * custom-computed for this HRegion. The HRegionServer computes and sorts the
726 * appropriate wal info for this HRegion. If there is a previous wal file
727 * (implying that the HRegion has been written-to before), then read it from
728 * the supplied path.
729 * @param fs is the filesystem.
730 * @param confParam is global configuration settings.
731 * @param regionInfo - RegionInfo that describes the region
732 * is new), then read them from the supplied path.
733 * @param htd the table descriptor
734 * @param rsServices reference to {@link RegionServerServices} or null
735 * @deprecated Use other constructors.
737 @Deprecated
738 public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
739 final Configuration confParam, final RegionInfo regionInfo,
740 final TableDescriptor htd, final RegionServerServices rsServices) {
741 this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
742 wal, confParam, htd, rsServices);
746 * HRegion constructor. This constructor should only be used for testing and
747 * extensions. Instances of HRegion should be instantiated with the
748 * {@link HRegion#createHRegion} or {@link HRegion#openHRegion} method.
750 * @param fs is the filesystem.
751 * @param wal The WAL is the outbound log for any updates to the HRegion
752 * The wal file is a logfile from the previous execution that's
753 * custom-computed for this HRegion. The HRegionServer computes and sorts the
754 * appropriate wal info for this HRegion. If there is a previous wal file
755 * (implying that the HRegion has been written-to before), then read it from
756 * the supplied path.
757 * @param confParam is global configuration settings.
758 * @param htd the table descriptor
759 * @param rsServices reference to {@link RegionServerServices} or null
761 public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
762 final TableDescriptor htd, final RegionServerServices rsServices) {
763 if (htd == null) {
764 throw new IllegalArgumentException("Need table descriptor");
767 if (confParam instanceof CompoundConfiguration) {
768 throw new IllegalArgumentException("Need original base configuration");
771 this.wal = wal;
772 this.fs = fs;
773 this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog());
775 // 'conf' renamed to 'confParam' b/c we use this.conf in the constructor
776 this.baseConf = confParam;
777 this.conf = new CompoundConfiguration().add(confParam).addBytesMap(htd.getValues());
778 this.cellComparator = htd.isMetaTable() ||
779 conf.getBoolean(USE_META_CELL_COMPARATOR, DEFAULT_USE_META_CELL_COMPARATOR) ?
780 MetaCellComparator.META_COMPARATOR : CellComparatorImpl.COMPARATOR;
781 this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK,
782 DEFAULT_FAIR_REENTRANT_CLOSE_LOCK));
783 this.regionLockHolders = new ConcurrentHashMap<>();
784 this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
785 DEFAULT_CACHE_FLUSH_INTERVAL);
786 this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES);
787 if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) {
788 throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
789 + MAX_FLUSH_PER_CHANGES);
791 int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration",
792 DEFAULT_ROWLOCK_WAIT_DURATION);
793 if (tmpRowLockDuration <= 0) {
794 LOG.info("Found hbase.rowlock.wait.duration set to {}. values <= 0 will cause all row " +
795 "locking to fail. Treating it as 1ms to avoid region failure.", tmpRowLockDuration);
796 tmpRowLockDuration = 1;
798 this.rowLockWaitDuration = tmpRowLockDuration;
800 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
801 this.htableDescriptor = htd;
802 Set<byte[]> families = this.htableDescriptor.getColumnFamilyNames();
803 for (byte[] family : families) {
804 if (!replicationScope.containsKey(family)) {
805 int scope = htd.getColumnFamily(family).getScope();
806 // Only store those families that has NON-DEFAULT scope
807 if (scope != REPLICATION_SCOPE_LOCAL) {
808 // Do a copy before storing it here.
809 replicationScope.put(Bytes.copy(family), scope);
814 this.rsServices = rsServices;
815 if (this.rsServices != null) {
816 this.blockCache = rsServices.getBlockCache().orElse(null);
817 this.mobFileCache = rsServices.getMobFileCache().orElse(null);
819 this.regionServicesForStores = new RegionServicesForStores(this, rsServices);
821 setHTableSpecificConf();
822 this.scannerReadPoints = new ConcurrentHashMap<>();
824 this.busyWaitDuration = conf.getLong(
825 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
826 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
827 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
828 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
829 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
830 + maxBusyWaitMultiplier + "). Their product should be positive");
832 this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout",
833 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
836 * timestamp.slop provides a server-side constraint on the timestamp. This
837 * assumes that you base your TS around EnvironmentEdgeManager.currentTime(). In this case,
838 * throw an error to the user if the user-specified TS is newer than now +
839 * slop. LATEST_TIMESTAMP == don't use this functionality
841 this.timestampSlop = conf.getLong(
842 "hbase.hregion.keyvalue.timestamp.slop.millisecs",
843 HConstants.LATEST_TIMESTAMP);
845 this.storeHotnessProtector = new StoreHotnessProtector(this, conf);
847 boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
849 * This is the global default value for durability. All tables/mutations not defining a
850 * durability or using USE_DEFAULT will default to this value.
852 Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
853 this.regionDurability =
854 this.htableDescriptor.getDurability() == Durability.USE_DEFAULT ? defaultDurability :
855 this.htableDescriptor.getDurability();
857 decorateRegionConfiguration(conf);
858 if (rsServices != null) {
859 this.rsAccounting = this.rsServices.getRegionServerAccounting();
860 // don't initialize coprocessors if not running within a regionserver
861 // TODO: revisit if coprocessors should load in other cases
862 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
863 this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
864 this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper, conf);
865 } else {
866 this.metricsRegionWrapper = null;
867 this.metricsRegion = null;
869 if (LOG.isDebugEnabled()) {
870 // Write out region name, its encoded name and storeHotnessProtector as string.
871 LOG.debug("Instantiated " + this +"; "+ storeHotnessProtector.toString());
874 configurationManager = null;
876 // disable stats tracking system tables, but check the config for everything else
877 this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
878 NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ?
879 false :
880 conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
881 HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
883 this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
884 this.miniBatchSize = conf.getInt(HBASE_REGIONSERVER_MINIBATCH_SIZE,
885 DEFAULT_HBASE_REGIONSERVER_MINIBATCH_SIZE);
887 // recover the metrics of read and write requests count if they were retained
888 if (rsServices != null && rsServices.getRegionServerAccounting() != null) {
889 Pair<Long, Long> retainedRWRequestsCnt = rsServices.getRegionServerAccounting()
890 .getRetainedRegionRWRequestsCnt().get(getRegionInfo().getEncodedName());
891 if (retainedRWRequestsCnt != null) {
892 this.addReadRequestsCount(retainedRWRequestsCnt.getFirst());
893 this.addWriteRequestsCount(retainedRWRequestsCnt.getSecond());
894 // remove them since won't use again
895 rsServices.getRegionServerAccounting().getRetainedRegionRWRequestsCnt()
896 .remove(getRegionInfo().getEncodedName());
901 private void setHTableSpecificConf() {
902 if (this.htableDescriptor == null) {
903 return;
905 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
907 if (flushSize <= 0) {
908 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
909 TableDescriptorBuilder.DEFAULT_MEMSTORE_FLUSH_SIZE);
911 this.memstoreFlushSize = flushSize;
912 long mult = conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
913 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
914 this.blockingMemStoreSize = this.memstoreFlushSize * mult;
918 * Initialize this region.
919 * Used only by tests and SplitTransaction to reopen the region.
920 * You should use createHRegion() or openHRegion()
921 * @return What the next sequence (edit) id should be.
922 * @throws IOException e
923 * @deprecated use HRegion.createHRegion() or HRegion.openHRegion()
925 @Deprecated
926 public long initialize() throws IOException {
927 return initialize(null);
931 * Initialize this region.
933 * @param reporter Tickle every so often if initialize is taking a while.
934 * @return What the next sequence (edit) id should be.
935 * @throws IOException e
937 long initialize(final CancelableProgressable reporter) throws IOException {
939 //Refuse to open the region if there is no column family in the table
940 if (htableDescriptor.getColumnFamilyCount() == 0) {
941 throw new DoNotRetryIOException("Table " + htableDescriptor.getTableName().getNameAsString()+
942 " should have at least one column family.");
945 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
946 status.enableStatusJournal(true);
947 long nextSeqId = -1;
948 try {
949 nextSeqId = initializeRegionInternals(reporter, status);
950 return nextSeqId;
951 } catch (IOException e) {
952 LOG.warn("Failed initialize of region= {}, starting to roll back memstore",
953 getRegionInfo().getRegionNameAsString(), e);
954 // global memstore size will be decreased when dropping memstore
955 try {
956 //drop the memory used by memstore if open region fails
957 dropMemStoreContents();
958 } catch (IOException ioE) {
959 if (conf.getBoolean(MemStoreLAB.USEMSLAB_KEY, MemStoreLAB.USEMSLAB_DEFAULT)) {
960 LOG.warn("Failed drop memstore of region= {}, "
961 + "some chunks may not released forever since MSLAB is enabled",
962 getRegionInfo().getRegionNameAsString());
966 throw e;
967 } finally {
968 // nextSeqid will be -1 if the initialization fails.
969 // At least it will be 0 otherwise.
970 if (nextSeqId == -1) {
971 status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() +
972 " initialization.");
974 if (LOG.isDebugEnabled()) {
975 LOG.debug("Region open journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
976 status.prettyPrintJournal());
978 status.cleanup();
982 private long initializeRegionInternals(final CancelableProgressable reporter,
983 final MonitoredTask status) throws IOException {
984 if (coprocessorHost != null) {
985 status.setStatus("Running coprocessor pre-open hook");
986 coprocessorHost.preOpen();
989 // Write HRI to a file in case we need to recover hbase:meta
990 // Only the primary replica should write .regioninfo
991 if (this.getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
992 status.setStatus("Writing region info on filesystem");
993 fs.checkRegionInfoOnFilesystem();
996 // Initialize all the HStores
997 status.setStatus("Initializing all the Stores");
998 long maxSeqId = initializeStores(reporter, status);
999 this.mvcc.advanceTo(maxSeqId);
1000 if (!isRestoredRegion && ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
1001 Collection<HStore> stores = this.stores.values();
1002 try {
1003 // update the stores that we are replaying
1004 LOG.debug("replaying wal for " + this.getRegionInfo().getEncodedName());
1005 stores.forEach(HStore::startReplayingFromWAL);
1006 // Recover any edits if available.
1007 maxSeqId = Math.max(maxSeqId,
1008 replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status));
1009 // Recover any hfiles if available
1010 maxSeqId = Math.max(maxSeqId, loadRecoveredHFilesIfAny(stores));
1011 // Make sure mvcc is up to max.
1012 this.mvcc.advanceTo(maxSeqId);
1013 } finally {
1014 LOG.debug("stopping wal replay for " + this.getRegionInfo().getEncodedName());
1015 // update the stores that we are done replaying
1016 stores.forEach(HStore::stopReplayingFromWAL);
1019 this.lastReplayedOpenRegionSeqId = maxSeqId;
1021 this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
1022 this.writestate.flushRequested = false;
1023 this.writestate.compacting.set(0);
1025 if (this.writestate.writesEnabled) {
1026 LOG.debug("Cleaning up temporary data for " + this.getRegionInfo().getEncodedName());
1027 // Remove temporary data left over from old regions
1028 status.setStatus("Cleaning up temporary data from old regions");
1029 fs.cleanupTempDir();
1032 // Initialize split policy
1033 this.splitPolicy = RegionSplitPolicy.create(this, conf);
1035 // Initialize split restriction
1036 splitRestriction = RegionSplitRestriction.create(getTableDescriptor(), conf);
1038 // Initialize flush policy
1039 this.flushPolicy = FlushPolicyFactory.create(this, conf);
1041 long lastFlushTime = EnvironmentEdgeManager.currentTime();
1042 for (HStore store: stores.values()) {
1043 this.lastStoreFlushTimeMap.put(store, lastFlushTime);
1046 // Use maximum of log sequenceid or that which was found in stores
1047 // (particularly if no recovered edits, seqid will be -1).
1048 long nextSeqId = maxSeqId + 1;
1049 if (!isRestoredRegion) {
1050 // always get openSeqNum from the default replica, even if we are secondary replicas
1051 long maxSeqIdFromFile = WALSplitUtil.getMaxRegionSequenceId(conf,
1052 RegionReplicaUtil.getRegionInfoForDefaultReplica(getRegionInfo()), this::getFilesystem,
1053 this::getWalFileSystem);
1054 nextSeqId = Math.max(maxSeqId, maxSeqIdFromFile) + 1;
1055 // The openSeqNum will always be increase even for read only region, as we rely on it to
1056 // determine whether a region has been successfully reopened, so here we always need to update
1057 // the max sequence id file.
1058 if (RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1059 LOG.debug("writing seq id for {}", this.getRegionInfo().getEncodedName());
1060 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1061 nextSeqId - 1);
1062 // This means we have replayed all the recovered edits and also written out the max sequence
1063 // id file, let's delete the wrong directories introduced in HBASE-20734, see HBASE-22617
1064 // for more details.
1065 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf,
1066 getRegionInfo().getTable(), getRegionInfo().getEncodedName());
1067 FileSystem walFs = getWalFileSystem();
1068 if (walFs.exists(wrongRegionWALDir)) {
1069 if (!walFs.delete(wrongRegionWALDir, true)) {
1070 LOG.debug("Failed to clean up wrong region WAL directory {}", wrongRegionWALDir);
1076 LOG.info("Opened {}; next sequenceid={}; {}, {}", this.getRegionInfo().getShortNameToLog(),
1077 nextSeqId, this.splitPolicy, this.flushPolicy);
1079 // A region can be reopened if failed a split; reset flags
1080 this.closing.set(false);
1081 this.closed.set(false);
1083 if (coprocessorHost != null) {
1084 LOG.debug("Running coprocessor post-open hooks for " + this.getRegionInfo().getEncodedName());
1085 status.setStatus("Running coprocessor post-open hooks");
1086 coprocessorHost.postOpen();
1089 status.markComplete("Region opened successfully");
1090 return nextSeqId;
1094 * Open all Stores.
1095 * @param reporter
1096 * @param status
1097 * @return Highest sequenceId found out in a Store.
1098 * @throws IOException
1100 private long initializeStores(CancelableProgressable reporter, MonitoredTask status)
1101 throws IOException {
1102 return initializeStores(reporter, status, false);
1105 private long initializeStores(CancelableProgressable reporter, MonitoredTask status,
1106 boolean warmup) throws IOException {
1107 // Load in all the HStores.
1108 long maxSeqId = -1;
1109 // initialized to -1 so that we pick up MemstoreTS from column families
1110 long maxMemstoreTS = -1;
1112 if (htableDescriptor.getColumnFamilyCount() != 0) {
1113 // initialize the thread pool for opening stores in parallel.
1114 ThreadPoolExecutor storeOpenerThreadPool =
1115 getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
1116 CompletionService<HStore> completionService = new ExecutorCompletionService<>(storeOpenerThreadPool);
1118 // initialize each store in parallel
1119 for (final ColumnFamilyDescriptor family : htableDescriptor.getColumnFamilies()) {
1120 status.setStatus("Instantiating store for column family " + family);
1121 completionService.submit(new Callable<HStore>() {
1122 @Override
1123 public HStore call() throws IOException {
1124 return instantiateHStore(family, warmup);
1128 boolean allStoresOpened = false;
1129 boolean hasSloppyStores = false;
1130 try {
1131 for (int i = 0; i < htableDescriptor.getColumnFamilyCount(); i++) {
1132 Future<HStore> future = completionService.take();
1133 HStore store = future.get();
1134 this.stores.put(store.getColumnFamilyDescriptor().getName(), store);
1135 if (store.isSloppyMemStore()) {
1136 hasSloppyStores = true;
1139 long storeMaxSequenceId = store.getMaxSequenceId().orElse(0L);
1140 maxSeqIdInStores.put(Bytes.toBytes(store.getColumnFamilyName()),
1141 storeMaxSequenceId);
1142 if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
1143 maxSeqId = storeMaxSequenceId;
1145 long maxStoreMemstoreTS = store.getMaxMemStoreTS().orElse(0L);
1146 if (maxStoreMemstoreTS > maxMemstoreTS) {
1147 maxMemstoreTS = maxStoreMemstoreTS;
1150 allStoresOpened = true;
1151 if(hasSloppyStores) {
1152 htableDescriptor = TableDescriptorBuilder.newBuilder(htableDescriptor)
1153 .setFlushPolicyClassName(FlushNonSloppyStoresFirstPolicy.class.getName())
1154 .build();
1155 LOG.info("Setting FlushNonSloppyStoresFirstPolicy for the region=" + this);
1157 } catch (InterruptedException e) {
1158 throw throwOnInterrupt(e);
1159 } catch (ExecutionException e) {
1160 throw new IOException(e.getCause());
1161 } finally {
1162 storeOpenerThreadPool.shutdownNow();
1163 if (!allStoresOpened) {
1164 // something went wrong, close all opened stores
1165 LOG.error("Could not initialize all stores for the region=" + this);
1166 for (HStore store : this.stores.values()) {
1167 try {
1168 store.close();
1169 } catch (IOException e) {
1170 LOG.warn("close store {} failed in region {}", store.toString(), this, e);
1176 return Math.max(maxSeqId, maxMemstoreTS + 1);
1179 private void initializeWarmup(final CancelableProgressable reporter) throws IOException {
1180 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
1181 // Initialize all the HStores
1182 status.setStatus("Warmup all stores of " + this.getRegionInfo().getRegionNameAsString());
1183 try {
1184 initializeStores(reporter, status, true);
1185 } finally {
1186 status.markComplete("Warmed up " + this.getRegionInfo().getRegionNameAsString());
1191 * @return Map of StoreFiles by column family
1193 private NavigableMap<byte[], List<Path>> getStoreFiles() {
1194 NavigableMap<byte[], List<Path>> allStoreFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1195 for (HStore store : stores.values()) {
1196 Collection<HStoreFile> storeFiles = store.getStorefiles();
1197 if (storeFiles == null) {
1198 continue;
1200 List<Path> storeFileNames = new ArrayList<>();
1201 for (HStoreFile storeFile : storeFiles) {
1202 storeFileNames.add(storeFile.getPath());
1204 allStoreFiles.put(store.getColumnFamilyDescriptor().getName(), storeFileNames);
1206 return allStoreFiles;
1209 protected void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
1210 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1211 RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
1212 RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
1213 getRegionServerServices().getServerName(), storeFiles);
1214 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionOpenDesc,
1215 mvcc);
1218 private void writeRegionCloseMarker(WAL wal) throws IOException {
1219 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1220 RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
1221 RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
1222 getRegionServerServices().getServerName(), storeFiles);
1223 WALUtil.writeRegionEventMarker(wal, getReplicationScope(), getRegionInfo(), regionEventDesc,
1224 mvcc);
1226 // Store SeqId in WAL FileSystem when a region closes
1227 // checking region folder exists is due to many tests which delete the table folder while a
1228 // table is still online
1229 if (getWalFileSystem().exists(getWALRegionDir())) {
1230 WALSplitUtil.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1231 mvcc.getReadPoint());
1236 * @return True if this region has references.
1238 public boolean hasReferences() {
1239 return stores.values().stream().anyMatch(HStore::hasReferences);
1242 public void blockUpdates() {
1243 this.updatesLock.writeLock().lock();
1246 public void unblockUpdates() {
1247 this.updatesLock.writeLock().unlock();
1250 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
1251 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1252 stores.values().stream().filter(s -> s.getStorefiles() != null)
1253 .flatMap(s -> s.getStorefiles().stream()).map(HStoreFile::getHDFSBlockDistribution)
1254 .forEachOrdered(hdfsBlocksDistribution::add);
1255 return hdfsBlocksDistribution;
1259 * This is a helper function to compute HDFS block distribution on demand
1260 * @param conf configuration
1261 * @param tableDescriptor TableDescriptor of the table
1262 * @param regionInfo encoded name of the region
1263 * @return The HDFS blocks distribution for the given region.
1265 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1266 TableDescriptor tableDescriptor, RegionInfo regionInfo) throws IOException {
1267 Path tablePath =
1268 CommonFSUtils.getTableDir(CommonFSUtils.getRootDir(conf), tableDescriptor.getTableName());
1269 return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
1273 * This is a helper function to compute HDFS block distribution on demand
1274 * @param conf configuration
1275 * @param tableDescriptor TableDescriptor of the table
1276 * @param regionInfo encoded name of the region
1277 * @param tablePath the table directory
1278 * @return The HDFS blocks distribution for the given region.
1279 * @throws IOException
1281 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(Configuration conf,
1282 TableDescriptor tableDescriptor, RegionInfo regionInfo, Path tablePath) throws IOException {
1283 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1284 FileSystem fs = tablePath.getFileSystem(conf);
1286 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
1287 for (ColumnFamilyDescriptor family : tableDescriptor.getColumnFamilies()) {
1288 List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem
1289 .getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true);
1290 if (locatedFileStatusList == null) {
1291 continue;
1294 for (LocatedFileStatus status : locatedFileStatusList) {
1295 Path p = status.getPath();
1296 if (StoreFileInfo.isReference(p) || HFileLink.isHFileLink(p)) {
1297 // Only construct StoreFileInfo object if its not a hfile, save obj
1298 // creation
1299 StoreFileInfo storeFileInfo = new StoreFileInfo(conf, fs, status);
1300 hdfsBlocksDistribution.add(storeFileInfo
1301 .computeHDFSBlocksDistribution(fs));
1302 } else if (StoreFileInfo.isHFile(p)) {
1303 // If its a HFile, then lets just add to the block distribution
1304 // lets not create more objects here, not even another HDFSBlocksDistribution
1305 FSUtils.addToHDFSBlocksDistribution(hdfsBlocksDistribution,
1306 status.getBlockLocations());
1307 } else {
1308 throw new IOException("path=" + p
1309 + " doesn't look like a valid StoreFile");
1313 return hdfsBlocksDistribution;
1317 * Increase the size of mem store in this region and the size of global mem
1318 * store
1320 private void incMemStoreSize(MemStoreSize mss) {
1321 incMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1322 mss.getCellsCount());
1325 void incMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1326 int cellsCountDelta) {
1327 if (this.rsAccounting != null) {
1328 rsAccounting.incGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1330 long dataSize = this.memStoreSizing.incMemStoreSize(dataSizeDelta, heapSizeDelta,
1331 offHeapSizeDelta, cellsCountDelta);
1332 checkNegativeMemStoreDataSize(dataSize, dataSizeDelta);
1335 void decrMemStoreSize(MemStoreSize mss) {
1336 decrMemStoreSize(mss.getDataSize(), mss.getHeapSize(), mss.getOffHeapSize(),
1337 mss.getCellsCount());
1340 private void decrMemStoreSize(long dataSizeDelta, long heapSizeDelta, long offHeapSizeDelta,
1341 int cellsCountDelta) {
1342 if (this.rsAccounting != null) {
1343 rsAccounting.decGlobalMemStoreSize(dataSizeDelta, heapSizeDelta, offHeapSizeDelta);
1345 long dataSize = this.memStoreSizing.decMemStoreSize(dataSizeDelta, heapSizeDelta,
1346 offHeapSizeDelta, cellsCountDelta);
1347 checkNegativeMemStoreDataSize(dataSize, -dataSizeDelta);
1350 private void checkNegativeMemStoreDataSize(long memStoreDataSize, long delta) {
1351 // This is extremely bad if we make memStoreSizing negative. Log as much info on the offending
1352 // caller as possible. (memStoreSizing might be a negative value already -- freeing memory)
1353 if (memStoreDataSize < 0) {
1354 LOG.error("Asked to modify this region's (" + this.toString()
1355 + ") memStoreSizing to a negative value which is incorrect. Current memStoreSizing="
1356 + (memStoreDataSize - delta) + ", delta=" + delta, new Exception());
1360 @Override
1361 public RegionInfo getRegionInfo() {
1362 return this.fs.getRegionInfo();
1366 * @return Instance of {@link RegionServerServices} used by this HRegion.
1367 * Can be null.
1369 RegionServerServices getRegionServerServices() {
1370 return this.rsServices;
1373 @Override
1374 public long getReadRequestsCount() {
1375 return readRequestsCount.sum();
1378 @Override
1379 public long getCpRequestsCount() {
1380 return cpRequestsCount.sum();
1383 @Override
1384 public long getFilteredReadRequestsCount() {
1385 return filteredReadRequestsCount.sum();
1388 @Override
1389 public long getWriteRequestsCount() {
1390 return writeRequestsCount.sum();
1393 @Override
1394 public long getMemStoreDataSize() {
1395 return memStoreSizing.getDataSize();
1398 @Override
1399 public long getMemStoreHeapSize() {
1400 return memStoreSizing.getHeapSize();
1403 @Override
1404 public long getMemStoreOffHeapSize() {
1405 return memStoreSizing.getOffHeapSize();
1408 /** @return store services for this region, to access services required by store level needs */
1409 public RegionServicesForStores getRegionServicesForStores() {
1410 return regionServicesForStores;
1413 @Override
1414 public long getNumMutationsWithoutWAL() {
1415 return numMutationsWithoutWAL.sum();
1418 @Override
1419 public long getDataInMemoryWithoutWAL() {
1420 return dataInMemoryWithoutWAL.sum();
1423 @Override
1424 public long getBlockedRequestsCount() {
1425 return blockedRequestsCount.sum();
1428 @Override
1429 public long getCheckAndMutateChecksPassed() {
1430 return checkAndMutateChecksPassed.sum();
1433 @Override
1434 public long getCheckAndMutateChecksFailed() {
1435 return checkAndMutateChecksFailed.sum();
1438 // TODO Needs to check whether we should expose our metrics system to CPs. If CPs themselves doing
1439 // the op and bypassing the core, this might be needed? Should be stop supporting the bypass
1440 // feature?
1441 public MetricsRegion getMetrics() {
1442 return metricsRegion;
1445 @Override
1446 public boolean isClosed() {
1447 return this.closed.get();
1450 @Override
1451 public boolean isClosing() {
1452 return this.closing.get();
1455 @Override
1456 public boolean isReadOnly() {
1457 return this.writestate.isReadOnly();
1460 @Override
1461 public boolean isAvailable() {
1462 return !isClosed() && !isClosing();
1465 @Override
1466 public boolean isSplittable() {
1467 return splitPolicy.canSplit();
1470 @Override
1471 public boolean isMergeable() {
1472 if (!isAvailable()) {
1473 LOG.debug("Region " + this
1474 + " is not mergeable because it is closing or closed");
1475 return false;
1477 if (hasReferences()) {
1478 LOG.debug("Region " + this
1479 + " is not mergeable because it has references");
1480 return false;
1483 return true;
1486 public boolean areWritesEnabled() {
1487 synchronized(this.writestate) {
1488 return this.writestate.writesEnabled;
1492 public MultiVersionConcurrencyControl getMVCC() {
1493 return mvcc;
1496 @Override
1497 public long getMaxFlushedSeqId() {
1498 return maxFlushedSeqId;
1502 * @return readpoint considering given IsolationLevel. Pass {@code null} for default
1504 public long getReadPoint(IsolationLevel isolationLevel) {
1505 if (isolationLevel != null && isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
1506 // This scan can read even uncommitted transactions
1507 return Long.MAX_VALUE;
1509 return mvcc.getReadPoint();
1512 public boolean isLoadingCfsOnDemandDefault() {
1513 return this.isLoadingCfsOnDemandDefault;
1517 * Close down this HRegion. Flush the cache, shut down each HStore, don't
1518 * service any more calls.
1520 * <p>This method could take some time to execute, so don't call it from a
1521 * time-sensitive thread.
1523 * @return Vector of all the storage files that the HRegion's component
1524 * HStores make use of. It's a list of all StoreFile objects. Returns empty
1525 * vector if already closed and null if judged that it should not close.
1527 * @throws IOException e
1528 * @throws DroppedSnapshotException Thrown when replay of wal is required
1529 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1530 * caller MUST abort after this.
1532 public Map<byte[], List<HStoreFile>> close() throws IOException {
1533 return close(false);
1536 private final Object closeLock = new Object();
1538 /** Conf key for fair locking policy */
1539 public static final String FAIR_REENTRANT_CLOSE_LOCK =
1540 "hbase.regionserver.fair.region.close.lock";
1541 public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true;
1542 /** Conf key for the periodic flush interval */
1543 public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL =
1544 "hbase.regionserver.optionalcacheflushinterval";
1545 /** Default interval for the memstore flush */
1546 public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1547 /** Default interval for System tables memstore flush */
1548 public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000; // 5 minutes
1550 /** Conf key to force a flush if there are already enough changes for one region in memstore */
1551 public static final String MEMSTORE_FLUSH_PER_CHANGES =
1552 "hbase.regionserver.flush.per.changes";
1553 public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000; // 30 millions
1555 * The following MAX_FLUSH_PER_CHANGES is large enough because each KeyValue has 20+ bytes
1556 * overhead. Therefore, even 1G empty KVs occupy at least 20GB memstore size for a single region
1558 public static final long MAX_FLUSH_PER_CHANGES = 1000000000; // 1G
1560 public static final String CLOSE_WAIT_ABORT = "hbase.regionserver.close.wait.abort";
1561 public static final boolean DEFAULT_CLOSE_WAIT_ABORT = true;
1562 public static final String CLOSE_WAIT_TIME = "hbase.regionserver.close.wait.time.ms";
1563 public static final long DEFAULT_CLOSE_WAIT_TIME = 60000; // 1 minute
1564 public static final String CLOSE_WAIT_INTERVAL = "hbase.regionserver.close.wait.interval.ms";
1565 public static final long DEFAULT_CLOSE_WAIT_INTERVAL = 10000; // 10 seconds
1567 public Map<byte[], List<HStoreFile>> close(boolean abort) throws IOException {
1568 return close(abort, false);
1572 * Close down this HRegion. Flush the cache unless abort parameter is true,
1573 * Shut down each HStore, don't service any more calls.
1575 * This method could take some time to execute, so don't call it from a
1576 * time-sensitive thread.
1578 * @param abort true if server is aborting (only during testing)
1579 * @param ignoreStatus true if ignore the status (wont be showed on task list)
1580 * @return Vector of all the storage files that the HRegion's component
1581 * HStores make use of. It's a list of StoreFile objects. Can be null if
1582 * we are not to close at this time or we are already closed.
1584 * @throws IOException e
1585 * @throws DroppedSnapshotException Thrown when replay of wal is required
1586 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
1587 * caller MUST abort after this.
1589 public Map<byte[], List<HStoreFile>> close(boolean abort, boolean ignoreStatus)
1590 throws IOException {
1591 // Only allow one thread to close at a time. Serialize them so dual
1592 // threads attempting to close will run up against each other.
1593 MonitoredTask status = TaskMonitor.get().createStatus(
1594 "Closing region " + this.getRegionInfo().getEncodedName() +
1595 (abort ? " due to abort" : ""), ignoreStatus);
1596 status.enableStatusJournal(true);
1597 status.setStatus("Waiting for close lock");
1598 try {
1599 synchronized (closeLock) {
1600 return doClose(abort, status);
1602 } finally {
1603 if (LOG.isDebugEnabled()) {
1604 LOG.debug("Region close journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
1605 status.prettyPrintJournal());
1607 status.cleanup();
1612 * Exposed for some very specific unit tests.
1614 public void setClosing(boolean closing) {
1615 this.closing.set(closing);
1619 * The {@link HRegion#doClose} will block forever if someone tries proving the dead lock via the unit test.
1620 * Instead of blocking, the {@link HRegion#doClose} will throw exception if you set the timeout.
1621 * @param timeoutForWriteLock the second time to wait for the write lock in {@link HRegion#doClose}
1623 public void setTimeoutForWriteLock(long timeoutForWriteLock) {
1624 assert timeoutForWriteLock >= 0;
1625 this.timeoutForWriteLock = timeoutForWriteLock;
1628 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
1629 justification="I think FindBugs is confused")
1630 private Map<byte[], List<HStoreFile>> doClose(boolean abort, MonitoredTask status)
1631 throws IOException {
1632 if (isClosed()) {
1633 LOG.warn("Region " + this + " already closed");
1634 return null;
1637 if (coprocessorHost != null) {
1638 status.setStatus("Running coprocessor pre-close hooks");
1639 this.coprocessorHost.preClose(abort);
1641 status.setStatus("Disabling compacts and flushes for region");
1642 boolean canFlush = true;
1643 synchronized (writestate) {
1644 // Disable compacting and flushing by background threads for this
1645 // region.
1646 canFlush = !writestate.readOnly;
1647 writestate.writesEnabled = false;
1648 LOG.debug("Closing {}, disabling compactions & flushes",
1649 this.getRegionInfo().getEncodedName());
1650 waitForFlushesAndCompactions();
1652 // If we were not just flushing, is it worth doing a preflush...one
1653 // that will clear out of the bulk of the memstore before we put up
1654 // the close flag?
1655 if (!abort && worthPreFlushing() && canFlush) {
1656 status.setStatus("Pre-flushing region before close");
1657 LOG.info("Running close preflush of {}", this.getRegionInfo().getEncodedName());
1658 try {
1659 internalFlushcache(status);
1660 } catch (IOException ioe) {
1661 // Failed to flush the region. Keep going.
1662 status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1666 // Set the closing flag
1667 // From this point new arrivals at the region lock will get NSRE.
1669 this.closing.set(true);
1670 LOG.info("Closing region {}", this);
1672 // Acquire the close lock
1674 // The configuration parameter CLOSE_WAIT_ABORT is overloaded to enable both
1675 // the new regionserver abort condition and interrupts for running requests.
1676 // If CLOSE_WAIT_ABORT is not enabled there is no change from earlier behavior,
1677 // we will not attempt to interrupt threads servicing requests nor crash out
1678 // the regionserver if something remains stubborn.
1680 final boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT);
1681 boolean useTimedWait = false;
1682 if (timeoutForWriteLock != null && timeoutForWriteLock != Long.MAX_VALUE) {
1683 // convert legacy use of timeoutForWriteLock in seconds to new use in millis
1684 timeoutForWriteLock = TimeUnit.SECONDS.toMillis(timeoutForWriteLock);
1685 useTimedWait = true;
1686 } else if (canAbort) {
1687 timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME);
1688 useTimedWait = true;
1690 if (LOG.isDebugEnabled()) {
1691 LOG.debug((useTimedWait ? "Time limited wait" : "Waiting without time limit") +
1692 " for close lock on " + this);
1694 final long closeWaitInterval = conf.getLong(CLOSE_WAIT_INTERVAL, DEFAULT_CLOSE_WAIT_INTERVAL);
1695 long elapsedWaitTime = 0;
1696 if (useTimedWait) {
1697 // Sanity check configuration
1698 long remainingWaitTime = timeoutForWriteLock;
1699 if (remainingWaitTime < closeWaitInterval) {
1700 LOG.warn("Time limit for close wait of " + timeoutForWriteLock +
1701 " ms is less than the configured lock acquisition wait interval " +
1702 closeWaitInterval + " ms, using wait interval as time limit");
1703 remainingWaitTime = closeWaitInterval;
1705 boolean acquired = false;
1706 do {
1707 long start = EnvironmentEdgeManager.currentTime();
1708 try {
1709 acquired = lock.writeLock().tryLock(Math.min(remainingWaitTime, closeWaitInterval),
1710 TimeUnit.MILLISECONDS);
1711 } catch (InterruptedException e) {
1712 // Interrupted waiting for close lock. More likely the server is shutting down, not
1713 // normal operation, so aborting upon interrupt while waiting on this lock would not
1714 // provide much value. Throw an IOE (as IIOE) like we would in the case where we
1715 // fail to acquire the lock.
1716 String msg = "Interrupted while waiting for close lock on " + this;
1717 LOG.warn(msg, e);
1718 throw (InterruptedIOException) new InterruptedIOException(msg).initCause(e);
1720 long elapsed = EnvironmentEdgeManager.currentTime() - start;
1721 elapsedWaitTime += elapsed;
1722 remainingWaitTime -= elapsed;
1723 if (canAbort && !acquired && remainingWaitTime > 0) {
1724 // Before we loop to wait again, interrupt all region operations that might
1725 // still be in progress, to encourage them to break out of waiting states or
1726 // inner loops, throw an exception to clients, and release the read lock via
1727 // endRegionOperation.
1728 if (LOG.isDebugEnabled()) {
1729 LOG.debug("Interrupting region operations after waiting for close lock for " +
1730 elapsedWaitTime + " ms on " + this + ", " + remainingWaitTime +
1731 " ms remaining");
1733 interruptRegionOperations();
1735 } while (!acquired && remainingWaitTime > 0);
1737 // If we fail to acquire the lock, trigger an abort if we can; otherwise throw an IOE
1738 // to let the caller know we could not proceed with the close.
1739 if (!acquired) {
1740 String msg = "Failed to acquire close lock on " + this + " after waiting " +
1741 elapsedWaitTime + " ms";
1742 LOG.error(msg);
1743 if (canAbort) {
1744 // If we failed to acquire the write lock, abort the server
1745 rsServices.abort(msg, null);
1747 throw new IOException(msg);
1750 } else {
1752 long start = EnvironmentEdgeManager.currentTime();
1753 lock.writeLock().lock();
1754 elapsedWaitTime = EnvironmentEdgeManager.currentTime() - start;
1758 if (LOG.isDebugEnabled()) {
1759 LOG.debug("Acquired close lock on " + this + " after waiting " +
1760 elapsedWaitTime + " ms");
1763 status.setStatus("Disabling writes for close");
1764 try {
1765 if (this.isClosed()) {
1766 status.abort("Already got closed by another process");
1767 // SplitTransaction handles the null
1768 return null;
1770 LOG.debug("Updates disabled for region " + this);
1771 // Don't flush the cache if we are aborting
1772 if (!abort && canFlush) {
1773 int failedfFlushCount = 0;
1774 int flushCount = 0;
1775 long tmp = 0;
1776 long remainingSize = this.memStoreSizing.getDataSize();
1777 while (remainingSize > 0) {
1778 try {
1779 internalFlushcache(status);
1780 if(flushCount >0) {
1781 LOG.info("Running extra flush, " + flushCount +
1782 " (carrying snapshot?) " + this);
1784 flushCount++;
1785 tmp = this.memStoreSizing.getDataSize();
1786 if (tmp >= remainingSize) {
1787 failedfFlushCount++;
1789 remainingSize = tmp;
1790 if (failedfFlushCount > 5) {
1791 // If we failed 5 times and are unable to clear memory, abort
1792 // so we do not lose data
1793 throw new DroppedSnapshotException("Failed clearing memory after " +
1794 flushCount + " attempts on region: " +
1795 Bytes.toStringBinary(getRegionInfo().getRegionName()));
1797 } catch (IOException ioe) {
1798 status.setStatus("Failed flush " + this + ", putting online again");
1799 synchronized (writestate) {
1800 writestate.writesEnabled = true;
1802 // Have to throw to upper layers. I can't abort server from here.
1803 throw ioe;
1808 Map<byte[], List<HStoreFile>> result = new TreeMap<>(Bytes.BYTES_COMPARATOR);
1809 if (!stores.isEmpty()) {
1810 // initialize the thread pool for closing stores in parallel.
1811 ThreadPoolExecutor storeCloserThreadPool =
1812 getStoreOpenAndCloseThreadPool("StoreCloser-" +
1813 getRegionInfo().getRegionNameAsString());
1814 CompletionService<Pair<byte[], Collection<HStoreFile>>> completionService =
1815 new ExecutorCompletionService<>(storeCloserThreadPool);
1817 // close each store in parallel
1818 for (HStore store : stores.values()) {
1819 MemStoreSize mss = store.getFlushableSize();
1820 if (!(abort || mss.getDataSize() == 0 || writestate.readOnly)) {
1821 if (getRegionServerServices() != null) {
1822 getRegionServerServices().abort("Assertion failed while closing store "
1823 + getRegionInfo().getRegionNameAsString() + " " + store
1824 + ". flushableSize expected=0, actual={" + mss
1825 + "}. Current memStoreSize=" + this.memStoreSizing.getMemStoreSize() +
1826 ". Maybe a coprocessor "
1827 + "operation failed and left the memstore in a partially updated state.", null);
1830 completionService
1831 .submit(new Callable<Pair<byte[], Collection<HStoreFile>>>() {
1832 @Override
1833 public Pair<byte[], Collection<HStoreFile>> call() throws IOException {
1834 return new Pair<>(store.getColumnFamilyDescriptor().getName(), store.close());
1838 try {
1839 for (int i = 0; i < stores.size(); i++) {
1840 Future<Pair<byte[], Collection<HStoreFile>>> future = completionService.take();
1841 Pair<byte[], Collection<HStoreFile>> storeFiles = future.get();
1842 List<HStoreFile> familyFiles = result.get(storeFiles.getFirst());
1843 if (familyFiles == null) {
1844 familyFiles = new ArrayList<>();
1845 result.put(storeFiles.getFirst(), familyFiles);
1847 familyFiles.addAll(storeFiles.getSecond());
1849 } catch (InterruptedException e) {
1850 throw throwOnInterrupt(e);
1851 } catch (ExecutionException e) {
1852 Throwable cause = e.getCause();
1853 if (cause instanceof IOException) {
1854 throw (IOException) cause;
1856 throw new IOException(cause);
1857 } finally {
1858 storeCloserThreadPool.shutdownNow();
1862 status.setStatus("Writing region close event to WAL");
1863 // Always write close marker to wal even for read only table. This is not a big problem as we
1864 // do not write any data into the region; it is just a meta edit in the WAL file.
1865 if (!abort && wal != null && getRegionServerServices() != null &&
1866 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
1867 writeRegionCloseMarker(wal);
1870 this.closed.set(true);
1871 if (!canFlush) {
1872 decrMemStoreSize(this.memStoreSizing.getMemStoreSize());
1873 } else if (this.memStoreSizing.getDataSize() != 0) {
1874 LOG.error("Memstore data size is {} in region {}", this.memStoreSizing.getDataSize(), this);
1876 if (coprocessorHost != null) {
1877 status.setStatus("Running coprocessor post-close hooks");
1878 this.coprocessorHost.postClose(abort);
1880 if (this.metricsRegion != null) {
1881 this.metricsRegion.close();
1883 if (this.metricsRegionWrapper != null) {
1884 Closeables.close(this.metricsRegionWrapper, true);
1886 status.markComplete("Closed");
1887 LOG.info("Closed {}", this);
1888 return result;
1889 } finally {
1890 lock.writeLock().unlock();
1894 /** Wait for all current flushes and compactions of the region to complete */
1895 // TODO HBASE-18906. Check the usage (if any) in Phoenix and expose this or give alternate way for
1896 // Phoenix needs.
1897 public void waitForFlushesAndCompactions() {
1898 synchronized (writestate) {
1899 if (this.writestate.readOnly) {
1900 // we should not wait for replayed flushed if we are read only (for example in case the
1901 // region is a secondary replica).
1902 return;
1904 boolean interrupted = false;
1905 try {
1906 while (writestate.compacting.get() > 0 || writestate.flushing) {
1907 LOG.debug("waiting for " + writestate.compacting + " compactions"
1908 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1909 try {
1910 writestate.wait();
1911 } catch (InterruptedException iex) {
1912 // essentially ignore and propagate the interrupt back up
1913 LOG.warn("Interrupted while waiting in region {}", this);
1914 interrupted = true;
1915 break;
1918 } finally {
1919 if (interrupted) {
1920 Thread.currentThread().interrupt();
1927 * Wait for all current flushes of the region to complete
1929 public void waitForFlushes() {
1930 waitForFlushes(0);// Unbound wait
1933 @Override
1934 public boolean waitForFlushes(long timeout) {
1935 synchronized (writestate) {
1936 if (this.writestate.readOnly) {
1937 // we should not wait for replayed flushed if we are read only (for example in case the
1938 // region is a secondary replica).
1939 return true;
1941 if (!writestate.flushing) return true;
1942 long start = EnvironmentEdgeManager.currentTime();
1943 long duration = 0;
1944 boolean interrupted = false;
1945 LOG.debug("waiting for cache flush to complete for region " + this);
1946 try {
1947 while (writestate.flushing) {
1948 if (timeout > 0 && duration >= timeout) break;
1949 try {
1950 long toWait = timeout == 0 ? 0 : (timeout - duration);
1951 writestate.wait(toWait);
1952 } catch (InterruptedException iex) {
1953 // essentially ignore and propagate the interrupt back up
1954 LOG.warn("Interrupted while waiting in region {}", this);
1955 interrupted = true;
1956 break;
1957 } finally {
1958 duration = EnvironmentEdgeManager.currentTime() - start;
1961 } finally {
1962 if (interrupted) {
1963 Thread.currentThread().interrupt();
1966 LOG.debug("Waited {} ms for region {} flush to complete", duration, this);
1967 return !(writestate.flushing);
1971 @Override
1972 public Configuration getReadOnlyConfiguration() {
1973 return new ReadOnlyConfiguration(this.conf);
1976 private ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
1977 final String threadNamePrefix) {
1978 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
1979 int maxThreads = Math.min(numStores,
1980 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1981 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
1982 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1985 ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
1986 final String threadNamePrefix) {
1987 int numStores = Math.max(1, this.htableDescriptor.getColumnFamilyCount());
1988 int maxThreads = Math.max(1,
1989 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1990 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
1991 / numStores);
1992 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1995 private static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
1996 final String threadNamePrefix) {
1997 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
1998 new ThreadFactory() {
1999 private int count = 1;
2001 @Override
2002 public Thread newThread(Runnable r) {
2003 return new Thread(r, threadNamePrefix + "-" + count++);
2009 * @return True if its worth doing a flush before we put up the close flag.
2011 private boolean worthPreFlushing() {
2012 return this.memStoreSizing.getDataSize() >
2013 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
2016 //////////////////////////////////////////////////////////////////////////////
2017 // HRegion accessors
2018 //////////////////////////////////////////////////////////////////////////////
2020 @Override
2021 public TableDescriptor getTableDescriptor() {
2022 return this.htableDescriptor;
2025 public void setTableDescriptor(TableDescriptor desc) {
2026 htableDescriptor = desc;
2029 /** @return WAL in use for this region */
2030 public WAL getWAL() {
2031 return this.wal;
2034 public BlockCache getBlockCache() {
2035 return this.blockCache;
2039 * Only used for unit test which doesn't start region server.
2041 public void setBlockCache(BlockCache blockCache) {
2042 this.blockCache = blockCache;
2045 public MobFileCache getMobFileCache() {
2046 return this.mobFileCache;
2050 * Only used for unit test which doesn't start region server.
2052 public void setMobFileCache(MobFileCache mobFileCache) {
2053 this.mobFileCache = mobFileCache;
2057 * @return split policy for this region.
2059 RegionSplitPolicy getSplitPolicy() {
2060 return this.splitPolicy;
2064 * A split takes the config from the parent region & passes it to the daughter
2065 * region's constructor. If 'conf' was passed, you would end up using the HTD
2066 * of the parent region in addition to the new daughter HTD. Pass 'baseConf'
2067 * to the daughter regions to avoid this tricky dedupe problem.
2068 * @return Configuration object
2070 Configuration getBaseConf() {
2071 return this.baseConf;
2074 /** @return {@link FileSystem} being used by this region */
2075 public FileSystem getFilesystem() {
2076 return fs.getFileSystem();
2079 /** @return the {@link HRegionFileSystem} used by this region */
2080 public HRegionFileSystem getRegionFileSystem() {
2081 return this.fs;
2084 /** @return the WAL {@link HRegionFileSystem} used by this region */
2085 HRegionWALFileSystem getRegionWALFileSystem() throws IOException {
2086 return new HRegionWALFileSystem(conf, getWalFileSystem(),
2087 CommonFSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
2090 /** @return the WAL {@link FileSystem} being used by this region */
2091 FileSystem getWalFileSystem() throws IOException {
2092 if (walFS == null) {
2093 walFS = CommonFSUtils.getWALFileSystem(conf);
2095 return walFS;
2099 * @return the Region directory under WALRootDirectory
2100 * @throws IOException if there is an error getting WALRootDir
2102 public Path getWALRegionDir() throws IOException {
2103 if (regionDir == null) {
2104 regionDir = CommonFSUtils.getWALRegionDir(conf, getRegionInfo().getTable(),
2105 getRegionInfo().getEncodedName());
2107 return regionDir;
2110 @Override
2111 public long getEarliestFlushTimeForAllStores() {
2112 return Collections.min(lastStoreFlushTimeMap.values());
2115 @Override
2116 public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
2117 long result = Long.MAX_VALUE;
2118 for (HStore store : stores.values()) {
2119 Collection<HStoreFile> storeFiles = store.getStorefiles();
2120 if (storeFiles == null) {
2121 continue;
2123 for (HStoreFile file : storeFiles) {
2124 StoreFileReader sfReader = file.getReader();
2125 if (sfReader == null) {
2126 continue;
2128 HFile.Reader reader = sfReader.getHFileReader();
2129 if (reader == null) {
2130 continue;
2132 if (majorCompactionOnly) {
2133 byte[] val = reader.getHFileInfo().get(MAJOR_COMPACTION_KEY);
2134 if (val == null || !Bytes.toBoolean(val)) {
2135 continue;
2138 result = Math.min(result, reader.getFileContext().getFileCreateTime());
2141 return result == Long.MAX_VALUE ? 0 : result;
2144 RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) {
2145 long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId;
2146 byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes();
2147 regionLoadBldr.clearStoreCompleteSequenceId();
2148 for (byte[] familyName : this.stores.keySet()) {
2149 long earliest = this.wal.getEarliestMemStoreSeqNum(encodedRegionName, familyName);
2150 // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will
2151 // give us a sequence id that is for sure flushed. We want edit replay to start after this
2152 // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id.
2153 long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
2154 regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.newBuilder()
2155 .setFamilyName(UnsafeByteOperations.unsafeWrap(familyName)).setSequenceId(csid).build());
2157 return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
2160 //////////////////////////////////////////////////////////////////////////////
2161 // HRegion maintenance.
2163 // These methods are meant to be called periodically by the HRegionServer for
2164 // upkeep.
2165 //////////////////////////////////////////////////////////////////////////////
2167 * Do preparation for pending compaction.
2168 * @throws IOException
2170 protected void doRegionCompactionPrep() throws IOException {
2174 * Synchronously compact all stores in the region.
2175 * <p>This operation could block for a long time, so don't call it from a
2176 * time-sensitive thread.
2177 * <p>Note that no locks are taken to prevent possible conflicts between
2178 * compaction and splitting activities. The regionserver does not normally compact
2179 * and split in parallel. However by calling this method you may introduce
2180 * unexpected and unhandled concurrency. Don't do this unless you know what
2181 * you are doing.
2183 * @param majorCompaction True to force a major compaction regardless of thresholds
2184 * @throws IOException
2186 public void compact(boolean majorCompaction) throws IOException {
2187 if (majorCompaction) {
2188 stores.values().forEach(HStore::triggerMajorCompaction);
2190 for (HStore s : stores.values()) {
2191 Optional<CompactionContext> compaction = s.requestCompaction();
2192 if (compaction.isPresent()) {
2193 ThroughputController controller = null;
2194 if (rsServices != null) {
2195 controller = CompactionThroughputControllerFactory.create(rsServices, conf);
2197 if (controller == null) {
2198 controller = NoLimitThroughputController.INSTANCE;
2200 compact(compaction.get(), s, controller, null);
2206 * This is a helper function that compact all the stores synchronously.
2207 * <p>
2208 * It is used by utilities and testing
2210 public void compactStores() throws IOException {
2211 for (HStore s : stores.values()) {
2212 Optional<CompactionContext> compaction = s.requestCompaction();
2213 if (compaction.isPresent()) {
2214 compact(compaction.get(), s, NoLimitThroughputController.INSTANCE, null);
2220 * This is a helper function that compact the given store.
2221 * <p>
2222 * It is used by utilities and testing
2224 void compactStore(byte[] family, ThroughputController throughputController) throws IOException {
2225 HStore s = getStore(family);
2226 Optional<CompactionContext> compaction = s.requestCompaction();
2227 if (compaction.isPresent()) {
2228 compact(compaction.get(), s, throughputController, null);
2233 * Called by compaction thread and after region is opened to compact the
2234 * HStores if necessary.
2236 * <p>This operation could block for a long time, so don't call it from a
2237 * time-sensitive thread.
2239 * Note that no locking is necessary at this level because compaction only
2240 * conflicts with a region split, and that cannot happen because the region
2241 * server does them sequentially and not in parallel.
2243 * @param compaction Compaction details, obtained by requestCompaction()
2244 * @param throughputController
2245 * @return whether the compaction completed
2247 public boolean compact(CompactionContext compaction, HStore store,
2248 ThroughputController throughputController) throws IOException {
2249 return compact(compaction, store, throughputController, null);
2252 private boolean shouldForbidMajorCompaction() {
2253 if (rsServices != null && rsServices.getReplicationSourceService() != null) {
2254 return rsServices.getReplicationSourceService().getSyncReplicationPeerInfoProvider()
2255 .checkState(getRegionInfo().getTable(), ForbidMajorCompactionChecker.get());
2257 return false;
2261 * We are trying to remove / relax the region read lock for compaction.
2262 * Let's see what are the potential race conditions among the operations (user scan,
2263 * region split, region close and region bulk load).
2265 * user scan ---> region read lock
2266 * region split --> region close first --> region write lock
2267 * region close --> region write lock
2268 * region bulk load --> region write lock
2270 * read lock is compatible with read lock. ---> no problem with user scan/read
2271 * region bulk load does not cause problem for compaction (no consistency problem, store lock
2272 * will help the store file accounting).
2273 * They can run almost concurrently at the region level.
2275 * The only remaining race condition is between the region close and compaction.
2276 * So we will evaluate, below, how region close intervenes with compaction if compaction does
2277 * not acquire region read lock.
2279 * Here are the steps for compaction:
2280 * 1. obtain list of StoreFile's
2281 * 2. create StoreFileScanner's based on list from #1
2282 * 3. perform compaction and save resulting files under tmp dir
2283 * 4. swap in compacted files
2285 * #1 is guarded by store lock. This patch does not change this --> no worse or better
2286 * For #2, we obtain smallest read point (for region) across all the Scanners (for both default
2287 * compactor and stripe compactor).
2288 * The read points are for user scans. Region keeps the read points for all currently open
2289 * user scanners.
2290 * Compaction needs to know the smallest read point so that during re-write of the hfiles,
2291 * it can remove the mvcc points for the cells if their mvccs are older than the smallest
2292 * since they are not needed anymore.
2293 * This will not conflict with compaction.
2294 * For #3, it can be performed in parallel to other operations.
2295 * For #4 bulk load and compaction don't conflict with each other on the region level
2296 * (for multi-family atomicy).
2297 * Region close and compaction are guarded pretty well by the 'writestate'.
2298 * In HRegion#doClose(), we have :
2299 * synchronized (writestate) {
2300 * // Disable compacting and flushing by background threads for this
2301 * // region.
2302 * canFlush = !writestate.readOnly;
2303 * writestate.writesEnabled = false;
2304 * LOG.debug("Closing " + this + ": disabling compactions & flushes");
2305 * waitForFlushesAndCompactions();
2307 * waitForFlushesAndCompactions() would wait for writestate.compacting to come down to 0.
2308 * and in HRegion.compact()
2309 * try {
2310 * synchronized (writestate) {
2311 * if (writestate.writesEnabled) {
2312 * wasStateSet = true;
2313 * ++writestate.compacting;
2314 * } else {
2315 * String msg = "NOT compacting region " + this + ". Writes disabled.";
2316 * LOG.info(msg);
2317 * status.abort(msg);
2318 * return false;
2321 * Also in compactor.performCompaction():
2322 * check periodically to see if a system stop is requested
2323 * if (closeChecker != null && closeChecker.isTimeLimit(store, now)) {
2324 * progress.cancel();
2325 * return false;
2327 * if (closeChecker != null && closeChecker.isSizeLimit(store, len)) {
2328 * progress.cancel();
2329 * return false;
2332 public boolean compact(CompactionContext compaction, HStore store,
2333 ThroughputController throughputController, User user) throws IOException {
2334 assert compaction != null && compaction.hasSelection();
2335 assert !compaction.getRequest().getFiles().isEmpty();
2336 if (this.closing.get() || this.closed.get()) {
2337 LOG.debug("Skipping compaction on " + this + " because closing/closed");
2338 store.cancelRequestedCompaction(compaction);
2339 return false;
2342 if (compaction.getRequest().isAllFiles() && shouldForbidMajorCompaction()) {
2343 LOG.warn("Skipping major compaction on " + this
2344 + " because this cluster is transiting sync replication state"
2345 + " from STANDBY to DOWNGRADE_ACTIVE");
2346 store.cancelRequestedCompaction(compaction);
2347 return false;
2350 MonitoredTask status = null;
2351 boolean requestNeedsCancellation = true;
2352 try {
2353 byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
2354 if (stores.get(cf) != store) {
2355 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
2356 + " has been re-instantiated, cancel this compaction request. "
2357 + " It may be caused by the roll back of split transaction");
2358 return false;
2361 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
2362 status.enableStatusJournal(false);
2363 if (this.closed.get()) {
2364 String msg = "Skipping compaction on " + this + " because closed";
2365 LOG.debug(msg);
2366 status.abort(msg);
2367 return false;
2369 boolean wasStateSet = false;
2370 try {
2371 synchronized (writestate) {
2372 if (writestate.writesEnabled) {
2373 wasStateSet = true;
2374 writestate.compacting.incrementAndGet();
2375 } else {
2376 String msg = "NOT compacting region " + this + ". Writes disabled.";
2377 LOG.info(msg);
2378 status.abort(msg);
2379 return false;
2382 LOG.info("Starting compaction of {} in {}{}", store, this,
2383 (compaction.getRequest().isOffPeak()?" as an off-peak compaction":""));
2384 doRegionCompactionPrep();
2385 try {
2386 status.setStatus("Compacting store " + store);
2387 // We no longer need to cancel the request on the way out of this
2388 // method because Store#compact will clean up unconditionally
2389 requestNeedsCancellation = false;
2390 store.compact(compaction, throughputController, user);
2391 } catch (InterruptedIOException iioe) {
2392 String msg = "region " + this + " compaction interrupted";
2393 LOG.info(msg, iioe);
2394 status.abort(msg);
2395 return false;
2397 } finally {
2398 if (wasStateSet) {
2399 synchronized (writestate) {
2400 writestate.compacting.decrementAndGet();
2401 if (writestate.compacting.get() <= 0) {
2402 writestate.notifyAll();
2407 status.markComplete("Compaction complete");
2408 return true;
2409 } finally {
2410 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
2411 if (status != null) {
2412 LOG.debug("Compaction status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2413 status.prettyPrintJournal());
2414 status.cleanup();
2420 * Flush the cache.
2422 * <p>When this method is called the cache will be flushed unless:
2423 * <ol>
2424 * <li>the cache is empty</li>
2425 * <li>the region is closed.</li>
2426 * <li>a flush is already in progress</li>
2427 * <li>writes are disabled</li>
2428 * </ol>
2430 * <p>This method may block for some time, so it should not be called from a
2431 * time-sensitive thread.
2432 * @param flushAllStores whether we want to force a flush of all stores
2433 * @return FlushResult indicating whether the flush was successful or not and if
2434 * the region needs compacting
2436 * @throws IOException general io exceptions
2437 * because a snapshot was not properly persisted.
2439 // TODO HBASE-18905. We might have to expose a requestFlush API for CPs
2440 public FlushResult flush(boolean flushAllStores) throws IOException {
2441 return flushcache(flushAllStores, false, FlushLifeCycleTracker.DUMMY);
2444 public interface FlushResult {
2445 enum Result {
2446 FLUSHED_NO_COMPACTION_NEEDED,
2447 FLUSHED_COMPACTION_NEEDED,
2448 // Special case where a flush didn't run because there's nothing in the memstores. Used when
2449 // bulk loading to know when we can still load even if a flush didn't happen.
2450 CANNOT_FLUSH_MEMSTORE_EMPTY,
2451 CANNOT_FLUSH
2454 /** @return the detailed result code */
2455 Result getResult();
2457 /** @return true if the memstores were flushed, else false */
2458 boolean isFlushSucceeded();
2460 /** @return True if the flush requested a compaction, else false */
2461 boolean isCompactionNeeded();
2464 FlushResultImpl flushcache(boolean flushAllStores, boolean writeFlushRequestWalMarker,
2465 FlushLifeCycleTracker tracker) throws IOException {
2466 List<byte[]> families = null;
2467 if (flushAllStores) {
2468 families = new ArrayList<>();
2469 families.addAll(this.getTableDescriptor().getColumnFamilyNames());
2471 return this.flushcache(families, writeFlushRequestWalMarker, tracker);
2475 * Flush the cache.
2477 * When this method is called the cache will be flushed unless:
2478 * <ol>
2479 * <li>the cache is empty</li>
2480 * <li>the region is closed.</li>
2481 * <li>a flush is already in progress</li>
2482 * <li>writes are disabled</li>
2483 * </ol>
2485 * <p>This method may block for some time, so it should not be called from a
2486 * time-sensitive thread.
2487 * @param families stores of region to flush.
2488 * @param writeFlushRequestWalMarker whether to write the flush request marker to WAL
2489 * @param tracker used to track the life cycle of this flush
2490 * @return whether the flush is success and whether the region needs compacting
2492 * @throws IOException general io exceptions
2493 * @throws DroppedSnapshotException Thrown when replay of wal is required
2494 * because a Snapshot was not properly persisted. The region is put in closing mode, and the
2495 * caller MUST abort after this.
2497 public FlushResultImpl flushcache(List<byte[]> families,
2498 boolean writeFlushRequestWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2499 // fail-fast instead of waiting on the lock
2500 if (this.closing.get()) {
2501 String msg = "Skipping flush on " + this + " because closing";
2502 LOG.debug(msg);
2503 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2505 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
2506 status.enableStatusJournal(false);
2507 status.setStatus("Acquiring readlock on region");
2508 // block waiting for the lock for flushing cache
2509 lock.readLock().lock();
2510 boolean flushed = true;
2511 try {
2512 if (this.closed.get()) {
2513 String msg = "Skipping flush on " + this + " because closed";
2514 LOG.debug(msg);
2515 status.abort(msg);
2516 flushed = false;
2517 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2519 if (coprocessorHost != null) {
2520 status.setStatus("Running coprocessor pre-flush hooks");
2521 coprocessorHost.preFlush(tracker);
2523 // TODO: this should be managed within memstore with the snapshot, updated only after flush
2524 // successful
2525 if (numMutationsWithoutWAL.sum() > 0) {
2526 numMutationsWithoutWAL.reset();
2527 dataInMemoryWithoutWAL.reset();
2529 synchronized (writestate) {
2530 if (!writestate.flushing && writestate.writesEnabled) {
2531 this.writestate.flushing = true;
2532 } else {
2533 String msg = "NOT flushing " + this + " as " + (writestate.flushing ? "already flushing"
2534 : "writes are not enabled");
2535 LOG.debug(msg);
2536 status.abort(msg);
2537 flushed = false;
2538 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2542 try {
2543 // The reason that we do not always use flushPolicy is, when the flush is
2544 // caused by logRoller, we should select stores which must be flushed
2545 // rather than could be flushed.
2546 Collection<HStore> specificStoresToFlush = null;
2547 if (families != null) {
2548 specificStoresToFlush = getSpecificStores(families);
2549 } else {
2550 specificStoresToFlush = flushPolicy.selectStoresToFlush();
2552 FlushResultImpl fs =
2553 internalFlushcache(specificStoresToFlush, status, writeFlushRequestWalMarker, tracker);
2555 if (coprocessorHost != null) {
2556 status.setStatus("Running post-flush coprocessor hooks");
2557 coprocessorHost.postFlush(tracker);
2560 if(fs.isFlushSucceeded()) {
2561 flushesQueued.reset();
2564 status.markComplete("Flush successful " + fs.toString());
2565 return fs;
2566 } finally {
2567 synchronized (writestate) {
2568 writestate.flushing = false;
2569 this.writestate.flushRequested = false;
2570 writestate.notifyAll();
2573 } finally {
2574 lock.readLock().unlock();
2575 if (flushed) {
2576 // Don't log this journal stuff if no flush -- confusing.
2577 LOG.debug("Flush status journal for {}:\n{}", this.getRegionInfo().getEncodedName(),
2578 status.prettyPrintJournal());
2580 status.cleanup();
2585 * get stores which matches the specified families
2587 * @return the stores need to be flushed.
2589 private Collection<HStore> getSpecificStores(List<byte[]> families) {
2590 Collection<HStore> specificStoresToFlush = new ArrayList<>();
2591 for (byte[] family : families) {
2592 specificStoresToFlush.add(stores.get(family));
2594 return specificStoresToFlush;
2598 * Should the store be flushed because it is old enough.
2599 * <p>
2600 * Every FlushPolicy should call this to determine whether a store is old enough to flush (except
2601 * that you always flush all stores). Otherwise the method will always
2602 * returns true which will make a lot of flush requests.
2604 boolean shouldFlushStore(HStore store) {
2605 long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
2606 store.getColumnFamilyDescriptor().getName()) - 1;
2607 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
2608 if (LOG.isDebugEnabled()) {
2609 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
2610 getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest +
2611 " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
2613 return true;
2615 if (this.flushCheckInterval <= 0) {
2616 return false;
2618 long now = EnvironmentEdgeManager.currentTime();
2619 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2620 if (LOG.isDebugEnabled()) {
2621 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " +
2622 getRegionInfo().getEncodedName() + " because time of oldest edit=" +
2623 store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2625 return true;
2627 return false;
2631 * Should the memstore be flushed now
2633 boolean shouldFlush(final StringBuilder whyFlush) {
2634 whyFlush.setLength(0);
2635 // This is a rough measure.
2636 if (this.maxFlushedSeqId > 0
2637 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) {
2638 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2639 return true;
2641 long modifiedFlushCheckInterval = flushCheckInterval;
2642 if (getRegionInfo().getTable().isSystemTable() &&
2643 getRegionInfo().getReplicaId() == RegionInfo.DEFAULT_REPLICA_ID) {
2644 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2646 if (modifiedFlushCheckInterval <= 0) { //disabled
2647 return false;
2649 long now = EnvironmentEdgeManager.currentTime();
2650 //if we flushed in the recent past, we don't need to do again now
2651 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2652 return false;
2654 //since we didn't flush in the recent past, flush now if certain conditions
2655 //are met. Return true on first such memstore hit.
2656 for (HStore s : stores.values()) {
2657 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2658 // we have an old enough edit in the memstore, flush
2659 whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2660 return true;
2663 return false;
2667 * Flushing all stores.
2668 * @see #internalFlushcache(Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2670 private FlushResult internalFlushcache(MonitoredTask status) throws IOException {
2671 return internalFlushcache(stores.values(), status, false, FlushLifeCycleTracker.DUMMY);
2675 * Flushing given stores.
2676 * @see #internalFlushcache(WAL, long, Collection, MonitoredTask, boolean, FlushLifeCycleTracker)
2678 private FlushResultImpl internalFlushcache(Collection<HStore> storesToFlush, MonitoredTask status,
2679 boolean writeFlushWalMarker, FlushLifeCycleTracker tracker) throws IOException {
2680 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush, status,
2681 writeFlushWalMarker, tracker);
2685 * Flush the memstore. Flushing the memstore is a little tricky. We have a lot of updates in the
2686 * memstore, all of which have also been written to the wal. We need to write those updates in the
2687 * memstore out to disk, while being able to process reads/writes as much as possible during the
2688 * flush operation.
2689 * <p>
2690 * This method may block for some time. Every time you call it, we up the regions sequence id even
2691 * if we don't flush; i.e. the returned region id will be at least one larger than the last edit
2692 * applied to this region. The returned id does not refer to an actual edit. The returned id can
2693 * be used for say installing a bulk loaded file just ahead of the last hfile that was the result
2694 * of this flush, etc.
2695 * @param wal Null if we're NOT to go via wal.
2696 * @param myseqid The seqid to use if <code>wal</code> is null writing out flush file.
2697 * @param storesToFlush The list of stores to flush.
2698 * @return object describing the flush's state
2699 * @throws IOException general io exceptions
2700 * @throws DroppedSnapshotException Thrown when replay of WAL is required.
2702 protected FlushResultImpl internalFlushcache(WAL wal, long myseqid,
2703 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2704 FlushLifeCycleTracker tracker) throws IOException {
2705 PrepareFlushResult result =
2706 internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker, tracker);
2707 if (result.result == null) {
2708 return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2709 } else {
2710 return result.result; // early exit due to failure from prepare stage
2714 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
2715 justification="FindBugs seems confused about trxId")
2716 protected PrepareFlushResult internalPrepareFlushCache(WAL wal, long myseqid,
2717 Collection<HStore> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker,
2718 FlushLifeCycleTracker tracker) throws IOException {
2719 if (this.rsServices != null && this.rsServices.isAborted()) {
2720 // Don't flush when server aborting, it's unsafe
2721 throw new IOException("Aborting flush because server is aborted...");
2723 final long startTime = EnvironmentEdgeManager.currentTime();
2724 // If nothing to flush, return, but return with a valid unused sequenceId.
2725 // Its needed by bulk upload IIRC. It flushes until no edits in memory so it can insert a
2726 // bulk loaded file between memory and existing hfiles. It wants a good seqeunceId that belongs
2727 // to no other that it can use to associate with the bulk load. Hence this little dance below
2728 // to go get one.
2729 if (this.memStoreSizing.getDataSize() <= 0) {
2730 // Take an update lock so no edits can come into memory just yet.
2731 this.updatesLock.writeLock().lock();
2732 WriteEntry writeEntry = null;
2733 try {
2734 if (this.memStoreSizing.getDataSize() <= 0) {
2735 // Presume that if there are still no edits in the memstore, then there are no edits for
2736 // this region out in the WAL subsystem so no need to do any trickery clearing out
2737 // edits in the WAL sub-system. Up the sequence number so the resulting flush id is for
2738 // sure just beyond the last appended region edit and not associated with any edit
2739 // (useful as marker when bulk loading, etc.).
2740 if (wal != null) {
2741 writeEntry = mvcc.begin();
2742 long flushOpSeqId = writeEntry.getWriteNumber();
2743 FlushResultImpl flushResult =
2744 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, flushOpSeqId,
2745 "Nothing to flush", writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
2746 mvcc.completeAndWait(writeEntry);
2747 // Set to null so we don't complete it again down in finally block.
2748 writeEntry = null;
2749 return new PrepareFlushResult(flushResult, myseqid);
2750 } else {
2751 return new PrepareFlushResult(new FlushResultImpl(
2752 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY, "Nothing to flush", false), myseqid);
2755 } finally {
2756 if (writeEntry != null) {
2757 // If writeEntry is non-null, this operation failed; the mvcc transaction failed...
2758 // but complete it anyways so it doesn't block the mvcc queue.
2759 mvcc.complete(writeEntry);
2761 this.updatesLock.writeLock().unlock();
2764 logFatLineOnFlush(storesToFlush, myseqid);
2765 // Stop updates while we snapshot the memstore of all of these regions' stores. We only have
2766 // to do this for a moment. It is quick. We also set the memstore size to zero here before we
2767 // allow updates again so its value will represent the size of the updates received
2768 // during flush
2770 // We have to take an update lock during snapshot, or else a write could end up in both snapshot
2771 // and memstore (makes it difficult to do atomic rows then)
2772 status.setStatus("Obtaining lock to block concurrent updates");
2773 // block waiting for the lock for internal flush
2774 this.updatesLock.writeLock().lock();
2775 status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName());
2776 MemStoreSizing totalSizeOfFlushableStores = new NonThreadSafeMemStoreSizing();
2778 Map<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
2779 for (HStore store : storesToFlush) {
2780 flushedFamilyNamesToSeq.put(store.getColumnFamilyDescriptor().getName(),
2781 store.preFlushSeqIDEstimation());
2784 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2785 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2786 TreeMap<byte[], MemStoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR);
2787 // The sequence id of this flush operation which is used to log FlushMarker and pass to
2788 // createFlushContext to use as the store file's sequence id. It can be in advance of edits
2789 // still in the memstore, edits that are in other column families yet to be flushed.
2790 long flushOpSeqId = HConstants.NO_SEQNUM;
2791 // The max flushed sequence id after this flush operation completes. All edits in memstore
2792 // will be in advance of this sequence id.
2793 long flushedSeqId = HConstants.NO_SEQNUM;
2794 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2795 try {
2796 if (wal != null) {
2797 Long earliestUnflushedSequenceIdForTheRegion =
2798 wal.startCacheFlush(encodedRegionName, flushedFamilyNamesToSeq);
2799 if (earliestUnflushedSequenceIdForTheRegion == null) {
2800 // This should never happen. This is how startCacheFlush signals flush cannot proceed.
2801 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2802 status.setStatus(msg);
2803 return new PrepareFlushResult(
2804 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
2805 myseqid);
2807 flushOpSeqId = getNextSequenceId(wal);
2808 // Back up 1, minus 1 from oldest sequence id in memstore to get last 'flushed' edit
2809 flushedSeqId =
2810 earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
2811 flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2812 } else {
2813 // use the provided sequence Id as WAL is not being used for this flush.
2814 flushedSeqId = flushOpSeqId = myseqid;
2817 for (HStore s : storesToFlush) {
2818 storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(),
2819 s.createFlushContext(flushOpSeqId, tracker));
2820 // for writing stores to WAL
2821 committedFiles.put(s.getColumnFamilyDescriptor().getName(), null);
2824 // write the snapshot start to WAL
2825 if (wal != null && !writestate.readOnly) {
2826 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2827 getRegionInfo(), flushOpSeqId, committedFiles);
2828 // No sync. Sync is below where no updates lock and we do FlushAction.COMMIT_FLUSH
2829 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
2830 mvcc);
2833 // Prepare flush (take a snapshot)
2834 storeFlushCtxs.forEach((name, flush) -> {
2835 MemStoreSize snapshotSize = flush.prepare();
2836 totalSizeOfFlushableStores.incMemStoreSize(snapshotSize);
2837 storeFlushableSize.put(name, snapshotSize);
2839 } catch (IOException ex) {
2840 doAbortFlushToWAL(wal, flushOpSeqId, committedFiles);
2841 throw ex;
2842 } finally {
2843 this.updatesLock.writeLock().unlock();
2845 String s = "Finished memstore snapshotting " + this + ", syncing WAL and waiting on mvcc, " +
2846 "flushsize=" + totalSizeOfFlushableStores;
2847 status.setStatus(s);
2848 doSyncOfUnflushedWALChanges(wal, getRegionInfo());
2849 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2850 flushOpSeqId, flushedSeqId, totalSizeOfFlushableStores);
2854 * Utility method broken out of internalPrepareFlushCache so that method is smaller.
2856 private void logFatLineOnFlush(Collection<HStore> storesToFlush, long sequenceId) {
2857 if (!LOG.isInfoEnabled()) {
2858 return;
2860 // Log a fat line detailing what is being flushed.
2861 StringBuilder perCfExtras = null;
2862 if (!isAllFamilies(storesToFlush)) {
2863 perCfExtras = new StringBuilder();
2864 for (HStore store: storesToFlush) {
2865 MemStoreSize mss = store.getFlushableSize();
2866 perCfExtras.append("; ").append(store.getColumnFamilyName());
2867 perCfExtras.append("={dataSize=")
2868 .append(StringUtils.byteDesc(mss.getDataSize()));
2869 perCfExtras.append(", heapSize=")
2870 .append(StringUtils.byteDesc(mss.getHeapSize()));
2871 perCfExtras.append(", offHeapSize=")
2872 .append(StringUtils.byteDesc(mss.getOffHeapSize()));
2873 perCfExtras.append("}");
2876 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
2877 LOG.info("Flushing " + this.getRegionInfo().getEncodedName() + " " +
2878 storesToFlush.size() + "/" + stores.size() + " column families," +
2879 " dataSize=" + StringUtils.byteDesc(mss.getDataSize()) +
2880 " heapSize=" + StringUtils.byteDesc(mss.getHeapSize()) +
2881 ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
2882 ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + sequenceId));
2885 private void doAbortFlushToWAL(final WAL wal, final long flushOpSeqId,
2886 final Map<byte[], List<Path>> committedFiles) {
2887 if (wal == null) return;
2888 try {
2889 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2890 getRegionInfo(), flushOpSeqId, committedFiles);
2891 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, false,
2892 mvcc);
2893 } catch (Throwable t) {
2894 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL: {} in "
2895 + " region {}", StringUtils.stringifyException(t), this);
2896 // ignore this since we will be aborting the RS with DSE.
2898 // we have called wal.startCacheFlush(), now we have to abort it
2899 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2903 * Sync unflushed WAL changes. See HBASE-8208 for details
2905 private static void doSyncOfUnflushedWALChanges(final WAL wal, final RegionInfo hri)
2906 throws IOException {
2907 if (wal == null) {
2908 return;
2910 try {
2911 wal.sync(); // ensure that flush marker is sync'ed
2912 } catch (IOException ioe) {
2913 wal.abortCacheFlush(hri.getEncodedNameAsBytes());
2914 throw ioe;
2919 * @return True if passed Set is all families in the region.
2921 private boolean isAllFamilies(Collection<HStore> families) {
2922 return families == null || this.stores.size() == families.size();
2926 * Writes a marker to WAL indicating a flush is requested but cannot be complete due to various
2927 * reasons. Ignores exceptions from WAL. Returns whether the write succeeded.
2928 * @param wal
2929 * @return whether WAL write was successful
2931 private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) {
2932 if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
2933 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
2934 getRegionInfo(), -1, new TreeMap<>(Bytes.BYTES_COMPARATOR));
2935 try {
2936 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true,
2937 mvcc);
2938 return true;
2939 } catch (IOException e) {
2940 LOG.warn(getRegionInfo().getEncodedName() + " : "
2941 + "Received exception while trying to write the flush request to wal", e);
2944 return false;
2947 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
2948 justification="Intentional; notify is about completed flush")
2949 FlushResultImpl internalFlushCacheAndCommit(WAL wal, MonitoredTask status,
2950 PrepareFlushResult prepareResult, Collection<HStore> storesToFlush) throws IOException {
2951 // prepare flush context is carried via PrepareFlushResult
2952 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
2953 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
2954 long startTime = prepareResult.startTime;
2955 long flushOpSeqId = prepareResult.flushOpSeqId;
2956 long flushedSeqId = prepareResult.flushedSeqId;
2958 String s = "Flushing stores of " + this;
2959 status.setStatus(s);
2960 if (LOG.isTraceEnabled()) LOG.trace(s);
2962 // Any failure from here on out will be catastrophic requiring server
2963 // restart so wal content can be replayed and put back into the memstore.
2964 // Otherwise, the snapshot content while backed up in the wal, it will not
2965 // be part of the current running servers state.
2966 boolean compactionRequested = false;
2967 long flushedOutputFileSize = 0;
2968 try {
2969 // A. Flush memstore to all the HStores.
2970 // Keep running vector of all store files that includes both old and the
2971 // just-made new flush store file. The new flushed file is still in the
2972 // tmp directory.
2974 for (StoreFlushContext flush : storeFlushCtxs.values()) {
2975 flush.flushCache(status);
2978 // Switch snapshot (in memstore) -> new hfile (thus causing
2979 // all the store scanners to reset/reseek).
2980 for (Map.Entry<byte[], StoreFlushContext> flushEntry : storeFlushCtxs.entrySet()) {
2981 StoreFlushContext sfc = flushEntry.getValue();
2982 boolean needsCompaction = sfc.commit(status);
2983 if (needsCompaction) {
2984 compactionRequested = true;
2986 byte[] storeName = flushEntry.getKey();
2987 List<Path> storeCommittedFiles = sfc.getCommittedFiles();
2988 committedFiles.put(storeName, storeCommittedFiles);
2989 // Flush committed no files, indicating flush is empty or flush was canceled
2990 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
2991 MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName);
2992 prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize);
2994 flushedOutputFileSize += sfc.getOutputFileSize();
2996 storeFlushCtxs.clear();
2998 // Set down the memstore size by amount of flush.
2999 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3000 this.decrMemStoreSize(mss);
3002 // Increase the size of this Region for the purposes of quota. Noop if quotas are disabled.
3003 // During startup, quota manager may not be initialized yet.
3004 if (rsServices != null) {
3005 RegionServerSpaceQuotaManager quotaManager = rsServices.getRegionServerSpaceQuotaManager();
3006 if (quotaManager != null) {
3007 quotaManager.getRegionSizeStore().incrementRegionSize(
3008 this.getRegionInfo(), flushedOutputFileSize);
3012 if (wal != null) {
3013 // write flush marker to WAL. If fail, we should throw DroppedSnapshotException
3014 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
3015 getRegionInfo(), flushOpSeqId, committedFiles);
3016 WALUtil.writeFlushMarker(wal, this.getReplicationScope(), getRegionInfo(), desc, true,
3017 mvcc);
3019 } catch (Throwable t) {
3020 // An exception here means that the snapshot was not persisted.
3021 // The wal needs to be replayed so its content is restored to memstore.
3022 // Currently, only a server restart will do this.
3023 // We used to only catch IOEs but its possible that we'd get other
3024 // exceptions -- e.g. HBASE-659 was about an NPE -- so now we catch
3025 // all and sundry.
3026 if (wal != null) {
3027 try {
3028 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
3029 getRegionInfo(), flushOpSeqId, committedFiles);
3030 WALUtil.writeFlushMarker(wal, this.replicationScope, getRegionInfo(), desc, false, mvcc);
3031 } catch (Throwable ex) {
3032 LOG.warn(getRegionInfo().getEncodedName() + " : "
3033 + "failed writing ABORT_FLUSH marker to WAL", ex);
3034 // ignore this since we will be aborting the RS with DSE.
3036 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
3038 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
3039 Bytes.toStringBinary(getRegionInfo().getRegionName()), t);
3040 status.abort("Flush failed: " + StringUtils.stringifyException(t));
3042 // Callers for flushcache() should catch DroppedSnapshotException and abort the region server.
3043 // However, since we may have the region read lock, we cannot call close(true) here since
3044 // we cannot promote to a write lock. Instead we are setting closing so that all other region
3045 // operations except for close will be rejected.
3046 this.closing.set(true);
3048 if (rsServices != null) {
3049 // This is a safeguard against the case where the caller fails to explicitly handle aborting
3050 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
3053 throw dse;
3056 // If we get to here, the HStores have been written.
3057 if (wal != null) {
3058 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes(), flushedSeqId);
3061 // Record latest flush time
3062 for (HStore store: storesToFlush) {
3063 this.lastStoreFlushTimeMap.put(store, startTime);
3066 this.maxFlushedSeqId = flushedSeqId;
3067 this.lastFlushOpSeqId = flushOpSeqId;
3069 // C. Finally notify anyone waiting on memstore to clear:
3070 // e.g. checkResources().
3071 synchronized (this) {
3072 notifyAll(); // FindBugs NN_NAKED_NOTIFY
3075 long time = EnvironmentEdgeManager.currentTime() - startTime;
3076 MemStoreSize mss = prepareResult.totalFlushableSize.getMemStoreSize();
3077 long memstoresize = this.memStoreSizing.getMemStoreSize().getDataSize();
3078 String msg = "Finished flush of"
3079 + " dataSize ~" + StringUtils.byteDesc(mss.getDataSize()) + "/" + mss.getDataSize()
3080 + ", heapSize ~" + StringUtils.byteDesc(mss.getHeapSize()) + "/" + mss.getHeapSize()
3081 + ", currentSize=" + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
3082 + " for " + this.getRegionInfo().getEncodedName() + " in " + time + "ms, sequenceid="
3083 + flushOpSeqId + ", compaction requested=" + compactionRequested
3084 + ((wal == null) ? "; wal=null" : "");
3085 LOG.info(msg);
3086 status.setStatus(msg);
3088 if (rsServices != null && rsServices.getMetrics() != null) {
3089 rsServices.getMetrics().updateFlush(getTableDescriptor().getTableName().getNameAsString(),
3090 time,
3091 mss.getDataSize(), flushedOutputFileSize);
3094 return new FlushResultImpl(compactionRequested ?
3095 FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
3096 FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
3100 * Method to safely get the next sequence number.
3101 * @return Next sequence number unassociated with any actual edit.
3102 * @throws IOException
3104 protected long getNextSequenceId(final WAL wal) throws IOException {
3105 WriteEntry we = mvcc.begin();
3106 mvcc.completeAndWait(we);
3107 return we.getWriteNumber();
3110 //////////////////////////////////////////////////////////////////////////////
3111 // get() methods for client use.
3112 //////////////////////////////////////////////////////////////////////////////
3114 @Override
3115 public RegionScannerImpl getScanner(Scan scan) throws IOException {
3116 return getScanner(scan, null);
3119 @Override
3120 public RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
3121 throws IOException {
3122 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
3125 private RegionScannerImpl getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
3126 long nonceGroup, long nonce) throws IOException {
3127 return TraceUtil.trace(() -> {
3128 startRegionOperation(Operation.SCAN);
3129 try {
3130 // Verify families are all valid
3131 if (!scan.hasFamilies()) {
3132 // Adding all families to scanner
3133 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
3134 scan.addFamily(family);
3136 } else {
3137 for (byte[] family : scan.getFamilyMap().keySet()) {
3138 checkFamily(family);
3141 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
3142 } finally {
3143 closeRegionOperation(Operation.SCAN);
3145 }, () -> createRegionSpan("Region.getScanner"));
3148 protected RegionScannerImpl instantiateRegionScanner(Scan scan,
3149 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
3150 if (scan.isReversed()) {
3151 if (scan.getFilter() != null) {
3152 scan.getFilter().setReversed(true);
3154 return new ReversedRegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3156 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
3160 * Prepare a delete for a row mutation processor
3161 * @param delete The passed delete is modified by this method. WARNING!
3163 private void prepareDelete(Delete delete) throws IOException {
3164 // Check to see if this is a deleteRow insert
3165 if(delete.getFamilyCellMap().isEmpty()){
3166 for(byte [] family : this.htableDescriptor.getColumnFamilyNames()){
3167 // Don't eat the timestamp
3168 delete.addFamily(family, delete.getTimestamp());
3170 } else {
3171 for(byte [] family : delete.getFamilyCellMap().keySet()) {
3172 if(family == null) {
3173 throw new NoSuchColumnFamilyException("Empty family is invalid");
3175 checkFamily(family, delete.getDurability());
3180 @Override
3181 public void delete(Delete delete) throws IOException {
3182 TraceUtil.trace(() -> {
3183 checkReadOnly();
3184 checkResources();
3185 startRegionOperation(Operation.DELETE);
3186 try {
3187 // All edits for the given row (across all column families) must happen atomically.
3188 return mutate(delete);
3189 } finally {
3190 closeRegionOperation(Operation.DELETE);
3192 }, () -> createRegionSpan("Region.delete"));
3196 * Set up correct timestamps in the KVs in Delete object.
3197 * <p/>
3198 * Caller should have the row and region locks.
3200 private void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
3201 byte[] byteNow) throws IOException {
3202 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3204 byte[] family = e.getKey();
3205 List<Cell> cells = e.getValue();
3206 assert cells instanceof RandomAccess;
3208 Map<byte[], Integer> kvCount = new TreeMap<>(Bytes.BYTES_COMPARATOR);
3209 int listSize = cells.size();
3210 for (int i=0; i < listSize; i++) {
3211 Cell cell = cells.get(i);
3212 // Check if time is LATEST, change to time of most recent addition if so
3213 // This is expensive.
3214 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP
3215 && PrivateCellUtil.isDeleteType(cell)) {
3216 byte[] qual = CellUtil.cloneQualifier(cell);
3218 Integer count = kvCount.get(qual);
3219 if (count == null) {
3220 kvCount.put(qual, 1);
3221 } else {
3222 kvCount.put(qual, count + 1);
3224 count = kvCount.get(qual);
3226 Get get = new Get(CellUtil.cloneRow(cell));
3227 get.readVersions(count);
3228 get.addColumn(family, qual);
3229 if (coprocessorHost != null) {
3230 if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
3231 byteNow, get)) {
3232 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3234 } else {
3235 updateDeleteLatestVersionTimestamp(cell, get, count, byteNow);
3237 } else {
3238 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3244 private void updateDeleteLatestVersionTimestamp(Cell cell, Get get, int count, byte[] byteNow)
3245 throws IOException {
3246 try (RegionScanner scanner = getScanner(new Scan(get))) {
3247 // NOTE: Please don't use HRegion.get() instead,
3248 // because it will copy cells to heap. See HBASE-26036
3249 List<Cell> result = new ArrayList<>();
3250 scanner.next(result);
3252 if (result.size() < count) {
3253 // Nothing to delete
3254 PrivateCellUtil.updateLatestStamp(cell, byteNow);
3255 return;
3257 if (result.size() > count) {
3258 throw new RuntimeException("Unexpected size: " + result.size());
3260 Cell getCell = result.get(count - 1);
3261 PrivateCellUtil.setTimestamp(cell, getCell.getTimestamp());
3265 @Override
3266 public void put(Put put) throws IOException {
3267 TraceUtil.trace(() -> {
3268 checkReadOnly();
3270 // Do a rough check that we have resources to accept a write. The check is
3271 // 'rough' in that between the resource check and the call to obtain a
3272 // read lock, resources may run out. For now, the thought is that this
3273 // will be extremely rare; we'll deal with it when it happens.
3274 checkResources();
3275 startRegionOperation(Operation.PUT);
3276 try {
3277 // All edits for the given row (across all column families) must happen atomically.
3278 return mutate(put);
3279 } finally {
3280 closeRegionOperation(Operation.PUT);
3282 }, () -> createRegionSpan("Region.put"));
3286 * Class that tracks the progress of a batch operations, accumulating status codes and tracking
3287 * the index at which processing is proceeding. These batch operations may get split into
3288 * mini-batches for processing.
3290 private abstract static class BatchOperation<T> {
3291 protected final T[] operations;
3292 protected final OperationStatus[] retCodeDetails;
3293 protected final WALEdit[] walEditsFromCoprocessors;
3294 // reference family cell maps directly so coprocessors can mutate them if desired
3295 protected final Map<byte[], List<Cell>>[] familyCellMaps;
3296 // For Increment/Append operations
3297 protected final Result[] results;
3299 protected final HRegion region;
3300 protected int nextIndexToProcess = 0;
3301 protected final ObservedExceptionsInBatch observedExceptions;
3302 //Durability of the batch (highest durability of all operations)
3303 protected Durability durability;
3304 protected boolean atomic = false;
3306 public BatchOperation(final HRegion region, T[] operations) {
3307 this.operations = operations;
3308 this.retCodeDetails = new OperationStatus[operations.length];
3309 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
3310 this.walEditsFromCoprocessors = new WALEdit[operations.length];
3311 familyCellMaps = new Map[operations.length];
3312 this.results = new Result[operations.length];
3314 this.region = region;
3315 observedExceptions = new ObservedExceptionsInBatch();
3316 durability = Durability.USE_DEFAULT;
3320 * Visitor interface for batch operations
3322 @FunctionalInterface
3323 interface Visitor {
3325 * @param index operation index
3326 * @return If true continue visiting remaining entries, break otherwise
3328 boolean visit(int index) throws IOException;
3332 * Helper method for visiting pending/ all batch operations
3334 public void visitBatchOperations(boolean pendingOnly, int lastIndexExclusive, Visitor visitor)
3335 throws IOException {
3336 assert lastIndexExclusive <= this.size();
3337 for (int i = nextIndexToProcess; i < lastIndexExclusive; i++) {
3338 if (!pendingOnly || isOperationPending(i)) {
3339 if (!visitor.visit(i)) {
3340 break;
3346 public abstract Mutation getMutation(int index);
3348 public abstract long getNonceGroup(int index);
3350 public abstract long getNonce(int index);
3353 * This method is potentially expensive and useful mostly for non-replay CP path.
3355 public abstract Mutation[] getMutationsForCoprocs();
3357 public abstract boolean isInReplay();
3359 public abstract long getOrigLogSeqNum();
3361 public abstract void startRegionOperation() throws IOException;
3363 public abstract void closeRegionOperation() throws IOException;
3366 * Validates each mutation and prepares a batch for write. If necessary (non-replay case), runs
3367 * CP prePut()/preDelete()/preIncrement()/preAppend() hooks for all mutations in a batch. This
3368 * is intended to operate on entire batch and will be called from outside of class to check
3369 * and prepare batch. This can be implemented by calling helper method
3370 * {@link #checkAndPrepareMutation(int, long)} in a 'for' loop over mutations.
3372 public abstract void checkAndPrepare() throws IOException;
3375 * Implement any Put request specific check and prepare logic here. Please refer to
3376 * {@link #checkAndPrepareMutation(Mutation, long)} for how its used.
3378 protected abstract void checkAndPreparePut(final Put p) throws IOException;
3381 * If necessary, calls preBatchMutate() CP hook for a mini-batch and updates metrics, cell
3382 * count, tags and timestamp for all cells of all operations in a mini-batch.
3384 public abstract void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation>
3385 miniBatchOp, long timestamp, final List<RowLock> acquiredRowLocks) throws IOException;
3388 * Write mini-batch operations to MemStore
3390 public abstract WriteEntry writeMiniBatchOperationsToMemStore(
3391 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3392 throws IOException;
3394 protected void writeMiniBatchOperationsToMemStore(
3395 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final long writeNumber)
3396 throws IOException {
3397 MemStoreSizing memStoreAccounting = new NonThreadSafeMemStoreSizing();
3398 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3399 // We need to update the sequence id for following reasons.
3400 // 1) If the op is in replay mode, FSWALEntry#stampRegionSequenceId won't stamp sequence id.
3401 // 2) If no WAL, FSWALEntry won't be used
3402 // we use durability of the original mutation for the mutation passed by CP.
3403 if (isInReplay() || getMutation(index).getDurability() == Durability.SKIP_WAL) {
3404 region.updateSequenceId(familyCellMaps[index].values(), writeNumber);
3406 applyFamilyMapToMemStore(familyCellMaps[index], memStoreAccounting);
3407 return true;
3409 // update memStore size
3410 region.incMemStoreSize(memStoreAccounting.getDataSize(), memStoreAccounting.getHeapSize(),
3411 memStoreAccounting.getOffHeapSize(), memStoreAccounting.getCellsCount());
3414 public boolean isDone() {
3415 return nextIndexToProcess == operations.length;
3418 public int size() {
3419 return operations.length;
3422 public boolean isOperationPending(int index) {
3423 return retCodeDetails[index].getOperationStatusCode() == OperationStatusCode.NOT_RUN;
3426 public List<UUID> getClusterIds() {
3427 assert size() != 0;
3428 return getMutation(0).getClusterIds();
3431 boolean isAtomic() {
3432 return atomic;
3436 * Helper method that checks and prepares only one mutation. This can be used to implement
3437 * {@link #checkAndPrepare()} for entire Batch.
3438 * NOTE: As CP prePut()/preDelete()/preIncrement()/preAppend() hooks may modify mutations,
3439 * this method should be called after prePut()/preDelete()/preIncrement()/preAppend() CP hooks
3440 * are run for the mutation
3442 protected void checkAndPrepareMutation(Mutation mutation, final long timestamp)
3443 throws IOException {
3444 region.checkRow(mutation.getRow(), "batchMutate");
3445 if (mutation instanceof Put) {
3446 // Check the families in the put. If bad, skip this one.
3447 checkAndPreparePut((Put) mutation);
3448 region.checkTimestamps(mutation.getFamilyCellMap(), timestamp);
3449 } else if (mutation instanceof Delete) {
3450 region.prepareDelete((Delete) mutation);
3451 } else if (mutation instanceof Increment || mutation instanceof Append) {
3452 region.checkFamilies(mutation.getFamilyCellMap().keySet(), mutation.getDurability());
3456 protected void checkAndPrepareMutation(int index, long timestamp) throws IOException {
3457 Mutation mutation = getMutation(index);
3458 try {
3459 this.checkAndPrepareMutation(mutation, timestamp);
3461 if (mutation instanceof Put || mutation instanceof Delete) {
3462 // store the family map reference to allow for mutations
3463 familyCellMaps[index] = mutation.getFamilyCellMap();
3466 // store durability for the batch (highest durability of all operations in the batch)
3467 Durability tmpDur = region.getEffectiveDurability(mutation.getDurability());
3468 if (tmpDur.ordinal() > durability.ordinal()) {
3469 durability = tmpDur;
3471 } catch (NoSuchColumnFamilyException nscfe) {
3472 final String msg = "No such column family in batch mutation in region " + this;
3473 if (observedExceptions.hasSeenNoSuchFamily()) {
3474 LOG.warn(msg + nscfe.getMessage());
3475 } else {
3476 LOG.warn(msg, nscfe);
3477 observedExceptions.sawNoSuchFamily();
3479 retCodeDetails[index] = new OperationStatus(
3480 OperationStatusCode.BAD_FAMILY, nscfe.getMessage());
3481 if (isAtomic()) { // fail, atomic means all or none
3482 throw nscfe;
3484 } catch (FailedSanityCheckException fsce) {
3485 final String msg = "Batch Mutation did not pass sanity check in region " + this;
3486 if (observedExceptions.hasSeenFailedSanityCheck()) {
3487 LOG.warn(msg + fsce.getMessage());
3488 } else {
3489 LOG.warn(msg, fsce);
3490 observedExceptions.sawFailedSanityCheck();
3492 retCodeDetails[index] = new OperationStatus(
3493 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3494 if (isAtomic()) {
3495 throw fsce;
3497 } catch (WrongRegionException we) {
3498 final String msg = "Batch mutation had a row that does not belong to this region " + this;
3499 if (observedExceptions.hasSeenWrongRegion()) {
3500 LOG.warn(msg + we.getMessage());
3501 } else {
3502 LOG.warn(msg, we);
3503 observedExceptions.sawWrongRegion();
3505 retCodeDetails[index] = new OperationStatus(
3506 OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3507 if (isAtomic()) {
3508 throw we;
3514 * Creates Mini-batch of all operations [nextIndexToProcess, lastIndexExclusive) for which
3515 * a row lock can be acquired. All mutations with locked rows are considered to be
3516 * In-progress operations and hence the name {@link MiniBatchOperationInProgress}. Mini batch
3517 * is window over {@link BatchOperation} and contains contiguous pending operations.
3519 * @param acquiredRowLocks keeps track of rowLocks acquired.
3521 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
3522 List<RowLock> acquiredRowLocks) throws IOException {
3523 int readyToWriteCount = 0;
3524 int lastIndexExclusive = 0;
3525 RowLock prevRowLock = null;
3526 for (; lastIndexExclusive < size(); lastIndexExclusive++) {
3527 // It reaches the miniBatchSize, stop here and process the miniBatch
3528 // This only applies to non-atomic batch operations.
3529 if (!isAtomic() && (readyToWriteCount == region.miniBatchSize)) {
3530 break;
3533 if (!isOperationPending(lastIndexExclusive)) {
3534 continue;
3537 // HBASE-19389 Limit concurrency of put with dense (hundreds) columns to avoid exhausting
3538 // RS handlers, covering both MutationBatchOperation and ReplayBatchOperation
3539 // The BAD_FAMILY/SANITY_CHECK_FAILURE cases are handled in checkAndPrepare phase and won't
3540 // pass the isOperationPending check
3541 Map<byte[], List<Cell>> curFamilyCellMap =
3542 getMutation(lastIndexExclusive).getFamilyCellMap();
3543 try {
3544 // start the protector before acquiring row lock considering performance, and will finish
3545 // it when encountering exception
3546 region.storeHotnessProtector.start(curFamilyCellMap);
3547 } catch (RegionTooBusyException rtbe) {
3548 region.storeHotnessProtector.finish(curFamilyCellMap);
3549 if (isAtomic()) {
3550 throw rtbe;
3552 retCodeDetails[lastIndexExclusive] =
3553 new OperationStatus(OperationStatusCode.STORE_TOO_BUSY, rtbe.getMessage());
3554 continue;
3557 Mutation mutation = getMutation(lastIndexExclusive);
3558 // If we haven't got any rows in our batch, we should block to get the next one.
3559 RowLock rowLock = null;
3560 boolean throwException = false;
3561 try {
3562 // if atomic then get exclusive lock, else shared lock
3563 rowLock = region.getRowLock(mutation.getRow(), !isAtomic(), prevRowLock);
3564 } catch (TimeoutIOException | InterruptedIOException e) {
3565 // NOTE: We will retry when other exceptions, but we should stop if we receive
3566 // TimeoutIOException or InterruptedIOException as operation has timed out or
3567 // interrupted respectively.
3568 throwException = true;
3569 throw e;
3570 } catch (IOException ioe) {
3571 LOG.warn("Failed getting lock, row={}, in region {}",
3572 Bytes.toStringBinary(mutation.getRow()), this, ioe);
3573 if (isAtomic()) { // fail, atomic means all or none
3574 throwException = true;
3575 throw ioe;
3577 } catch (Throwable throwable) {
3578 throwException = true;
3579 throw throwable;
3580 } finally {
3581 if (throwException) {
3582 region.storeHotnessProtector.finish(curFamilyCellMap);
3585 if (rowLock == null) {
3586 // We failed to grab another lock
3587 if (isAtomic()) {
3588 region.storeHotnessProtector.finish(curFamilyCellMap);
3589 throw new IOException("Can't apply all operations atomically!");
3591 break; // Stop acquiring more rows for this batch
3592 } else {
3593 if (rowLock != prevRowLock) {
3594 // It is a different row now, add this to the acquiredRowLocks and
3595 // set prevRowLock to the new returned rowLock
3596 acquiredRowLocks.add(rowLock);
3597 prevRowLock = rowLock;
3601 readyToWriteCount++;
3603 return createMiniBatch(lastIndexExclusive, readyToWriteCount);
3606 protected MiniBatchOperationInProgress<Mutation> createMiniBatch(final int lastIndexExclusive,
3607 final int readyToWriteCount) {
3608 return new MiniBatchOperationInProgress<>(getMutationsForCoprocs(), retCodeDetails,
3609 walEditsFromCoprocessors, nextIndexToProcess, lastIndexExclusive, readyToWriteCount);
3613 * Builds separate WALEdit per nonce by applying input mutations. If WALEdits from CP are
3614 * present, they are merged to result WALEdit.
3616 public List<Pair<NonceKey, WALEdit>> buildWALEdits(
3617 final MiniBatchOperationInProgress<Mutation> miniBatchOp) throws IOException {
3618 List<Pair<NonceKey, WALEdit>> walEdits = new ArrayList<>();
3620 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), new Visitor() {
3621 private Pair<NonceKey, WALEdit> curWALEditForNonce;
3623 @Override
3624 public boolean visit(int index) throws IOException {
3625 Mutation m = getMutation(index);
3626 // we use durability of the original mutation for the mutation passed by CP.
3627 if (region.getEffectiveDurability(m.getDurability()) == Durability.SKIP_WAL) {
3628 region.recordMutationWithoutWal(m.getFamilyCellMap());
3629 return true;
3632 // the batch may contain multiple nonce keys (replay case). If so, write WALEdit for each.
3633 // Given how nonce keys are originally written, these should be contiguous.
3634 // They don't have to be, it will still work, just write more WALEdits than needed.
3635 long nonceGroup = getNonceGroup(index);
3636 long nonce = getNonce(index);
3637 if (curWALEditForNonce == null ||
3638 curWALEditForNonce.getFirst().getNonceGroup() != nonceGroup ||
3639 curWALEditForNonce.getFirst().getNonce() != nonce) {
3640 curWALEditForNonce = new Pair<>(new NonceKey(nonceGroup, nonce),
3641 new WALEdit(miniBatchOp.getCellCount(), isInReplay()));
3642 walEdits.add(curWALEditForNonce);
3644 WALEdit walEdit = curWALEditForNonce.getSecond();
3646 // Add WAL edits from CPs.
3647 WALEdit fromCP = walEditsFromCoprocessors[index];
3648 if (fromCP != null) {
3649 for (Cell cell : fromCP.getCells()) {
3650 walEdit.add(cell);
3653 walEdit.add(familyCellMaps[index]);
3655 return true;
3658 return walEdits;
3662 * This method completes mini-batch operations by calling postBatchMutate() CP hook (if
3663 * required) and completing mvcc.
3665 public void completeMiniBatchOperations(
3666 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
3667 throws IOException {
3668 if (writeEntry != null) {
3669 region.mvcc.completeAndWait(writeEntry);
3673 public void doPostOpCleanupForMiniBatch(
3674 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WALEdit walEdit,
3675 boolean success) throws IOException {
3676 doFinishHotnessProtector(miniBatchOp);
3679 private void doFinishHotnessProtector(
3680 final MiniBatchOperationInProgress<Mutation> miniBatchOp) {
3681 // check and return if the protector is not enabled
3682 if (!region.storeHotnessProtector.isEnable()) {
3683 return;
3685 // miniBatchOp is null, if and only if lockRowsAndBuildMiniBatch throwing exception.
3686 // This case was handled.
3687 if (miniBatchOp == null) {
3688 return;
3691 final int finalLastIndexExclusive = miniBatchOp.getLastIndexExclusive();
3693 for (int i = nextIndexToProcess; i < finalLastIndexExclusive; i++) {
3694 switch (retCodeDetails[i].getOperationStatusCode()) {
3695 case SUCCESS:
3696 case FAILURE:
3697 region.storeHotnessProtector.finish(getMutation(i).getFamilyCellMap());
3698 break;
3699 default:
3700 // do nothing
3701 // We won't start the protector for NOT_RUN/BAD_FAMILY/SANITY_CHECK_FAILURE and the
3702 // STORE_TOO_BUSY case is handled in StoreHotnessProtector#start
3703 break;
3709 * Atomically apply the given map of family->edits to the memstore.
3710 * This handles the consistency control on its own, but the caller
3711 * should already have locked updatesLock.readLock(). This also does
3712 * <b>not</b> check the families for validity.
3714 * @param familyMap Map of Cells by family
3716 protected void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
3717 MemStoreSizing memstoreAccounting) throws IOException {
3718 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3719 byte[] family = e.getKey();
3720 List<Cell> cells = e.getValue();
3721 assert cells instanceof RandomAccess;
3722 region.applyToMemStore(region.getStore(family), cells, false, memstoreAccounting);
3729 * Batch of mutation operations. Base class is shared with {@link ReplayBatchOperation} as most of
3730 * the logic is same.
3732 private static class MutationBatchOperation extends BatchOperation<Mutation> {
3734 // For nonce operations
3735 private long nonceGroup;
3736 private long nonce;
3737 protected boolean canProceed;
3739 public MutationBatchOperation(final HRegion region, Mutation[] operations, boolean atomic,
3740 long nonceGroup, long nonce) {
3741 super(region, operations);
3742 this.atomic = atomic;
3743 this.nonceGroup = nonceGroup;
3744 this.nonce = nonce;
3747 @Override
3748 public Mutation getMutation(int index) {
3749 return this.operations[index];
3752 @Override
3753 public long getNonceGroup(int index) {
3754 return nonceGroup;
3757 @Override
3758 public long getNonce(int index) {
3759 return nonce;
3762 @Override
3763 public Mutation[] getMutationsForCoprocs() {
3764 return this.operations;
3767 @Override
3768 public boolean isInReplay() {
3769 return false;
3772 @Override
3773 public long getOrigLogSeqNum() {
3774 return SequenceId.NO_SEQUENCE_ID;
3777 @Override
3778 public void startRegionOperation() throws IOException {
3779 region.startRegionOperation(Operation.BATCH_MUTATE);
3782 @Override
3783 public void closeRegionOperation() throws IOException {
3784 region.closeRegionOperation(Operation.BATCH_MUTATE);
3787 @Override
3788 public void checkAndPreparePut(Put p) throws IOException {
3789 region.checkFamilies(p.getFamilyCellMap().keySet(), p.getDurability());
3792 @Override
3793 public void checkAndPrepare() throws IOException {
3794 // index 0: puts, index 1: deletes, index 2: increments, index 3: append
3795 final int[] metrics = {0, 0, 0, 0};
3797 visitBatchOperations(true, this.size(), new Visitor() {
3798 private long now = EnvironmentEdgeManager.currentTime();
3799 private WALEdit walEdit;
3800 @Override
3801 public boolean visit(int index) throws IOException {
3802 // Run coprocessor pre hook outside of locks to avoid deadlock
3803 if (region.coprocessorHost != null) {
3804 if (walEdit == null) {
3805 walEdit = new WALEdit();
3807 callPreMutateCPHook(index, walEdit, metrics);
3808 if (!walEdit.isEmpty()) {
3809 walEditsFromCoprocessors[index] = walEdit;
3810 walEdit = null;
3813 if (isOperationPending(index)) {
3814 // TODO: Currently validation is done with current time before acquiring locks and
3815 // updates are done with different timestamps after acquiring locks. This behavior is
3816 // inherited from the code prior to this change. Can this be changed?
3817 checkAndPrepareMutation(index, now);
3819 return true;
3823 // FIXME: we may update metrics twice! here for all operations bypassed by CP and later in
3824 // normal processing.
3825 // Update metrics in same way as it is done when we go the normal processing route (we now
3826 // update general metrics though a Coprocessor did the work).
3827 if (region.metricsRegion != null) {
3828 if (metrics[0] > 0) {
3829 // There were some Puts in the batch.
3830 region.metricsRegion.updatePut();
3832 if (metrics[1] > 0) {
3833 // There were some Deletes in the batch.
3834 region.metricsRegion.updateDelete();
3836 if (metrics[2] > 0) {
3837 // There were some Increment in the batch.
3838 region.metricsRegion.updateIncrement();
3840 if (metrics[3] > 0) {
3841 // There were some Append in the batch.
3842 region.metricsRegion.updateAppend();
3847 @Override
3848 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
3849 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
3850 // For nonce operations
3851 canProceed = startNonceOperation();
3853 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
3854 Mutation mutation = getMutation(index);
3855 if (mutation instanceof Put) {
3856 HRegion.updateCellTimestamps(familyCellMaps[index].values(), Bytes.toBytes(timestamp));
3857 miniBatchOp.incrementNumOfPuts();
3858 } else if (mutation instanceof Delete) {
3859 region.prepareDeleteTimestamps(mutation, familyCellMaps[index],
3860 Bytes.toBytes(timestamp));
3861 miniBatchOp.incrementNumOfDeletes();
3862 } else if (mutation instanceof Increment || mutation instanceof Append) {
3863 boolean returnResults;
3864 if (mutation instanceof Increment) {
3865 returnResults = ((Increment) mutation).isReturnResults();
3866 } else {
3867 returnResults = ((Append) mutation).isReturnResults();
3870 // For nonce operations
3871 if (!canProceed) {
3872 Result result;
3873 if (returnResults) {
3874 // convert duplicate increment/append to get
3875 List<Cell> results = region.get(toGet(mutation), false, nonceGroup, nonce);
3876 result = Result.create(results);
3877 } else {
3878 result = Result.EMPTY_RESULT;
3880 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
3881 return true;
3884 Result result = null;
3885 if (region.coprocessorHost != null) {
3886 if (mutation instanceof Increment) {
3887 result = region.coprocessorHost.preIncrementAfterRowLock((Increment) mutation);
3888 } else {
3889 result = region.coprocessorHost.preAppendAfterRowLock((Append) mutation);
3892 if (result != null) {
3893 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS,
3894 returnResults ? result : Result.EMPTY_RESULT);
3895 return true;
3898 List<Cell> results = returnResults ? new ArrayList<>(mutation.size()) : null;
3899 familyCellMaps[index] = reckonDeltas(mutation, results, timestamp);
3900 this.results[index] = results != null ? Result.create(results) : Result.EMPTY_RESULT;
3902 if (mutation instanceof Increment) {
3903 miniBatchOp.incrementNumOfIncrements();
3904 } else {
3905 miniBatchOp.incrementNumOfAppends();
3908 region.rewriteCellTags(familyCellMaps[index], mutation);
3910 // update cell count
3911 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
3912 for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
3913 miniBatchOp.addCellCount(cells.size());
3917 WALEdit fromCP = walEditsFromCoprocessors[index];
3918 if (fromCP != null) {
3919 miniBatchOp.addCellCount(fromCP.size());
3921 return true;
3924 if (region.coprocessorHost != null) {
3925 // calling the pre CP hook for batch mutation
3926 region.coprocessorHost.preBatchMutate(miniBatchOp);
3927 checkAndMergeCPMutations(miniBatchOp, acquiredRowLocks, timestamp);
3932 * Starts the nonce operation for a mutation, if needed.
3933 * @return whether to proceed this mutation.
3935 private boolean startNonceOperation() throws IOException {
3936 if (region.rsServices == null || region.rsServices.getNonceManager() == null
3937 || nonce == HConstants.NO_NONCE) {
3938 return true;
3940 boolean canProceed;
3941 try {
3942 canProceed = region.rsServices.getNonceManager()
3943 .startOperation(nonceGroup, nonce, region.rsServices);
3944 } catch (InterruptedException ex) {
3945 throw new InterruptedIOException("Nonce start operation interrupted");
3947 return canProceed;
3951 * Ends nonce operation for a mutation, if needed.
3952 * @param success Whether the operation for this nonce has succeeded.
3954 private void endNonceOperation(boolean success) {
3955 if (region.rsServices != null && region.rsServices.getNonceManager() != null
3956 && nonce != HConstants.NO_NONCE) {
3957 region.rsServices.getNonceManager().endOperation(nonceGroup, nonce, success);
3961 private static Get toGet(final Mutation mutation) throws IOException {
3962 assert mutation instanceof Increment || mutation instanceof Append;
3963 Get get = new Get(mutation.getRow());
3964 CellScanner cellScanner = mutation.cellScanner();
3965 while (!cellScanner.advance()) {
3966 Cell cell = cellScanner.current();
3967 get.addColumn(CellUtil.cloneFamily(cell), CellUtil.cloneQualifier(cell));
3969 if (mutation instanceof Increment) {
3970 // Increment
3971 Increment increment = (Increment) mutation;
3972 get.setTimeRange(increment.getTimeRange().getMin(), increment.getTimeRange().getMax());
3973 } else {
3974 // Append
3975 Append append = (Append) mutation;
3976 get.setTimeRange(append.getTimeRange().getMin(), append.getTimeRange().getMax());
3978 for (Entry<String, byte[]> entry : mutation.getAttributesMap().entrySet()) {
3979 get.setAttribute(entry.getKey(), entry.getValue());
3981 return get;
3984 private Map<byte[], List<Cell>> reckonDeltas(Mutation mutation, List<Cell> results,
3985 long now) throws IOException {
3986 assert mutation instanceof Increment || mutation instanceof Append;
3987 Map<byte[], List<Cell>> ret = new TreeMap<>(Bytes.BYTES_COMPARATOR);
3988 // Process a Store/family at a time.
3989 for (Map.Entry<byte [], List<Cell>> entry: mutation.getFamilyCellMap().entrySet()) {
3990 final byte[] columnFamilyName = entry.getKey();
3991 List<Cell> deltas = entry.getValue();
3992 // Reckon for the Store what to apply to WAL and MemStore.
3993 List<Cell> toApply = reckonDeltasByStore(region.stores.get(columnFamilyName), mutation,
3994 now, deltas, results);
3995 if (!toApply.isEmpty()) {
3996 for (Cell cell : toApply) {
3997 HStore store = region.getStore(cell);
3998 if (store == null) {
3999 region.checkFamily(CellUtil.cloneFamily(cell));
4000 } else {
4001 ret.computeIfAbsent(store.getColumnFamilyDescriptor().getName(),
4002 key -> new ArrayList<>()).add(cell);
4007 return ret;
4011 * Reckon the Cells to apply to WAL, memstore, and to return to the Client in passed
4012 * column family/Store.
4014 * Does Get of current value and then adds passed in deltas for this Store returning the
4015 * result.
4017 * @param mutation The encompassing Mutation object
4018 * @param deltas Changes to apply to this Store; either increment amount or data to append
4019 * @param results In here we accumulate all the Cells we are to return to the client. If null,
4020 * client doesn't want results returned.
4021 * @return Resulting Cells after <code>deltas</code> have been applied to current
4022 * values. Side effect is our filling out of the <code>results</code> List.
4024 private List<Cell> reckonDeltasByStore(HStore store, Mutation mutation, long now,
4025 List<Cell> deltas, List<Cell> results) throws IOException {
4026 assert mutation instanceof Increment || mutation instanceof Append;
4027 byte[] columnFamily = store.getColumnFamilyDescriptor().getName();
4028 List<Pair<Cell, Cell>> cellPairs = new ArrayList<>(deltas.size());
4030 // Sort the cells so that they match the order that they appear in the Get results.
4031 // Otherwise, we won't be able to find the existing values if the cells are not specified
4032 // in order by the client since cells are in an array list.
4033 deltas.sort(store.getComparator());
4035 // Get previous values for all columns in this family.
4036 Get get = new Get(mutation.getRow());
4037 for (Cell cell: deltas) {
4038 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
4040 TimeRange tr;
4041 if (mutation instanceof Increment) {
4042 tr = ((Increment) mutation).getTimeRange();
4043 } else {
4044 tr = ((Append) mutation).getTimeRange();
4047 if (tr != null) {
4048 get.setTimeRange(tr.getMin(), tr.getMax());
4051 try (RegionScanner scanner = region.getScanner(new Scan(get))) {
4052 // NOTE: Please don't use HRegion.get() instead,
4053 // because it will copy cells to heap. See HBASE-26036
4054 List<Cell> currentValues = new ArrayList<>();
4055 scanner.next(currentValues);
4056 // Iterate the input columns and update existing values if they were found, otherwise
4057 // add new column initialized to the delta amount
4058 int currentValuesIndex = 0;
4059 for (int i = 0; i < deltas.size(); i++) {
4060 Cell delta = deltas.get(i);
4061 Cell currentValue = null;
4062 if (currentValuesIndex < currentValues.size() && CellUtil
4063 .matchingQualifier(currentValues.get(currentValuesIndex), delta)) {
4064 currentValue = currentValues.get(currentValuesIndex);
4065 if (i < (deltas.size() - 1) && !CellUtil.matchingQualifier(delta, deltas.get(i + 1))) {
4066 currentValuesIndex++;
4069 // Switch on whether this an increment or an append building the new Cell to apply.
4070 Cell newCell;
4071 if (mutation instanceof Increment) {
4072 long deltaAmount = getLongValue(delta);
4073 final long newValue = currentValue == null ? deltaAmount :
4074 getLongValue(currentValue) + deltaAmount;
4075 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4076 (oldCell) -> Bytes.toBytes(newValue));
4077 } else {
4078 newCell = reckonDelta(delta, currentValue, columnFamily, now, mutation,
4079 (oldCell) -> ByteBuffer.wrap(new byte[delta.getValueLength() +
4080 oldCell.getValueLength()])
4081 .put(oldCell.getValueArray(), oldCell.getValueOffset(), oldCell.getValueLength())
4082 .put(delta.getValueArray(), delta.getValueOffset(), delta.getValueLength())
4083 .array());
4085 if (region.maxCellSize > 0) {
4086 int newCellSize = PrivateCellUtil.estimatedSerializedSizeOf(newCell);
4087 if (newCellSize > region.maxCellSize) {
4088 String msg =
4089 "Cell with size " + newCellSize + " exceeds limit of " + region.maxCellSize +
4090 " bytes in region " + this;
4091 LOG.debug(msg);
4092 throw new DoNotRetryIOException(msg);
4095 cellPairs.add(new Pair<>(currentValue, newCell));
4096 // Add to results to get returned to the Client. If null, cilent does not want results.
4097 if (results != null) {
4098 results.add(newCell);
4101 // Give coprocessors a chance to update the new cells before apply to WAL or memstore
4102 if (region.coprocessorHost != null) {
4103 // Here the operation must be increment or append.
4104 cellPairs = mutation instanceof Increment ?
4105 region.coprocessorHost.postIncrementBeforeWAL(mutation, cellPairs) :
4106 region.coprocessorHost.postAppendBeforeWAL(mutation, cellPairs);
4109 return cellPairs.stream().map(Pair::getSecond).collect(Collectors.toList());
4112 private static Cell reckonDelta(final Cell delta, final Cell currentCell,
4113 final byte[] columnFamily, final long now, Mutation mutation,
4114 Function<Cell, byte[]> supplier) throws IOException {
4115 // Forward any tags found on the delta.
4116 List<Tag> tags = TagUtil.carryForwardTags(delta);
4117 if (currentCell != null) {
4118 tags = TagUtil.carryForwardTags(tags, currentCell);
4119 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4120 byte[] newValue = supplier.apply(currentCell);
4121 return ExtendedCellBuilderFactory.create(CellBuilderType.SHALLOW_COPY)
4122 .setRow(mutation.getRow(), 0, mutation.getRow().length)
4123 .setFamily(columnFamily, 0, columnFamily.length)
4124 // copy the qualifier if the cell is located in shared memory.
4125 .setQualifier(CellUtil.cloneQualifier(delta))
4126 .setTimestamp(Math.max(currentCell.getTimestamp() + 1, now))
4127 .setType(KeyValue.Type.Put.getCode())
4128 .setValue(newValue, 0, newValue.length)
4129 .setTags(TagUtil.fromList(tags))
4130 .build();
4131 } else {
4132 tags = TagUtil.carryForwardTTLTag(tags, mutation.getTTL());
4133 PrivateCellUtil.updateLatestStamp(delta, now);
4134 return CollectionUtils.isEmpty(tags) ? delta : PrivateCellUtil.createCell(delta, tags);
4139 * @return Get the long out of the passed in Cell
4141 private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
4142 int len = cell.getValueLength();
4143 if (len != Bytes.SIZEOF_LONG) {
4144 // throw DoNotRetryIOException instead of IllegalArgumentException
4145 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
4147 return PrivateCellUtil.getValueAsLong(cell);
4150 @Override
4151 public List<Pair<NonceKey, WALEdit>> buildWALEdits(final MiniBatchOperationInProgress<Mutation>
4152 miniBatchOp) throws IOException {
4153 List<Pair<NonceKey, WALEdit>> walEdits = super.buildWALEdits(miniBatchOp);
4154 // for MutationBatchOperation, more than one nonce is not allowed
4155 if (walEdits.size() > 1) {
4156 throw new IOException("Found multiple nonce keys per batch!");
4158 return walEdits;
4161 @Override
4162 public WriteEntry writeMiniBatchOperationsToMemStore(
4163 final MiniBatchOperationInProgress<Mutation> miniBatchOp, @Nullable WriteEntry writeEntry)
4164 throws IOException {
4165 if (writeEntry == null) {
4166 writeEntry = region.mvcc.begin();
4168 super.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry.getWriteNumber());
4169 return writeEntry;
4172 @Override
4173 public void completeMiniBatchOperations(
4174 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4175 throws IOException {
4176 // TODO: can it be done after completing mvcc?
4177 // calling the post CP hook for batch mutation
4178 if (region.coprocessorHost != null) {
4179 region.coprocessorHost.postBatchMutate(miniBatchOp);
4181 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4183 if (nonce != HConstants.NO_NONCE) {
4184 if (region.rsServices != null && region.rsServices.getNonceManager() != null) {
4185 region.rsServices.getNonceManager()
4186 .addMvccToOperationContext(nonceGroup, nonce, writeEntry.getWriteNumber());
4191 @Override
4192 public void doPostOpCleanupForMiniBatch(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4193 final WALEdit walEdit, boolean success) throws IOException {
4195 super.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, success);
4196 if (miniBatchOp != null) {
4197 // synced so that the coprocessor contract is adhered to.
4198 if (region.coprocessorHost != null) {
4199 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4200 // only for successful puts/deletes/increments/appends
4201 if (retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.SUCCESS) {
4202 Mutation m = getMutation(i);
4203 if (m instanceof Put) {
4204 region.coprocessorHost.postPut((Put) m, walEdit);
4205 } else if (m instanceof Delete) {
4206 region.coprocessorHost.postDelete((Delete) m, walEdit);
4207 } else if (m instanceof Increment) {
4208 Result result = region.getCoprocessorHost().postIncrement((Increment) m,
4209 results[i], walEdit);
4210 if (result != results[i]) {
4211 retCodeDetails[i] =
4212 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4214 } else if (m instanceof Append) {
4215 Result result = region.getCoprocessorHost().postAppend((Append) m, results[i],
4216 walEdit);
4217 if (result != results[i]) {
4218 retCodeDetails[i] =
4219 new OperationStatus(retCodeDetails[i].getOperationStatusCode(), result);
4223 return true;
4227 // For nonce operations
4228 if (canProceed && nonce != HConstants.NO_NONCE) {
4229 boolean[] areAllIncrementsAndAppendsSuccessful = new boolean[]{true};
4230 visitBatchOperations(false, miniBatchOp.getLastIndexExclusive(), (int i) -> {
4231 Mutation mutation = getMutation(i);
4232 if (mutation instanceof Increment || mutation instanceof Append) {
4233 if (retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) {
4234 areAllIncrementsAndAppendsSuccessful[0] = false;
4235 return false;
4238 return true;
4240 endNonceOperation(areAllIncrementsAndAppendsSuccessful[0]);
4243 // See if the column families were consistent through the whole thing.
4244 // if they were then keep them. If they were not then pass a null.
4245 // null will be treated as unknown.
4246 // Total time taken might be involving Puts, Deletes, Increments and Appends.
4247 // Split the time for puts and deletes based on the total number of Puts, Deletes,
4248 // Increments and Appends.
4249 if (region.metricsRegion != null) {
4250 if (miniBatchOp.getNumOfPuts() > 0) {
4251 // There were some Puts in the batch.
4252 region.metricsRegion.updatePut();
4254 if (miniBatchOp.getNumOfDeletes() > 0) {
4255 // There were some Deletes in the batch.
4256 region.metricsRegion.updateDelete();
4258 if (miniBatchOp.getNumOfIncrements() > 0) {
4259 // There were some Increments in the batch.
4260 region.metricsRegion.updateIncrement();
4262 if (miniBatchOp.getNumOfAppends() > 0) {
4263 // There were some Appends in the batch.
4264 region.metricsRegion.updateAppend();
4269 if (region.coprocessorHost != null) {
4270 // call the coprocessor hook to do any finalization steps after the put is done
4271 region.coprocessorHost.postBatchMutateIndispensably(
4272 miniBatchOp != null ? miniBatchOp : createMiniBatch(size(), 0), success);
4277 * Runs prePut/preDelete/preIncrement/preAppend coprocessor hook for input mutation in a batch
4278 * @param metrics Array of 2 ints. index 0: count of puts, index 1: count of deletes, index 2:
4279 * count of increments and 3: count of appends
4281 private void callPreMutateCPHook(int index, final WALEdit walEdit, final int[] metrics)
4282 throws IOException {
4283 Mutation m = getMutation(index);
4284 if (m instanceof Put) {
4285 if (region.coprocessorHost.prePut((Put) m, walEdit)) {
4286 // pre hook says skip this Put
4287 // mark as success and skip in doMiniBatchMutation
4288 metrics[0]++;
4289 retCodeDetails[index] = OperationStatus.SUCCESS;
4291 } else if (m instanceof Delete) {
4292 Delete curDel = (Delete) m;
4293 if (curDel.getFamilyCellMap().isEmpty()) {
4294 // handle deleting a row case
4295 // TODO: prepareDelete() has been called twice, before and after preDelete() CP hook.
4296 // Can this be avoided?
4297 region.prepareDelete(curDel);
4299 if (region.coprocessorHost.preDelete(curDel, walEdit)) {
4300 // pre hook says skip this Delete
4301 // mark as success and skip in doMiniBatchMutation
4302 metrics[1]++;
4303 retCodeDetails[index] = OperationStatus.SUCCESS;
4305 } else if (m instanceof Increment) {
4306 Increment increment = (Increment) m;
4307 Result result = region.coprocessorHost.preIncrement(increment, walEdit);
4308 if (result != null) {
4309 // pre hook says skip this Increment
4310 // mark as success and skip in doMiniBatchMutation
4311 metrics[2]++;
4312 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4314 } else if (m instanceof Append) {
4315 Append append = (Append) m;
4316 Result result = region.coprocessorHost.preAppend(append, walEdit);
4317 if (result != null) {
4318 // pre hook says skip this Append
4319 // mark as success and skip in doMiniBatchMutation
4320 metrics[3]++;
4321 retCodeDetails[index] = new OperationStatus(OperationStatusCode.SUCCESS, result);
4323 } else {
4324 String msg = "Put/Delete/Increment/Append mutations only supported in a batch";
4325 retCodeDetails[index] = new OperationStatus(OperationStatusCode.FAILURE, msg);
4326 if (isAtomic()) { // fail, atomic means all or none
4327 throw new IOException(msg);
4332 // TODO Support Increment/Append operations
4333 private void checkAndMergeCPMutations(final MiniBatchOperationInProgress<Mutation> miniBatchOp,
4334 final List<RowLock> acquiredRowLocks, final long timestamp) throws IOException {
4335 visitBatchOperations(true, nextIndexToProcess + miniBatchOp.size(), (int i) -> {
4336 // we pass (i - firstIndex) below since the call expects a relative index
4337 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - nextIndexToProcess);
4338 if (cpMutations == null) {
4339 return true;
4341 // Else Coprocessor added more Mutations corresponding to the Mutation at this index.
4342 Mutation mutation = getMutation(i);
4343 for (Mutation cpMutation : cpMutations) {
4344 this.checkAndPrepareMutation(cpMutation, timestamp);
4346 // Acquire row locks. If not, the whole batch will fail.
4347 acquiredRowLocks.add(region.getRowLock(cpMutation.getRow(), true, null));
4349 // Returned mutations from coprocessor correspond to the Mutation at index i. We can
4350 // directly add the cells from those mutations to the familyMaps of this mutation.
4351 Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
4352 region.rewriteCellTags(cpFamilyMap, mutation);
4353 // will get added to the memStore later
4354 mergeFamilyMaps(familyCellMaps[i], cpFamilyMap);
4356 // The durability of returned mutation is replaced by the corresponding mutation.
4357 // If the corresponding mutation contains the SKIP_WAL, we shouldn't count the
4358 // cells of returned mutation.
4359 if (region.getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
4360 for (List<Cell> cells : cpFamilyMap.values()) {
4361 miniBatchOp.addCellCount(cells.size());
4365 return true;
4369 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
4370 Map<byte[], List<Cell>> toBeMerged) {
4371 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
4372 List<Cell> cells = familyMap.get(entry.getKey());
4373 if (cells == null) {
4374 familyMap.put(entry.getKey(), entry.getValue());
4375 } else {
4376 cells.addAll(entry.getValue());
4383 * Batch of mutations for replay. Base class is shared with {@link MutationBatchOperation} as most
4384 * of the logic is same.
4386 private static final class ReplayBatchOperation extends BatchOperation<MutationReplay> {
4388 private long origLogSeqNum = 0;
4390 public ReplayBatchOperation(final HRegion region, MutationReplay[] operations,
4391 long origLogSeqNum) {
4392 super(region, operations);
4393 this.origLogSeqNum = origLogSeqNum;
4396 @Override
4397 public Mutation getMutation(int index) {
4398 return this.operations[index].mutation;
4401 @Override
4402 public long getNonceGroup(int index) {
4403 return this.operations[index].nonceGroup;
4406 @Override
4407 public long getNonce(int index) {
4408 return this.operations[index].nonce;
4411 @Override
4412 public Mutation[] getMutationsForCoprocs() {
4413 return null;
4416 @Override
4417 public boolean isInReplay() {
4418 return true;
4421 @Override
4422 public long getOrigLogSeqNum() {
4423 return this.origLogSeqNum;
4426 @Override
4427 public void startRegionOperation() throws IOException {
4428 region.startRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4431 @Override
4432 public void closeRegionOperation() throws IOException {
4433 region.closeRegionOperation(Operation.REPLAY_BATCH_MUTATE);
4437 * During replay, there could exist column families which are removed between region server
4438 * failure and replay
4440 @Override
4441 protected void checkAndPreparePut(Put p) throws IOException {
4442 Map<byte[], List<Cell>> familyCellMap = p.getFamilyCellMap();
4443 List<byte[]> nonExistentList = null;
4444 for (byte[] family : familyCellMap.keySet()) {
4445 if (!region.htableDescriptor.hasColumnFamily(family)) {
4446 if (nonExistentList == null) {
4447 nonExistentList = new ArrayList<>();
4449 nonExistentList.add(family);
4452 if (nonExistentList != null) {
4453 for (byte[] family : nonExistentList) {
4454 // Perhaps schema was changed between crash and replay
4455 LOG.info("No family for {} omit from reply in region {}.", Bytes.toString(family), this);
4456 familyCellMap.remove(family);
4461 @Override
4462 public void checkAndPrepare() throws IOException {
4463 long now = EnvironmentEdgeManager.currentTime();
4464 visitBatchOperations(true, this.size(), (int index) -> {
4465 checkAndPrepareMutation(index, now);
4466 return true;
4470 @Override
4471 public void prepareMiniBatchOperations(MiniBatchOperationInProgress<Mutation> miniBatchOp,
4472 long timestamp, final List<RowLock> acquiredRowLocks) throws IOException {
4473 visitBatchOperations(true, miniBatchOp.getLastIndexExclusive(), (int index) -> {
4474 // update cell count
4475 for (List<Cell> cells : getMutation(index).getFamilyCellMap().values()) {
4476 miniBatchOp.addCellCount(cells.size());
4478 return true;
4482 @Override
4483 public WriteEntry writeMiniBatchOperationsToMemStore(
4484 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4485 throws IOException {
4486 super.writeMiniBatchOperationsToMemStore(miniBatchOp, getOrigLogSeqNum());
4487 return writeEntry;
4490 @Override
4491 public void completeMiniBatchOperations(
4492 final MiniBatchOperationInProgress<Mutation> miniBatchOp, final WriteEntry writeEntry)
4493 throws IOException {
4494 super.completeMiniBatchOperations(miniBatchOp, writeEntry);
4495 region.mvcc.advanceTo(getOrigLogSeqNum());
4499 public OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic, long nonceGroup,
4500 long nonce) throws IOException {
4501 // As it stands, this is used for 3 things
4502 // * batchMutate with single mutation - put/delete/increment/append, separate or from
4503 // checkAndMutate.
4504 // * coprocessor calls (see ex. BulkDeleteEndpoint).
4505 // So nonces are not really ever used by HBase. They could be by coprocs, and checkAnd...
4506 return batchMutate(new MutationBatchOperation(this, mutations, atomic, nonceGroup, nonce));
4509 @Override
4510 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
4511 // If the mutations has any Increment/Append operations, we need to do batchMutate atomically
4512 boolean atomic =
4513 Arrays.stream(mutations).anyMatch(m -> m instanceof Increment || m instanceof Append);
4514 return batchMutate(mutations, atomic);
4517 OperationStatus[] batchMutate(Mutation[] mutations, boolean atomic) throws IOException {
4518 return TraceUtil.trace(
4519 () -> batchMutate(mutations, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE),
4520 () -> createRegionSpan("Region.batchMutate"));
4523 public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
4524 throws IOException {
4525 if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
4526 && replaySeqId < lastReplayedOpenRegionSeqId) {
4527 // if it is a secondary replica we should ignore these entries silently
4528 // since they are coming out of order
4529 if (LOG.isTraceEnabled()) {
4530 LOG.trace(getRegionInfo().getEncodedName() + " : "
4531 + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId
4532 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
4533 for (MutationReplay mut : mutations) {
4534 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
4538 OperationStatus[] statuses = new OperationStatus[mutations.length];
4539 for (int i = 0; i < statuses.length; i++) {
4540 statuses[i] = OperationStatus.SUCCESS;
4542 return statuses;
4544 return batchMutate(new ReplayBatchOperation(this, mutations, replaySeqId));
4548 * Perform a batch of mutations.
4549 * <p/>
4550 * Operations in a batch are stored with highest durability specified of for all operations in a
4551 * batch, except for {@link Durability#SKIP_WAL}.
4552 * <p/>
4553 * This function is called from {@link #batchReplay(WALSplitUtil.MutationReplay[], long)} with
4554 * {@link ReplayBatchOperation} instance and {@link #batchMutate(Mutation[])} with
4555 * {@link MutationBatchOperation} instance as an argument. As the processing of replay batch and
4556 * mutation batch is very similar, lot of code is shared by providing generic methods in base
4557 * class {@link BatchOperation}. The logic for this method and
4558 * {@link #doMiniBatchMutate(BatchOperation)} is implemented using methods in base class which are
4559 * overridden by derived classes to implement special behavior.
4560 * @param batchOp contains the list of mutations
4561 * @return an array of OperationStatus which internally contains the OperationStatusCode and the
4562 * exceptionMessage if any.
4563 * @throws IOException if an IO problem is encountered
4565 private OperationStatus[] batchMutate(BatchOperation<?> batchOp) throws IOException {
4566 boolean initialized = false;
4567 batchOp.startRegionOperation();
4568 try {
4569 while (!batchOp.isDone()) {
4570 if (!batchOp.isInReplay()) {
4571 checkReadOnly();
4573 checkResources();
4575 if (!initialized) {
4576 this.writeRequestsCount.add(batchOp.size());
4577 // validate and prepare batch for write, for MutationBatchOperation it also calls CP
4578 // prePut()/preDelete()/preIncrement()/preAppend() hooks
4579 batchOp.checkAndPrepare();
4580 initialized = true;
4582 doMiniBatchMutate(batchOp);
4583 requestFlushIfNeeded();
4585 } finally {
4586 if (rsServices != null && rsServices.getMetrics() != null) {
4587 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
4588 getTableName(), batchOp.size());
4590 batchOp.closeRegionOperation();
4592 return batchOp.retCodeDetails;
4596 * Called to do a piece of the batch that came in to {@link #batchMutate(Mutation[])}
4597 * In here we also handle replay of edits on region recover. Also gets change in size brought
4598 * about by applying {@code batchOp}.
4600 private void doMiniBatchMutate(BatchOperation<?> batchOp) throws IOException {
4601 boolean success = false;
4602 WALEdit walEdit = null;
4603 WriteEntry writeEntry = null;
4604 boolean locked = false;
4605 // We try to set up a batch in the range [batchOp.nextIndexToProcess,lastIndexExclusive)
4606 MiniBatchOperationInProgress<Mutation> miniBatchOp = null;
4607 /** Keep track of the locks we hold so we can release them in finally clause */
4608 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.size());
4610 // Check for thread interrupt status in case we have been signaled from
4611 // #interruptRegionOperation.
4612 checkInterrupt();
4614 try {
4615 // STEP 1. Try to acquire as many locks as we can and build mini-batch of operations with
4616 // locked rows
4617 miniBatchOp = batchOp.lockRowsAndBuildMiniBatch(acquiredRowLocks);
4619 // We've now grabbed as many mutations off the list as we can
4620 // Ensure we acquire at least one.
4621 if (miniBatchOp.getReadyToWriteCount() <= 0) {
4622 // Nothing to put/delete/increment/append -- an exception in the above such as
4623 // NoSuchColumnFamily?
4624 return;
4627 // Check for thread interrupt status in case we have been signaled from
4628 // #interruptRegionOperation. Do it before we take the lock and disable interrupts for
4629 // the WAL append.
4630 checkInterrupt();
4632 lock(this.updatesLock.readLock(), miniBatchOp.getReadyToWriteCount());
4633 locked = true;
4635 // From this point until memstore update this operation should not be interrupted.
4636 disableInterrupts();
4638 // STEP 2. Update mini batch of all operations in progress with LATEST_TIMESTAMP timestamp
4639 // We should record the timestamp only after we have acquired the rowLock,
4640 // otherwise, newer puts/deletes/increment/append are not guaranteed to have a newer
4641 // timestamp
4643 long now = EnvironmentEdgeManager.currentTime();
4644 batchOp.prepareMiniBatchOperations(miniBatchOp, now, acquiredRowLocks);
4646 // STEP 3. Build WAL edit
4648 List<Pair<NonceKey, WALEdit>> walEdits = batchOp.buildWALEdits(miniBatchOp);
4650 // STEP 4. Append the WALEdits to WAL and sync.
4652 for(Iterator<Pair<NonceKey, WALEdit>> it = walEdits.iterator(); it.hasNext();) {
4653 Pair<NonceKey, WALEdit> nonceKeyWALEditPair = it.next();
4654 walEdit = nonceKeyWALEditPair.getSecond();
4655 NonceKey nonceKey = nonceKeyWALEditPair.getFirst();
4657 if (walEdit != null && !walEdit.isEmpty()) {
4658 writeEntry = doWALAppend(walEdit, batchOp.durability, batchOp.getClusterIds(), now,
4659 nonceKey.getNonceGroup(), nonceKey.getNonce(), batchOp.getOrigLogSeqNum());
4662 // Complete mvcc for all but last writeEntry (for replay case)
4663 if (it.hasNext() && writeEntry != null) {
4664 mvcc.complete(writeEntry);
4665 writeEntry = null;
4669 // STEP 5. Write back to memStore
4670 // NOTE: writeEntry can be null here
4671 writeEntry = batchOp.writeMiniBatchOperationsToMemStore(miniBatchOp, writeEntry);
4673 // STEP 6. Complete MiniBatchOperations: If required calls postBatchMutate() CP hook and
4674 // complete mvcc for last writeEntry
4675 batchOp.completeMiniBatchOperations(miniBatchOp, writeEntry);
4676 writeEntry = null;
4677 success = true;
4678 } finally {
4679 // Call complete rather than completeAndWait because we probably had error if walKey != null
4680 if (writeEntry != null) mvcc.complete(writeEntry);
4682 if (locked) {
4683 this.updatesLock.readLock().unlock();
4685 releaseRowLocks(acquiredRowLocks);
4687 enableInterrupts();
4689 final int finalLastIndexExclusive =
4690 miniBatchOp != null ? miniBatchOp.getLastIndexExclusive() : batchOp.size();
4691 final boolean finalSuccess = success;
4692 batchOp.visitBatchOperations(true, finalLastIndexExclusive,
4693 (int i) -> {
4694 Mutation mutation = batchOp.getMutation(i);
4695 if (mutation instanceof Increment || mutation instanceof Append) {
4696 if (finalSuccess) {
4697 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.SUCCESS,
4698 batchOp.results[i]);
4699 } else {
4700 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
4702 } else {
4703 batchOp.retCodeDetails[i] =
4704 finalSuccess ? OperationStatus.SUCCESS : OperationStatus.FAILURE;
4706 return true;
4709 batchOp.doPostOpCleanupForMiniBatch(miniBatchOp, walEdit, finalSuccess);
4711 batchOp.nextIndexToProcess = finalLastIndexExclusive;
4716 * Returns effective durability from the passed durability and
4717 * the table descriptor.
4719 private Durability getEffectiveDurability(Durability d) {
4720 return d == Durability.USE_DEFAULT ? this.regionDurability : d;
4723 @Override
4724 @Deprecated
4725 public boolean checkAndMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4726 ByteArrayComparable comparator, TimeRange timeRange, Mutation mutation) throws IOException {
4727 CheckAndMutate checkAndMutate;
4728 try {
4729 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row)
4730 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange);
4731 if (mutation instanceof Put) {
4732 checkAndMutate = builder.build((Put) mutation);
4733 } else if (mutation instanceof Delete) {
4734 checkAndMutate = builder.build((Delete) mutation);
4735 } else {
4736 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4737 .getSimpleName().toUpperCase());
4739 } catch (IllegalArgumentException e) {
4740 throw new DoNotRetryIOException(e.getMessage());
4742 return checkAndMutate(checkAndMutate).isSuccess();
4745 @Override
4746 @Deprecated
4747 public boolean checkAndMutate(byte[] row, Filter filter, TimeRange timeRange, Mutation mutation)
4748 throws IOException {
4749 CheckAndMutate checkAndMutate;
4750 try {
4751 CheckAndMutate.Builder builder = CheckAndMutate.newBuilder(row).ifMatches(filter)
4752 .timeRange(timeRange);
4753 if (mutation instanceof Put) {
4754 checkAndMutate = builder.build((Put) mutation);
4755 } else if (mutation instanceof Delete) {
4756 checkAndMutate = builder.build((Delete) mutation);
4757 } else {
4758 throw new DoNotRetryIOException("Unsupported mutate type: " + mutation.getClass()
4759 .getSimpleName().toUpperCase());
4761 } catch (IllegalArgumentException e) {
4762 throw new DoNotRetryIOException(e.getMessage());
4764 return checkAndMutate(checkAndMutate).isSuccess();
4767 @Override
4768 @Deprecated
4769 public boolean checkAndRowMutate(byte[] row, byte[] family, byte[] qualifier, CompareOperator op,
4770 ByteArrayComparable comparator, TimeRange timeRange, RowMutations rm) throws IOException {
4771 CheckAndMutate checkAndMutate;
4772 try {
4773 checkAndMutate = CheckAndMutate.newBuilder(row)
4774 .ifMatches(family, qualifier, op, comparator.getValue()).timeRange(timeRange).build(rm);
4775 } catch (IllegalArgumentException e) {
4776 throw new DoNotRetryIOException(e.getMessage());
4778 return checkAndMutate(checkAndMutate).isSuccess();
4781 @Override
4782 @Deprecated
4783 public boolean checkAndRowMutate(byte[] row, Filter filter, TimeRange timeRange, RowMutations rm)
4784 throws IOException {
4785 CheckAndMutate checkAndMutate;
4786 try {
4787 checkAndMutate = CheckAndMutate.newBuilder(row).ifMatches(filter).timeRange(timeRange)
4788 .build(rm);
4789 } catch (IllegalArgumentException e) {
4790 throw new DoNotRetryIOException(e.getMessage());
4792 return checkAndMutate(checkAndMutate).isSuccess();
4795 @Override
4796 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate) throws IOException {
4797 return checkAndMutate(checkAndMutate, HConstants.NO_NONCE, HConstants.NO_NONCE);
4800 public CheckAndMutateResult checkAndMutate(CheckAndMutate checkAndMutate, long nonceGroup,
4801 long nonce) throws IOException {
4802 return TraceUtil.trace(() -> checkAndMutateInternal(checkAndMutate, nonceGroup, nonce),
4803 () -> createRegionSpan("Region.checkAndMutate"));
4806 private CheckAndMutateResult checkAndMutateInternal(CheckAndMutate checkAndMutate,
4807 long nonceGroup, long nonce) throws IOException {
4808 byte[] row = checkAndMutate.getRow();
4809 Filter filter = null;
4810 byte[] family = null;
4811 byte[] qualifier = null;
4812 CompareOperator op = null;
4813 ByteArrayComparable comparator = null;
4814 if (checkAndMutate.hasFilter()) {
4815 filter = checkAndMutate.getFilter();
4816 } else {
4817 family = checkAndMutate.getFamily();
4818 qualifier = checkAndMutate.getQualifier();
4819 op = checkAndMutate.getCompareOp();
4820 comparator = new BinaryComparator(checkAndMutate.getValue());
4822 TimeRange timeRange = checkAndMutate.getTimeRange();
4824 Mutation mutation = null;
4825 RowMutations rowMutations = null;
4826 if (checkAndMutate.getAction() instanceof Mutation) {
4827 mutation = (Mutation) checkAndMutate.getAction();
4828 } else {
4829 rowMutations = (RowMutations) checkAndMutate.getAction();
4832 if (mutation != null) {
4833 checkMutationType(mutation);
4834 checkRow(mutation, row);
4835 } else {
4836 checkRow(rowMutations, row);
4838 checkReadOnly();
4839 // TODO, add check for value length also move this check to the client
4840 checkResources();
4841 startRegionOperation();
4842 try {
4843 Get get = new Get(row);
4844 if (family != null) {
4845 checkFamily(family);
4846 get.addColumn(family, qualifier);
4848 if (filter != null) {
4849 get.setFilter(filter);
4851 if (timeRange != null) {
4852 get.setTimeRange(timeRange.getMin(), timeRange.getMax());
4854 // Lock row - note that doBatchMutate will relock this row if called
4855 checkRow(row, "doCheckAndRowMutate");
4856 RowLock rowLock = getRowLock(get.getRow(), false, null);
4857 try {
4858 if (this.getCoprocessorHost() != null) {
4859 CheckAndMutateResult result =
4860 getCoprocessorHost().preCheckAndMutateAfterRowLock(checkAndMutate);
4861 if (result != null) {
4862 return result;
4866 // NOTE: We used to wait here until mvcc caught up: mvcc.await();
4867 // Supposition is that now all changes are done under row locks, then when we go to read,
4868 // we'll get the latest on this row.
4869 boolean matches = false;
4870 long cellTs = 0;
4871 try (RegionScanner scanner = getScanner(new Scan(get))) {
4872 // NOTE: Please don't use HRegion.get() instead,
4873 // because it will copy cells to heap. See HBASE-26036
4874 List<Cell> result = new ArrayList<>(1);
4875 scanner.next(result);
4876 if (filter != null) {
4877 if (!result.isEmpty()) {
4878 matches = true;
4879 cellTs = result.get(0).getTimestamp();
4881 } else {
4882 boolean valueIsNull =
4883 comparator.getValue() == null || comparator.getValue().length == 0;
4884 if (result.isEmpty() && valueIsNull) {
4885 matches = true;
4886 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 && valueIsNull) {
4887 matches = true;
4888 cellTs = result.get(0).getTimestamp();
4889 } else if (result.size() == 1 && !valueIsNull) {
4890 Cell kv = result.get(0);
4891 cellTs = kv.getTimestamp();
4892 int compareResult = PrivateCellUtil.compareValue(kv, comparator);
4893 matches = matches(op, compareResult);
4898 // If matches, perform the mutation or the rowMutations
4899 if (matches) {
4900 // We have acquired the row lock already. If the system clock is NOT monotonically
4901 // non-decreasing (see HBASE-14070) we should make sure that the mutation has a
4902 // larger timestamp than what was observed via Get. doBatchMutate already does this, but
4903 // there is no way to pass the cellTs. See HBASE-14054.
4904 long now = EnvironmentEdgeManager.currentTime();
4905 long ts = Math.max(now, cellTs); // ensure write is not eclipsed
4906 byte[] byteTs = Bytes.toBytes(ts);
4907 if (mutation != null) {
4908 if (mutation instanceof Put) {
4909 updateCellTimestamps(mutation.getFamilyCellMap().values(), byteTs);
4911 // And else 'delete' is not needed since it already does a second get, and sets the
4912 // timestamp from get (see prepareDeleteTimestamps).
4913 } else {
4914 for (Mutation m : rowMutations.getMutations()) {
4915 if (m instanceof Put) {
4916 updateCellTimestamps(m.getFamilyCellMap().values(), byteTs);
4919 // And else 'delete' is not needed since it already does a second get, and sets the
4920 // timestamp from get (see prepareDeleteTimestamps).
4922 // All edits for the given row (across all column families) must happen atomically.
4923 Result r;
4924 if (mutation != null) {
4925 r = mutate(mutation, true, nonceGroup, nonce).getResult();
4926 } else {
4927 r = mutateRow(rowMutations, nonceGroup, nonce);
4929 this.checkAndMutateChecksPassed.increment();
4930 return new CheckAndMutateResult(true, r);
4932 this.checkAndMutateChecksFailed.increment();
4933 return new CheckAndMutateResult(false, null);
4934 } finally {
4935 rowLock.release();
4937 } finally {
4938 closeRegionOperation();
4942 private void checkMutationType(final Mutation mutation) throws DoNotRetryIOException {
4943 if (!(mutation instanceof Put) && !(mutation instanceof Delete) &&
4944 !(mutation instanceof Increment) && !(mutation instanceof Append)) {
4945 throw new org.apache.hadoop.hbase.DoNotRetryIOException(
4946 "Action must be Put or Delete or Increment or Delete");
4950 private void checkRow(final Row action, final byte[] row)
4951 throws DoNotRetryIOException {
4952 if (!Bytes.equals(row, action.getRow())) {
4953 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's getRow must match");
4957 private boolean matches(final CompareOperator op, final int compareResult) {
4958 boolean matches = false;
4959 switch (op) {
4960 case LESS:
4961 matches = compareResult < 0;
4962 break;
4963 case LESS_OR_EQUAL:
4964 matches = compareResult <= 0;
4965 break;
4966 case EQUAL:
4967 matches = compareResult == 0;
4968 break;
4969 case NOT_EQUAL:
4970 matches = compareResult != 0;
4971 break;
4972 case GREATER_OR_EQUAL:
4973 matches = compareResult >= 0;
4974 break;
4975 case GREATER:
4976 matches = compareResult > 0;
4977 break;
4978 default:
4979 throw new RuntimeException("Unknown Compare op " + op.name());
4981 return matches;
4984 private OperationStatus mutate(Mutation mutation) throws IOException {
4985 return mutate(mutation, false);
4988 private OperationStatus mutate(Mutation mutation, boolean atomic) throws IOException {
4989 return mutate(mutation, atomic, HConstants.NO_NONCE, HConstants.NO_NONCE);
4992 private OperationStatus mutate(Mutation mutation, boolean atomic, long nonceGroup, long nonce)
4993 throws IOException {
4994 OperationStatus[] status =
4995 this.batchMutate(new Mutation[] { mutation }, atomic, nonceGroup, nonce);
4996 if (status[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
4997 throw new FailedSanityCheckException(status[0].getExceptionMsg());
4998 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
4999 throw new NoSuchColumnFamilyException(status[0].getExceptionMsg());
5000 } else if (status[0].getOperationStatusCode().equals(OperationStatusCode.STORE_TOO_BUSY)) {
5001 throw new RegionTooBusyException(status[0].getExceptionMsg());
5003 return status[0];
5007 * Complete taking the snapshot on the region. Writes the region info and adds references to the
5008 * working snapshot directory.
5010 * TODO for api consistency, consider adding another version with no {@link ForeignExceptionSnare}
5011 * arg. (In the future other cancellable HRegion methods could eventually add a
5012 * {@link ForeignExceptionSnare}, or we could do something fancier).
5014 * @param desc snapshot description object
5015 * @param exnSnare ForeignExceptionSnare that captures external exceptions in case we need to
5016 * bail out. This is allowed to be null and will just be ignored in that case.
5017 * @throws IOException if there is an external or internal error causing the snapshot to fail
5019 public void addRegionToSnapshot(SnapshotDescription desc,
5020 ForeignExceptionSnare exnSnare) throws IOException {
5021 Path rootDir = CommonFSUtils.getRootDir(conf);
5022 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
5024 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
5025 snapshotDir, desc, exnSnare);
5026 manifest.addRegion(this);
5029 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId)
5030 throws IOException {
5031 for (List<Cell> cells: cellItr) {
5032 if (cells == null) return;
5033 for (Cell cell : cells) {
5034 PrivateCellUtil.setSequenceId(cell, sequenceId);
5040 * Replace any cell timestamps set to {@link org.apache.hadoop.hbase.HConstants#LATEST_TIMESTAMP}
5041 * provided current timestamp.
5042 * @param cellItr
5043 * @param now
5045 private static void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
5046 throws IOException {
5047 for (List<Cell> cells: cellItr) {
5048 if (cells == null) continue;
5049 // Optimization: 'foreach' loop is not used. See:
5050 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5051 assert cells instanceof RandomAccess;
5052 int listSize = cells.size();
5053 for (int i = 0; i < listSize; i++) {
5054 PrivateCellUtil.updateLatestStamp(cells.get(i), now);
5060 * Possibly rewrite incoming cell tags.
5062 private void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
5063 // Check if we have any work to do and early out otherwise
5064 // Update these checks as more logic is added here
5065 if (m.getTTL() == Long.MAX_VALUE) {
5066 return;
5069 // From this point we know we have some work to do
5070 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
5071 List<Cell> cells = e.getValue();
5072 assert cells instanceof RandomAccess;
5073 int listSize = cells.size();
5074 for (int i = 0; i < listSize; i++) {
5075 Cell cell = cells.get(i);
5076 List<Tag> newTags = TagUtil.carryForwardTags(null, cell);
5077 newTags = TagUtil.carryForwardTTLTag(newTags, m.getTTL());
5078 // Rewrite the cell with the updated set of tags
5079 cells.set(i, PrivateCellUtil.createCell(cell, newTags));
5085 * Check if resources to support an update.
5086 * <p/>
5087 * We throw RegionTooBusyException if above memstore limit and expect client to retry using some
5088 * kind of backoff
5090 private void checkResources() throws RegionTooBusyException {
5091 // If catalog region, do not impose resource constraints or block updates.
5092 if (this.getRegionInfo().isMetaRegion()) {
5093 return;
5096 MemStoreSize mss = this.memStoreSizing.getMemStoreSize();
5097 if (mss.getHeapSize() + mss.getOffHeapSize() > this.blockingMemStoreSize) {
5098 blockedRequestsCount.increment();
5099 requestFlush();
5100 // Don't print current limit because it will vary too much. The message is used as a key
5101 // over in RetriesExhaustedWithDetailsException processing.
5102 final String regionName =
5103 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getEncodedName();
5104 final String serverName = this.getRegionServerServices() == null ?
5105 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
5106 this.getRegionServerServices().getServerName().toString());
5107 RegionTooBusyException rtbe = new RegionTooBusyException(
5108 "Over memstore limit=" + org.apache.hadoop.hbase.procedure2.util.StringUtils
5109 .humanSize(this.blockingMemStoreSize) + ", regionName=" + regionName + ", server="
5110 + serverName);
5111 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
5112 throw rtbe;
5117 * @throws IOException Throws exception if region is in read-only mode.
5119 private void checkReadOnly() throws IOException {
5120 if (isReadOnly()) {
5121 throw new DoNotRetryIOException("region is read only");
5125 private void checkReadsEnabled() throws IOException {
5126 if (!this.writestate.readsEnabled) {
5127 throw new IOException(getRegionInfo().getEncodedName()
5128 + ": The region's reads are disabled. Cannot serve the request");
5132 public void setReadsEnabled(boolean readsEnabled) {
5133 if (readsEnabled && !this.writestate.readsEnabled) {
5134 LOG.info("Enabling reads for {}", getRegionInfo().getEncodedName());
5136 this.writestate.setReadsEnabled(readsEnabled);
5140 * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be
5141 * set; when set we will run operations that make sense in the increment/append scenario
5142 * but that do not make sense otherwise.
5144 private void applyToMemStore(HStore store, List<Cell> cells, boolean delta,
5145 MemStoreSizing memstoreAccounting) throws IOException {
5146 // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!!
5147 boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1;
5148 if (upsert) {
5149 store.upsert(cells, getSmallestReadPoint(), memstoreAccounting);
5150 } else {
5151 store.add(cells, memstoreAccounting);
5155 private void checkFamilies(Collection<byte[]> families, Durability durability)
5156 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5157 for (byte[] family : families) {
5158 checkFamily(family, durability);
5162 private void checkFamily(final byte[] family, Durability durability)
5163 throws NoSuchColumnFamilyException, InvalidMutationDurabilityException {
5164 checkFamily(family);
5165 if (durability.equals(Durability.SKIP_WAL)
5166 && htableDescriptor.getColumnFamily(family).getScope()
5167 != HConstants.REPLICATION_SCOPE_LOCAL) {
5168 throw new InvalidMutationDurabilityException(
5169 "Mutation's durability is SKIP_WAL but table's column family " + Bytes.toString(family)
5170 + " need replication");
5174 private void checkFamily(final byte[] family) throws NoSuchColumnFamilyException {
5175 if (!this.htableDescriptor.hasColumnFamily(family)) {
5176 throw new NoSuchColumnFamilyException(
5177 "Column family " + Bytes.toString(family) + " does not exist in region " + this
5178 + " in table " + this.htableDescriptor);
5183 * Check the collection of families for valid timestamps
5184 * @param familyMap
5185 * @param now current timestamp
5186 * @throws FailedSanityCheckException
5188 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
5189 throws FailedSanityCheckException {
5190 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
5191 return;
5193 long maxTs = now + timestampSlop;
5194 for (List<Cell> kvs : familyMap.values()) {
5195 // Optimization: 'foreach' loop is not used. See:
5196 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
5197 assert kvs instanceof RandomAccess;
5198 int listSize = kvs.size();
5199 for (int i=0; i < listSize; i++) {
5200 Cell cell = kvs.get(i);
5201 // see if the user-side TS is out of range. latest = server-side
5202 long ts = cell.getTimestamp();
5203 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
5204 throw new FailedSanityCheckException("Timestamp for KV out of range "
5205 + cell + " (too.new=" + timestampSlop + ")");
5212 * @param size
5213 * @return True if size is over the flush threshold
5215 private boolean isFlushSize(MemStoreSize size) {
5216 return size.getHeapSize() + size.getOffHeapSize() > getMemStoreFlushSize();
5219 private void deleteRecoveredEdits(FileSystem fs, Iterable<Path> files) throws IOException {
5220 for (Path file : files) {
5221 if (!fs.delete(file, false)) {
5222 LOG.error("Failed delete of {}", file);
5223 } else {
5224 LOG.debug("Deleted recovered.edits file={}", file);
5230 * Read the edits put under this region by wal splitting process. Put
5231 * the recovered edits back up into this region.
5233 * <p>We can ignore any wal message that has a sequence ID that's equal to or
5234 * lower than minSeqId. (Because we know such messages are already
5235 * reflected in the HFiles.)
5237 * <p>While this is running we are putting pressure on memory yet we are
5238 * outside of our usual accounting because we are not yet an onlined region
5239 * (this stuff is being run as part of Region initialization). This means
5240 * that if we're up against global memory limits, we'll not be flagged to flush
5241 * because we are not online. We can't be flushed by usual mechanisms anyways;
5242 * we're not yet online so our relative sequenceids are not yet aligned with
5243 * WAL sequenceids -- not till we come up online, post processing of split
5244 * edits.
5246 * <p>But to help relieve memory pressure, at least manage our own heap size
5247 * flushing if are in excess of per-region limits. Flushing, though, we have
5248 * to be careful and avoid using the regionserver/wal sequenceid. Its running
5249 * on a different line to whats going on in here in this region context so if we
5250 * crashed replaying these edits, but in the midst had a flush that used the
5251 * regionserver wal with a sequenceid in excess of whats going on in here
5252 * in this region and with its split editlogs, then we could miss edits the
5253 * next time we go to recover. So, we have to flush inline, using seqids that
5254 * make sense in a this single region context only -- until we online.
5256 * @param maxSeqIdInStores Any edit found in split editlogs needs to be in excess of
5257 * the maxSeqId for the store to be applied, else its skipped.
5258 * @return the sequence id of the last edit added to this region out of the
5259 * recovered edits log or <code>minSeqId</code> if nothing added from editlogs.
5261 long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
5262 final CancelableProgressable reporter, final MonitoredTask status) throws IOException {
5263 long minSeqIdForTheRegion = -1;
5264 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
5265 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
5266 minSeqIdForTheRegion = maxSeqIdInStore;
5269 long seqId = minSeqIdForTheRegion;
5270 String specialRecoveredEditsDirStr = conf.get(SPECIAL_RECOVERED_EDITS_DIR);
5271 if (org.apache.commons.lang3.StringUtils.isBlank(specialRecoveredEditsDirStr)) {
5272 FileSystem walFS = getWalFileSystem();
5273 FileSystem rootFS = getFilesystem();
5274 Path wrongRegionWALDir = CommonFSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
5275 getRegionInfo().getEncodedName());
5276 Path regionWALDir = getWALRegionDir();
5277 Path regionDir =
5278 FSUtils.getRegionDirFromRootDir(CommonFSUtils.getRootDir(conf), getRegionInfo());
5280 // We made a mistake in HBASE-20734 so we need to do this dirty hack...
5281 NavigableSet<Path> filesUnderWrongRegionWALDir =
5282 WALSplitUtil.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
5283 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
5284 filesUnderWrongRegionWALDir, reporter, regionDir));
5285 // This is to ensure backwards compatability with HBASE-20723 where recovered edits can appear
5286 // under the root dir even if walDir is set.
5287 NavigableSet<Path> filesUnderRootDir = Collections.emptyNavigableSet();
5288 if (!regionWALDir.equals(regionDir)) {
5289 filesUnderRootDir = WALSplitUtil.getSplitEditFilesSorted(rootFS, regionDir);
5290 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
5291 filesUnderRootDir, reporter, regionDir));
5294 NavigableSet<Path> files = WALSplitUtil.getSplitEditFilesSorted(walFS, regionWALDir);
5295 seqId = Math.max(seqId,
5296 replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS, files, reporter, regionWALDir));
5297 if (seqId > minSeqIdForTheRegion) {
5298 // Then we added some edits to memory. Flush and cleanup split edit files.
5299 internalFlushcache(null, seqId, stores.values(), status, false,
5300 FlushLifeCycleTracker.DUMMY);
5302 // Now delete the content of recovered edits. We're done w/ them.
5303 if (files.size() > 0 && this.conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
5304 // For debugging data loss issues!
5305 // If this flag is set, make use of the hfile archiving by making recovered.edits a fake
5306 // column family. Have to fake out file type too by casting our recovered.edits as
5307 // storefiles
5308 String fakeFamilyName = WALSplitUtil.getRegionDirRecoveredEditsDir(regionWALDir).getName();
5309 Set<HStoreFile> fakeStoreFiles = new HashSet<>(files.size());
5310 for (Path file : files) {
5311 fakeStoreFiles.add(new HStoreFile(walFS, file, this.conf, null, null, true));
5313 getRegionWALFileSystem().archiveRecoveredEdits(fakeFamilyName, fakeStoreFiles);
5314 } else {
5315 deleteRecoveredEdits(walFS, Iterables.concat(files, filesUnderWrongRegionWALDir));
5316 deleteRecoveredEdits(rootFS, filesUnderRootDir);
5318 } else {
5319 Path recoveredEditsDir = new Path(specialRecoveredEditsDirStr);
5320 FileSystem fs = recoveredEditsDir.getFileSystem(conf);
5321 FileStatus[] files = fs.listStatus(recoveredEditsDir);
5322 LOG.debug("Found {} recovered edits file(s) under {}", files == null ? 0 : files.length,
5323 recoveredEditsDir);
5324 if (files != null) {
5325 for (FileStatus file : files) {
5326 // it is safe to trust the zero-length in this case because we've been through rename and
5327 // lease recovery in the above.
5328 if (isZeroLengthThenDelete(fs, file, file.getPath())) {
5329 continue;
5331 seqId =
5332 Math.max(seqId, replayRecoveredEdits(file.getPath(), maxSeqIdInStores, reporter, fs));
5335 if (seqId > minSeqIdForTheRegion) {
5336 // Then we added some edits to memory. Flush and cleanup split edit files.
5337 internalFlushcache(null, seqId, stores.values(), status, false,
5338 FlushLifeCycleTracker.DUMMY);
5340 deleteRecoveredEdits(fs,
5341 Stream.of(files).map(FileStatus::getPath).collect(Collectors.toList()));
5344 return seqId;
5347 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
5348 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
5349 throws IOException {
5350 long seqid = minSeqIdForTheRegion;
5351 if (LOG.isDebugEnabled()) {
5352 LOG.debug("Found " + (files == null ? 0 : files.size())
5353 + " recovered edits file(s) under " + regionDir);
5356 if (files == null || files.isEmpty()) {
5357 return minSeqIdForTheRegion;
5360 for (Path edits: files) {
5361 if (edits == null || !fs.exists(edits)) {
5362 LOG.warn("Null or non-existent edits file: " + edits);
5363 continue;
5365 if (isZeroLengthThenDelete(fs, fs.getFileStatus(edits), edits)) {
5366 continue;
5369 long maxSeqId;
5370 String fileName = edits.getName();
5371 maxSeqId = Math.abs(Long.parseLong(fileName));
5372 if (maxSeqId <= minSeqIdForTheRegion) {
5373 if (LOG.isDebugEnabled()) {
5374 String msg = "Maximum sequenceid for this wal is " + maxSeqId
5375 + " and minimum sequenceid for the region " + this + " is " + minSeqIdForTheRegion
5376 + ", skipped the whole file, path=" + edits;
5377 LOG.debug(msg);
5379 continue;
5382 try {
5383 // replay the edits. Replay can return -1 if everything is skipped, only update
5384 // if seqId is greater
5385 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
5386 } catch (IOException e) {
5387 handleException(fs, edits, e);
5390 return seqid;
5393 private void handleException(FileSystem fs, Path edits, IOException e) throws IOException {
5394 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
5395 conf.getBoolean("hbase.skip.errors", HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
5396 if (conf.get("hbase.skip.errors") != null) {
5397 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use "
5398 + HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
5400 if (skipErrors) {
5401 Path p = WALSplitUtil.moveAsideBadEditsFile(fs, edits);
5402 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + "=true so continuing. Renamed "
5403 + edits + " as " + p,
5405 } else {
5406 throw e;
5411 * @param edits File of recovered edits.
5412 * @param maxSeqIdInStores Maximum sequenceid found in each store. Edits in wal must be larger
5413 * than this to be replayed for each store.
5414 * @return the sequence id of the last edit added to this region out of the recovered edits log or
5415 * <code>minSeqId</code> if nothing added from editlogs.
5417 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
5418 final CancelableProgressable reporter, FileSystem fs) throws IOException {
5419 String msg = "Replaying edits from " + edits;
5420 LOG.info(msg);
5421 MonitoredTask status = TaskMonitor.get().createStatus(msg);
5423 status.setStatus("Opening recovered edits");
5424 WAL.Reader reader = null;
5425 try {
5426 reader = WALFactory.createReader(fs, edits, conf);
5427 long currentEditSeqId = -1;
5428 long currentReplaySeqId = -1;
5429 long firstSeqIdInLog = -1;
5430 long skippedEdits = 0;
5431 long editsCount = 0;
5432 long intervalEdits = 0;
5433 WAL.Entry entry;
5434 HStore store = null;
5435 boolean reported_once = false;
5436 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
5438 try {
5439 // How many edits seen before we check elapsed time
5440 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
5441 // How often to send a progress report (default 1/2 master timeout)
5442 int period = this.conf.getInt("hbase.hstore.report.period", 300000);
5443 long lastReport = EnvironmentEdgeManager.currentTime();
5445 if (coprocessorHost != null) {
5446 coprocessorHost.preReplayWALs(this.getRegionInfo(), edits);
5449 while ((entry = reader.next()) != null) {
5450 WALKey key = entry.getKey();
5451 WALEdit val = entry.getEdit();
5453 if (ng != null) { // some test, or nonces disabled
5454 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
5457 if (reporter != null) {
5458 intervalEdits += val.size();
5459 if (intervalEdits >= interval) {
5460 // Number of edits interval reached
5461 intervalEdits = 0;
5462 long cur = EnvironmentEdgeManager.currentTime();
5463 if (lastReport + period <= cur) {
5464 status.setStatus("Replaying edits..." +
5465 " skipped=" + skippedEdits +
5466 " edits=" + editsCount);
5467 // Timeout reached
5468 if(!reporter.progress()) {
5469 msg = "Progressable reporter failed, stopping replay for region " + this;
5470 LOG.warn(msg);
5471 status.abort(msg);
5472 throw new IOException(msg);
5474 reported_once = true;
5475 lastReport = cur;
5480 if (firstSeqIdInLog == -1) {
5481 firstSeqIdInLog = key.getSequenceId();
5483 if (currentEditSeqId > key.getSequenceId()) {
5484 // when this condition is true, it means we have a serious defect because we need to
5485 // maintain increasing SeqId for WAL edits per region
5486 LOG.error(getRegionInfo().getEncodedName() + " : "
5487 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
5488 + "; edit=" + val);
5489 } else {
5490 currentEditSeqId = key.getSequenceId();
5492 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
5493 key.getOrigLogSeqNum() : currentEditSeqId;
5495 // Start coprocessor replay here. The coprocessor is for each WALEdit
5496 // instead of a KeyValue.
5497 if (coprocessorHost != null) {
5498 status.setStatus("Running pre-WAL-restore hook in coprocessors");
5499 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
5500 // if bypass this wal entry, ignore it ...
5501 continue;
5504 boolean checkRowWithinBoundary = false;
5505 // Check this edit is for this region.
5506 if (!Bytes.equals(key.getEncodedRegionName(),
5507 this.getRegionInfo().getEncodedNameAsBytes())) {
5508 checkRowWithinBoundary = true;
5511 boolean flush = false;
5512 MemStoreSizing memStoreSizing = new NonThreadSafeMemStoreSizing();
5513 for (Cell cell: val.getCells()) {
5514 // Check this edit is for me. Also, guard against writing the special
5515 // METACOLUMN info such as HBASE::CACHEFLUSH entries
5516 if (WALEdit.isMetaEditFamily(cell)) {
5517 // if region names don't match, skipp replaying compaction marker
5518 if (!checkRowWithinBoundary) {
5519 //this is a special edit, we should handle it
5520 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
5521 if (compaction != null) {
5522 //replay the compaction
5523 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
5526 skippedEdits++;
5527 continue;
5529 // Figure which store the edit is meant for.
5530 if (store == null || !CellUtil.matchingFamily(cell,
5531 store.getColumnFamilyDescriptor().getName())) {
5532 store = getStore(cell);
5534 if (store == null) {
5535 // This should never happen. Perhaps schema was changed between
5536 // crash and redeploy?
5537 LOG.warn("No family for cell {} in region {}", cell, this);
5538 skippedEdits++;
5539 continue;
5541 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
5542 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
5543 LOG.warn("Row of {} is not within region boundary for region {}", cell, this);
5544 skippedEdits++;
5545 continue;
5547 // Now, figure if we should skip this edit.
5548 if (key.getSequenceId() <= maxSeqIdInStores.get(store.getColumnFamilyDescriptor()
5549 .getName())) {
5550 skippedEdits++;
5551 continue;
5553 PrivateCellUtil.setSequenceId(cell, currentReplaySeqId);
5555 restoreEdit(store, cell, memStoreSizing);
5556 editsCount++;
5558 MemStoreSize mss = memStoreSizing.getMemStoreSize();
5559 incMemStoreSize(mss);
5560 flush = isFlushSize(this.memStoreSizing.getMemStoreSize());
5561 if (flush) {
5562 internalFlushcache(null, currentEditSeqId, stores.values(), status, false,
5563 FlushLifeCycleTracker.DUMMY);
5566 if (coprocessorHost != null) {
5567 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
5571 if (coprocessorHost != null) {
5572 coprocessorHost.postReplayWALs(this.getRegionInfo(), edits);
5574 } catch (EOFException eof) {
5575 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5576 msg = "EnLongAddered EOF. Most likely due to Master failure during "
5577 + "wal splitting, so we have this data in another edit. Continuing, but renaming "
5578 + edits + " as " + p + " for region " + this;
5579 LOG.warn(msg, eof);
5580 status.abort(msg);
5581 } catch (IOException ioe) {
5582 // If the IOE resulted from bad file format,
5583 // then this problem is idempotent and retrying won't help
5584 if (ioe.getCause() instanceof ParseException) {
5585 Path p = WALSplitUtil.moveAsideBadEditsFile(walFS, edits);
5586 msg = "File corruption enLongAddered! " +
5587 "Continuing, but renaming " + edits + " as " + p;
5588 LOG.warn(msg, ioe);
5589 status.setStatus(msg);
5590 } else {
5591 status.abort(StringUtils.stringifyException(ioe));
5592 // other IO errors may be transient (bad network connection,
5593 // checksum exception on one datanode, etc). throw & retry
5594 throw ioe;
5597 if (reporter != null && !reported_once) {
5598 reporter.progress();
5600 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
5601 ", firstSequenceIdInLog=" + firstSeqIdInLog +
5602 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
5603 status.markComplete(msg);
5604 LOG.debug(msg);
5605 return currentEditSeqId;
5606 } finally {
5607 status.cleanup();
5608 if (reader != null) {
5609 reader.close();
5615 * Call to complete a compaction. Its for the case where we find in the WAL a compaction
5616 * that was not finished. We could find one recovering a WAL after a regionserver crash.
5617 * See HBASE-2331.
5619 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
5620 boolean removeFiles, long replaySeqId)
5621 throws IOException {
5622 try {
5623 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
5624 "Compaction marker from WAL ", compaction);
5625 } catch (WrongRegionException wre) {
5626 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5627 // skip the compaction marker since it is not for this region
5628 return;
5630 throw wre;
5633 synchronized (writestate) {
5634 if (replaySeqId < lastReplayedOpenRegionSeqId) {
5635 LOG.warn(getRegionInfo().getEncodedName() + " : "
5636 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5637 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5638 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
5639 return;
5641 if (replaySeqId < lastReplayedCompactionSeqId) {
5642 LOG.warn(getRegionInfo().getEncodedName() + " : "
5643 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
5644 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5645 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
5646 return;
5647 } else {
5648 lastReplayedCompactionSeqId = replaySeqId;
5651 if (LOG.isDebugEnabled()) {
5652 LOG.debug(getRegionInfo().getEncodedName() + " : "
5653 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
5654 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
5655 + lastReplayedOpenRegionSeqId);
5658 startRegionOperation(Operation.REPLAY_EVENT);
5659 try {
5660 HStore store = this.getStore(compaction.getFamilyName().toByteArray());
5661 if (store == null) {
5662 LOG.warn(getRegionInfo().getEncodedName() + " : "
5663 + "Found Compaction WAL edit for deleted family:"
5664 + Bytes.toString(compaction.getFamilyName().toByteArray()));
5665 return;
5667 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
5668 logRegionFiles();
5669 } catch (FileNotFoundException ex) {
5670 LOG.warn(getRegionInfo().getEncodedName() + " : "
5671 + "At least one of the store files in compaction: "
5672 + TextFormat.shortDebugString(compaction)
5673 + " doesn't exist any more. Skip loading the file(s)", ex);
5674 } finally {
5675 closeRegionOperation(Operation.REPLAY_EVENT);
5680 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
5681 checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
5682 "Flush marker from WAL ", flush);
5684 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5685 return; // if primary nothing to do
5688 if (LOG.isDebugEnabled()) {
5689 LOG.debug(getRegionInfo().getEncodedName() + " : "
5690 + "Replaying flush marker " + TextFormat.shortDebugString(flush));
5693 startRegionOperation(Operation.REPLAY_EVENT); // use region close lock to guard against close
5694 try {
5695 FlushAction action = flush.getAction();
5696 switch (action) {
5697 case START_FLUSH:
5698 replayWALFlushStartMarker(flush);
5699 break;
5700 case COMMIT_FLUSH:
5701 replayWALFlushCommitMarker(flush);
5702 break;
5703 case ABORT_FLUSH:
5704 replayWALFlushAbortMarker(flush);
5705 break;
5706 case CANNOT_FLUSH:
5707 replayWALFlushCannotFlushMarker(flush, replaySeqId);
5708 break;
5709 default:
5710 LOG.warn(getRegionInfo().getEncodedName() + " : " +
5711 "Received a flush event with unknown action, ignoring. " +
5712 TextFormat.shortDebugString(flush));
5713 break;
5716 logRegionFiles();
5717 } finally {
5718 closeRegionOperation(Operation.REPLAY_EVENT);
5722 /** Replay the flush marker from primary region by creating a corresponding snapshot of
5723 * the store memstores, only if the memstores do not have a higher seqId from an earlier wal
5724 * edit (because the events may be coming out of order).
5726 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
5727 long flushSeqId = flush.getFlushSequenceNumber();
5729 HashSet<HStore> storesToFlush = new HashSet<>();
5730 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
5731 byte[] family = storeFlush.getFamilyName().toByteArray();
5732 HStore store = getStore(family);
5733 if (store == null) {
5734 LOG.warn(getRegionInfo().getEncodedName() + " : "
5735 + "Received a flush start marker from primary, but the family is not found. Ignoring"
5736 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
5737 continue;
5739 storesToFlush.add(store);
5742 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
5744 // we will use writestate as a coarse-grain lock for all the replay events
5745 // (flush, compaction, region open etc)
5746 synchronized (writestate) {
5747 try {
5748 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5749 LOG.warn(getRegionInfo().getEncodedName() + " : "
5750 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5751 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5752 + " of " + lastReplayedOpenRegionSeqId);
5753 return null;
5755 if (numMutationsWithoutWAL.sum() > 0) {
5756 numMutationsWithoutWAL.reset();
5757 dataInMemoryWithoutWAL.reset();
5760 if (!writestate.flushing) {
5761 // we do not have an active snapshot and corresponding this.prepareResult. This means
5762 // we can just snapshot our memstores and continue as normal.
5764 // invoke prepareFlushCache. Send null as wal since we do not want the flush events in wal
5765 PrepareFlushResult prepareResult = internalPrepareFlushCache(null, flushSeqId,
5766 storesToFlush, status, false, FlushLifeCycleTracker.DUMMY);
5767 if (prepareResult.result == null) {
5768 // save the PrepareFlushResult so that we can use it later from commit flush
5769 this.writestate.flushing = true;
5770 this.prepareFlushResult = prepareResult;
5771 status.markComplete("Flush prepare successful");
5772 if (LOG.isDebugEnabled()) {
5773 LOG.debug(getRegionInfo().getEncodedName() + " : "
5774 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
5776 } else {
5777 // special case empty memstore. We will still save the flush result in this case, since
5778 // our memstore ie empty, but the primary is still flushing
5779 if (prepareResult.getResult().getResult() ==
5780 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
5781 this.writestate.flushing = true;
5782 this.prepareFlushResult = prepareResult;
5783 if (LOG.isDebugEnabled()) {
5784 LOG.debug(getRegionInfo().getEncodedName() + " : "
5785 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
5788 status.abort("Flush prepare failed with " + prepareResult.result);
5789 // nothing much to do. prepare flush failed because of some reason.
5791 return prepareResult;
5792 } else {
5793 // we already have an active snapshot.
5794 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
5795 // They define the same flush. Log and continue.
5796 LOG.warn(getRegionInfo().getEncodedName() + " : "
5797 + "Received a flush prepare marker with the same seqId: " +
5798 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5799 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5800 // ignore
5801 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
5802 // We received a flush with a smaller seqNum than what we have prepared. We can only
5803 // ignore this prepare flush request.
5804 LOG.warn(getRegionInfo().getEncodedName() + " : "
5805 + "Received a flush prepare marker with a smaller seqId: " +
5806 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5807 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5808 // ignore
5809 } else {
5810 // We received a flush with a larger seqNum than what we have prepared
5811 LOG.warn(getRegionInfo().getEncodedName() + " : "
5812 + "Received a flush prepare marker with a larger seqId: " +
5813 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
5814 + prepareFlushResult.flushOpSeqId + ". Ignoring");
5815 // We do not have multiple active snapshots in the memstore or a way to merge current
5816 // memstore snapshot with the contents and resnapshot for now. We cannot take
5817 // another snapshot and drop the previous one because that will cause temporary
5818 // data loss in the secondary. So we ignore this for now, deferring the resolution
5819 // to happen when we see the corresponding flush commit marker. If we have a memstore
5820 // snapshot with x, and later received another prepare snapshot with y (where x < y),
5821 // when we see flush commit for y, we will drop snapshot for x, and can also drop all
5822 // the memstore edits if everything in memstore is < y. This is the usual case for
5823 // RS crash + recovery where we might see consequtive prepare flush wal markers.
5824 // Otherwise, this will cause more memory to be used in secondary replica until a
5825 // further prapare + commit flush is seen and replayed.
5828 } finally {
5829 status.cleanup();
5830 writestate.notifyAll();
5833 return null;
5836 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
5837 justification="Intentional; post memstore flush")
5838 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
5839 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
5841 // check whether we have the memstore snapshot with the corresponding seqId. Replay to
5842 // secondary region replicas are in order, except for when the region moves or then the
5843 // region server crashes. In those cases, we may receive replay requests out of order from
5844 // the original seqIds.
5845 synchronized (writestate) {
5846 try {
5847 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5848 LOG.warn(getRegionInfo().getEncodedName() + " : "
5849 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5850 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5851 + " of " + lastReplayedOpenRegionSeqId);
5852 return;
5855 if (writestate.flushing) {
5856 PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
5857 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
5858 if (LOG.isDebugEnabled()) {
5859 LOG.debug(getRegionInfo().getEncodedName() + " : "
5860 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5861 + " and a previous prepared snapshot was found");
5863 // This is the regular case where we received commit flush after prepare flush
5864 // corresponding to the same seqId.
5865 replayFlushInStores(flush, prepareFlushResult, true);
5867 // Set down the memstore size by amount of flush.
5868 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5869 this.prepareFlushResult = null;
5870 writestate.flushing = false;
5871 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
5872 // This should not happen normally. However, lets be safe and guard against these cases
5873 // we received a flush commit with a smaller seqId than what we have prepared
5874 // we will pick the flush file up from this commit (if we have not seen it), but we
5875 // will not drop the memstore
5876 LOG.warn(getRegionInfo().getEncodedName() + " : "
5877 + "Received a flush commit marker with smaller seqId: "
5878 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
5879 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
5880 +" prepared memstore snapshot");
5881 replayFlushInStores(flush, prepareFlushResult, false);
5883 // snapshot is not dropped, so memstore sizes should not be decremented
5884 // we still have the prepared snapshot, flushing should still be true
5885 } else {
5886 // This should not happen normally. However, lets be safe and guard against these cases
5887 // we received a flush commit with a larger seqId than what we have prepared
5888 // we will pick the flush file for this. We will also obtain the updates lock and
5889 // look for contents of the memstore to see whether we have edits after this seqId.
5890 // If not, we will drop all the memstore edits and the snapshot as well.
5891 LOG.warn(getRegionInfo().getEncodedName() + " : "
5892 + "Received a flush commit marker with larger seqId: "
5893 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
5894 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
5895 +" memstore snapshot");
5897 replayFlushInStores(flush, prepareFlushResult, true);
5899 // Set down the memstore size by amount of flush.
5900 this.decrMemStoreSize(prepareFlushResult.totalFlushableSize.getMemStoreSize());
5902 // Inspect the memstore contents to see whether the memstore contains only edits
5903 // with seqId smaller than the flush seqId. If so, we can discard those edits.
5904 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5906 this.prepareFlushResult = null;
5907 writestate.flushing = false;
5909 // If we were waiting for observing a flush or region opening event for not showing
5910 // partial data after a secondary region crash, we can allow reads now. We can only make
5911 // sure that we are not showing partial data (for example skipping some previous edits)
5912 // until we observe a full flush start and flush commit. So if we were not able to find
5913 // a previous flush we will not enable reads now.
5914 this.setReadsEnabled(true);
5915 } else {
5916 LOG.warn(getRegionInfo().getEncodedName() + " : "
5917 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5918 + ", but no previous prepared snapshot was found");
5919 // There is no corresponding prepare snapshot from before.
5920 // We will pick up the new flushed file
5921 replayFlushInStores(flush, null, false);
5923 // Inspect the memstore contents to see whether the memstore contains only edits
5924 // with seqId smaller than the flush seqId. If so, we can discard those edits.
5925 dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5928 status.markComplete("Flush commit successful");
5930 // Update the last flushed sequence id for region.
5931 this.maxFlushedSeqId = flush.getFlushSequenceNumber();
5933 // advance the mvcc read point so that the new flushed file is visible.
5934 mvcc.advanceTo(flush.getFlushSequenceNumber());
5936 } catch (FileNotFoundException ex) {
5937 LOG.warn(getRegionInfo().getEncodedName() + " : "
5938 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
5939 + " doesn't exist any more. Skip loading the file(s)", ex);
5941 finally {
5942 status.cleanup();
5943 writestate.notifyAll();
5947 // C. Finally notify anyone waiting on memstore to clear:
5948 // e.g. checkResources().
5949 synchronized (this) {
5950 notifyAll(); // FindBugs NN_NAKED_NOTIFY
5955 * Replays the given flush descriptor by opening the flush files in stores and dropping the
5956 * memstore snapshots if requested.
5957 * @param flush
5958 * @param prepareFlushResult
5959 * @param dropMemstoreSnapshot
5960 * @throws IOException
5962 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
5963 boolean dropMemstoreSnapshot)
5964 throws IOException {
5965 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
5966 byte[] family = storeFlush.getFamilyName().toByteArray();
5967 HStore store = getStore(family);
5968 if (store == null) {
5969 LOG.warn(getRegionInfo().getEncodedName() + " : "
5970 + "Received a flush commit marker from primary, but the family is not found."
5971 + "Ignoring StoreFlushDescriptor:" + storeFlush);
5972 continue;
5974 List<String> flushFiles = storeFlush.getFlushOutputList();
5975 StoreFlushContext ctx = null;
5976 long startTime = EnvironmentEdgeManager.currentTime();
5977 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
5978 ctx = store.createFlushContext(flush.getFlushSequenceNumber(), FlushLifeCycleTracker.DUMMY);
5979 } else {
5980 ctx = prepareFlushResult.storeFlushCtxs.get(family);
5981 startTime = prepareFlushResult.startTime;
5984 if (ctx == null) {
5985 LOG.warn(getRegionInfo().getEncodedName() + " : "
5986 + "Unexpected: flush commit marker received from store "
5987 + Bytes.toString(family) + " but no associated flush context. Ignoring");
5988 continue;
5991 ctx.replayFlush(flushFiles, dropMemstoreSnapshot); // replay the flush
5993 // Record latest flush time
5994 this.lastStoreFlushTimeMap.put(store, startTime);
5998 private long loadRecoveredHFilesIfAny(Collection<HStore> stores) throws IOException {
5999 Path regionDir = fs.getRegionDir();
6000 long maxSeqId = -1;
6001 for (HStore store : stores) {
6002 String familyName = store.getColumnFamilyName();
6003 FileStatus[] files =
6004 WALSplitUtil.getRecoveredHFiles(fs.getFileSystem(), regionDir, familyName);
6005 if (files != null && files.length != 0) {
6006 for (FileStatus file : files) {
6007 Path filePath = file.getPath();
6008 // If file length is zero then delete it
6009 if (isZeroLengthThenDelete(fs.getFileSystem(), file, filePath)) {
6010 continue;
6012 try {
6013 HStoreFile storefile = store.tryCommitRecoveredHFile(file.getPath());
6014 maxSeqId = Math.max(maxSeqId, storefile.getReader().getSequenceID());
6015 } catch (IOException e) {
6016 handleException(fs.getFileSystem(), filePath, e);
6017 continue;
6020 if (this.rsServices != null && store.needsCompaction()) {
6021 this.rsServices.getCompactionRequestor()
6022 .requestCompaction(this, store, "load recovered hfiles request compaction",
6023 Store.PRIORITY_USER + 1, CompactionLifeCycleTracker.DUMMY, null);
6027 return maxSeqId;
6031 * Be careful, this method will drop all data in the memstore of this region.
6032 * Currently, this method is used to drop memstore to prevent memory leak
6033 * when replaying recovered.edits while opening region.
6035 private MemStoreSize dropMemStoreContents() throws IOException {
6036 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6037 this.updatesLock.writeLock().lock();
6038 try {
6039 for (HStore s : stores.values()) {
6040 MemStoreSize memStoreSize = doDropStoreMemStoreContentsForSeqId(s, HConstants.NO_SEQNUM);
6041 LOG.info("Drop memstore for Store " + s.getColumnFamilyName() + " in region "
6042 + this.getRegionInfo().getRegionNameAsString()
6043 + " , dropped memstoresize: [" + memStoreSize + " }");
6044 totalFreedSize.incMemStoreSize(memStoreSize);
6046 return totalFreedSize.getMemStoreSize();
6047 } finally {
6048 this.updatesLock.writeLock().unlock();
6053 * Drops the memstore contents after replaying a flush descriptor or region open event replay
6054 * if the memstore edits have seqNums smaller than the given seq id
6055 * @throws IOException
6057 private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException {
6058 MemStoreSizing totalFreedSize = new NonThreadSafeMemStoreSizing();
6059 this.updatesLock.writeLock().lock();
6060 try {
6062 long currentSeqId = mvcc.getReadPoint();
6063 if (seqId >= currentSeqId) {
6064 // then we can drop the memstore contents since everything is below this seqId
6065 LOG.info(getRegionInfo().getEncodedName() + " : "
6066 + "Dropping memstore contents as well since replayed flush seqId: "
6067 + seqId + " is greater than current seqId:" + currentSeqId);
6069 // Prepare flush (take a snapshot) and then abort (drop the snapshot)
6070 if (store == null) {
6071 for (HStore s : stores.values()) {
6072 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId));
6074 } else {
6075 totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId));
6077 } else {
6078 LOG.info(getRegionInfo().getEncodedName() + " : "
6079 + "Not dropping memstore contents since replayed flush seqId: "
6080 + seqId + " is smaller than current seqId:" + currentSeqId);
6082 } finally {
6083 this.updatesLock.writeLock().unlock();
6085 return totalFreedSize.getMemStoreSize();
6088 private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId)
6089 throws IOException {
6090 MemStoreSize flushableSize = s.getFlushableSize();
6091 this.decrMemStoreSize(flushableSize);
6092 StoreFlushContext ctx = s.createFlushContext(currentSeqId, FlushLifeCycleTracker.DUMMY);
6093 ctx.prepare();
6094 ctx.abort();
6095 return flushableSize;
6098 private void replayWALFlushAbortMarker(FlushDescriptor flush) {
6099 // nothing to do for now. A flush abort will cause a RS abort which means that the region
6100 // will be opened somewhere else later. We will see the region open event soon, and replaying
6101 // that will drop the snapshot
6104 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
6105 synchronized (writestate) {
6106 if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
6107 LOG.warn(getRegionInfo().getEncodedName() + " : "
6108 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
6109 + " because its sequence id " + replaySeqId + " is smaller than this regions "
6110 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
6111 return;
6114 // If we were waiting for observing a flush or region opening event for not showing partial
6115 // data after a secondary region crash, we can allow reads now. This event means that the
6116 // primary was not able to flush because memstore is empty when we requested flush. By the
6117 // time we observe this, we are guaranteed to have up to date seqId with our previous
6118 // assignment.
6119 this.setReadsEnabled(true);
6123 PrepareFlushResult getPrepareFlushResult() {
6124 return prepareFlushResult;
6127 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
6128 justification="Intentional; cleared the memstore")
6129 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
6130 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
6131 "RegionEvent marker from WAL ", regionEvent);
6133 startRegionOperation(Operation.REPLAY_EVENT);
6134 try {
6135 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6136 return; // if primary nothing to do
6139 if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
6140 // nothing to do on REGION_CLOSE for now.
6141 return;
6143 if (regionEvent.getEventType() != EventType.REGION_OPEN) {
6144 LOG.warn(getRegionInfo().getEncodedName() + " : "
6145 + "Unknown region event received, ignoring :"
6146 + TextFormat.shortDebugString(regionEvent));
6147 return;
6150 if (LOG.isDebugEnabled()) {
6151 LOG.debug(getRegionInfo().getEncodedName() + " : "
6152 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
6155 // we will use writestate as a coarse-grain lock for all the replay events
6156 synchronized (writestate) {
6157 // Replication can deliver events out of order when primary region moves or the region
6158 // server crashes, since there is no coordination between replication of different wal files
6159 // belonging to different region servers. We have to safe guard against this case by using
6160 // region open event's seqid. Since this is the first event that the region puts (after
6161 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6162 // smaller than this seqId
6163 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
6164 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
6165 } else {
6166 LOG.warn(getRegionInfo().getEncodedName() + " : "
6167 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
6168 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
6169 + " of " + lastReplayedOpenRegionSeqId);
6170 return;
6173 // region open lists all the files that the region has at the time of the opening. Just pick
6174 // all the files and drop prepared flushes and empty memstores
6175 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
6176 // stores of primary may be different now
6177 byte[] family = storeDescriptor.getFamilyName().toByteArray();
6178 HStore store = getStore(family);
6179 if (store == null) {
6180 LOG.warn(getRegionInfo().getEncodedName() + " : "
6181 + "Received a region open marker from primary, but the family is not found. "
6182 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6183 continue;
6186 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6187 List<String> storeFiles = storeDescriptor.getStoreFileList();
6188 try {
6189 store.refreshStoreFiles(storeFiles); // replace the files with the new ones
6190 } catch (FileNotFoundException ex) {
6191 LOG.warn(getRegionInfo().getEncodedName() + " : "
6192 + "At least one of the store files: " + storeFiles
6193 + " doesn't exist any more. Skip loading the file(s)", ex);
6194 continue;
6196 if (store.getMaxSequenceId().orElse(0L) != storeSeqId) {
6197 // Record latest flush time if we picked up new files
6198 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
6201 if (writestate.flushing) {
6202 // only drop memstore snapshots if they are smaller than last flush for the store
6203 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
6204 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6205 null : this.prepareFlushResult.storeFlushCtxs.get(family);
6206 if (ctx != null) {
6207 MemStoreSize mss = store.getFlushableSize();
6208 ctx.abort();
6209 this.decrMemStoreSize(mss);
6210 this.prepareFlushResult.storeFlushCtxs.remove(family);
6215 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6216 dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
6217 if (storeSeqId > this.maxFlushedSeqId) {
6218 this.maxFlushedSeqId = storeSeqId;
6222 // if all stores ended up dropping their snapshots, we can safely drop the
6223 // prepareFlushResult
6224 dropPrepareFlushIfPossible();
6226 // advance the mvcc read point so that the new flushed file is visible.
6227 mvcc.await();
6229 // If we were waiting for observing a flush or region opening event for not showing partial
6230 // data after a secondary region crash, we can allow reads now.
6231 this.setReadsEnabled(true);
6233 // C. Finally notify anyone waiting on memstore to clear:
6234 // e.g. checkResources().
6235 synchronized (this) {
6236 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6239 logRegionFiles();
6240 } finally {
6241 closeRegionOperation(Operation.REPLAY_EVENT);
6245 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
6246 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
6247 "BulkLoad marker from WAL ", bulkLoadEvent);
6249 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6250 return; // if primary nothing to do
6253 if (LOG.isDebugEnabled()) {
6254 LOG.debug(getRegionInfo().getEncodedName() + " : "
6255 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
6257 // check if multiple families involved
6258 boolean multipleFamilies = false;
6259 byte[] family = null;
6260 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6261 byte[] fam = storeDescriptor.getFamilyName().toByteArray();
6262 if (family == null) {
6263 family = fam;
6264 } else if (!Bytes.equals(family, fam)) {
6265 multipleFamilies = true;
6266 break;
6270 startBulkRegionOperation(multipleFamilies);
6271 try {
6272 // we will use writestate as a coarse-grain lock for all the replay events
6273 synchronized (writestate) {
6274 // Replication can deliver events out of order when primary region moves or the region
6275 // server crashes, since there is no coordination between replication of different wal files
6276 // belonging to different region servers. We have to safe guard against this case by using
6277 // region open event's seqid. Since this is the first event that the region puts (after
6278 // possibly flushing recovered.edits), after seeing this event, we can ignore every edit
6279 // smaller than this seqId
6280 if (bulkLoadEvent.getBulkloadSeqNum() >= 0
6281 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
6282 LOG.warn(getRegionInfo().getEncodedName() + " : "
6283 + "Skipping replaying bulkload event :"
6284 + TextFormat.shortDebugString(bulkLoadEvent)
6285 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
6286 + " =" + lastReplayedOpenRegionSeqId);
6288 return;
6291 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
6292 // stores of primary may be different now
6293 family = storeDescriptor.getFamilyName().toByteArray();
6294 HStore store = getStore(family);
6295 if (store == null) {
6296 LOG.warn(getRegionInfo().getEncodedName() + " : "
6297 + "Received a bulk load marker from primary, but the family is not found. "
6298 + "Ignoring. StoreDescriptor:" + storeDescriptor);
6299 continue;
6302 List<String> storeFiles = storeDescriptor.getStoreFileList();
6303 for (String storeFile : storeFiles) {
6304 StoreFileInfo storeFileInfo = null;
6305 try {
6306 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
6307 store.bulkLoadHFile(storeFileInfo);
6308 } catch(FileNotFoundException ex) {
6309 LOG.warn(getRegionInfo().getEncodedName() + " : "
6310 + ((storeFileInfo != null) ? storeFileInfo.toString() :
6311 (new Path(Bytes.toString(family), storeFile)).toString())
6312 + " doesn't exist any more. Skip loading the file");
6317 if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
6318 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
6320 } finally {
6321 closeBulkRegionOperation();
6326 * If all stores ended up dropping their snapshots, we can safely drop the prepareFlushResult
6328 private void dropPrepareFlushIfPossible() {
6329 if (writestate.flushing) {
6330 boolean canDrop = true;
6331 if (prepareFlushResult.storeFlushCtxs != null) {
6332 for (Entry<byte[], StoreFlushContext> entry : prepareFlushResult.storeFlushCtxs
6333 .entrySet()) {
6334 HStore store = getStore(entry.getKey());
6335 if (store == null) {
6336 continue;
6338 if (store.getSnapshotSize().getDataSize() > 0) {
6339 canDrop = false;
6340 break;
6345 // this means that all the stores in the region has finished flushing, but the WAL marker
6346 // may not have been written or we did not receive it yet.
6347 if (canDrop) {
6348 writestate.flushing = false;
6349 this.prepareFlushResult = null;
6354 @Override
6355 public boolean refreshStoreFiles() throws IOException {
6356 return refreshStoreFiles(false);
6359 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
6360 justification = "Notify is about post replay. Intentional")
6361 protected boolean refreshStoreFiles(boolean force) throws IOException {
6362 if (!force && ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
6363 return false; // if primary nothing to do
6366 if (LOG.isDebugEnabled()) {
6367 LOG.debug(getRegionInfo().getEncodedName() + " : "
6368 + "Refreshing store files to see whether we can free up memstore");
6371 long totalFreedDataSize = 0;
6373 long smallestSeqIdInStores = Long.MAX_VALUE;
6375 startRegionOperation(); // obtain region close lock
6376 try {
6377 Map<HStore, Long> map = new HashMap<>();
6378 synchronized (writestate) {
6379 for (HStore store : stores.values()) {
6380 // TODO: some stores might see new data from flush, while others do not which
6381 // MIGHT break atomic edits across column families.
6382 long maxSeqIdBefore = store.getMaxSequenceId().orElse(0L);
6384 // refresh the store files. This is similar to observing a region open wal marker.
6385 store.refreshStoreFiles();
6387 long storeSeqId = store.getMaxSequenceId().orElse(0L);
6388 if (storeSeqId < smallestSeqIdInStores) {
6389 smallestSeqIdInStores = storeSeqId;
6392 // see whether we can drop the memstore or the snapshot
6393 if (storeSeqId > maxSeqIdBefore) {
6394 if (writestate.flushing) {
6395 // only drop memstore snapshots if they are smaller than last flush for the store
6396 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
6397 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
6398 null : this.prepareFlushResult.storeFlushCtxs.get(
6399 store.getColumnFamilyDescriptor().getName());
6400 if (ctx != null) {
6401 MemStoreSize mss = store.getFlushableSize();
6402 ctx.abort();
6403 this.decrMemStoreSize(mss);
6404 this.prepareFlushResult.storeFlushCtxs.
6405 remove(store.getColumnFamilyDescriptor().getName());
6406 totalFreedDataSize += mss.getDataSize();
6411 map.put(store, storeSeqId);
6415 // if all stores ended up dropping their snapshots, we can safely drop the
6416 // prepareFlushResult
6417 dropPrepareFlushIfPossible();
6419 // advance the mvcc read point so that the new flushed files are visible.
6420 // either greater than flush seq number or they were already picked up via flush.
6421 for (HStore s : stores.values()) {
6422 mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L));
6426 // smallestSeqIdInStores is the seqId that we have a corresponding hfile for. We can safely
6427 // skip all edits that are to be replayed in the future with that has a smaller seqId
6428 // than this. We are updating lastReplayedOpenRegionSeqId so that we can skip all edits
6429 // that we have picked the flush files for
6430 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
6431 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
6434 if (!map.isEmpty()) {
6435 for (Map.Entry<HStore, Long> entry : map.entrySet()) {
6436 // Drop the memstore contents if they are now smaller than the latest seen flushed file
6437 totalFreedDataSize += dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey())
6438 .getDataSize();
6441 // C. Finally notify anyone waiting on memstore to clear:
6442 // e.g. checkResources().
6443 synchronized (this) {
6444 notifyAll(); // FindBugs NN_NAKED_NOTIFY
6446 return totalFreedDataSize > 0;
6447 } finally {
6448 closeRegionOperation();
6452 private void logRegionFiles() {
6453 if (LOG.isTraceEnabled()) {
6454 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
6455 stores.values().stream().filter(s -> s.getStorefiles() != null)
6456 .flatMap(s -> s.getStorefiles().stream())
6457 .forEachOrdered(sf -> LOG.trace(getRegionInfo().getEncodedName() + " : " + sf));
6461 /** Checks whether the given regionName is either equal to our region, or that
6462 * the regionName is the primary region to our corresponding range for the secondary replica.
6464 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
6465 throws WrongRegionException {
6466 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
6467 return;
6470 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
6471 Bytes.equals(encodedRegionName,
6472 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
6473 return;
6476 throw new WrongRegionException(exceptionMsg + payload
6477 + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
6478 + " does not match this region: " + this.getRegionInfo());
6482 * Used by tests
6483 * @param s Store to add edit too.
6484 * @param cell Cell to add.
6486 protected void restoreEdit(HStore s, Cell cell, MemStoreSizing memstoreAccounting) {
6487 s.add(cell, memstoreAccounting);
6491 * make sure have been through lease recovery before get file status, so the file length can be
6492 * trusted.
6493 * @param p File to check.
6494 * @return True if file was zero-length (and if so, we'll delete it in here).
6495 * @throws IOException
6497 private static boolean isZeroLengthThenDelete(final FileSystem fs, final FileStatus stat,
6498 final Path p) throws IOException {
6499 if (stat.getLen() > 0) {
6500 return false;
6502 LOG.warn("File " + p + " is zero-length, deleting.");
6503 fs.delete(p, false);
6504 return true;
6507 protected HStore instantiateHStore(final ColumnFamilyDescriptor family, boolean warmup)
6508 throws IOException {
6509 if (family.isMobEnabled()) {
6510 if (HFile.getFormatVersion(this.conf) < HFile.MIN_FORMAT_VERSION_WITH_TAGS) {
6511 throw new IOException("A minimum HFile version of " + HFile.MIN_FORMAT_VERSION_WITH_TAGS +
6512 " is required for MOB feature. Consider setting " + HFile.FORMAT_VERSION_KEY +
6513 " accordingly.");
6515 return new HMobStore(this, family, this.conf, warmup);
6517 return new HStore(this, family, this.conf, warmup);
6520 @Override
6521 public HStore getStore(byte[] column) {
6522 return this.stores.get(column);
6526 * Return HStore instance. Does not do any copy: as the number of store is limited, we iterate on
6527 * the list.
6529 private HStore getStore(Cell cell) {
6530 return stores.entrySet().stream().filter(e -> CellUtil.matchingFamily(cell, e.getKey()))
6531 .map(e -> e.getValue()).findFirst().orElse(null);
6534 @Override
6535 public List<HStore> getStores() {
6536 return new ArrayList<>(stores.values());
6539 @Override
6540 public List<String> getStoreFileList(byte[][] columns) throws IllegalArgumentException {
6541 List<String> storeFileNames = new ArrayList<>();
6542 synchronized (closeLock) {
6543 for (byte[] column : columns) {
6544 HStore store = this.stores.get(column);
6545 if (store == null) {
6546 throw new IllegalArgumentException(
6547 "No column family : " + new String(column, StandardCharsets.UTF_8) + " available");
6549 Collection<HStoreFile> storeFiles = store.getStorefiles();
6550 if (storeFiles == null) {
6551 continue;
6553 for (HStoreFile storeFile : storeFiles) {
6554 storeFileNames.add(storeFile.getPath().toString());
6557 logRegionFiles();
6560 return storeFileNames;
6563 //////////////////////////////////////////////////////////////////////////////
6564 // Support code
6565 //////////////////////////////////////////////////////////////////////////////
6567 /** Make sure this is a valid row for the HRegion */
6568 void checkRow(byte[] row, String op) throws IOException {
6569 if (!rowIsInRange(getRegionInfo(), row)) {
6570 throw new WrongRegionException("Requested row out of range for " +
6571 op + " on HRegion " + this + ", startKey='" +
6572 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
6573 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
6574 Bytes.toStringBinary(row) + "'");
6580 * Get an exclusive ( write lock ) lock on a given row.
6581 * @param row Which row to lock.
6582 * @return A locked RowLock. The lock is exclusive and already aqquired.
6583 * @throws IOException
6585 public RowLock getRowLock(byte[] row) throws IOException {
6586 return getRowLock(row, false);
6589 @Override
6590 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
6591 checkRow(row, "row lock");
6592 return getRowLock(row, readLock, null);
6595 Span createRegionSpan(String name) {
6596 return TraceUtil.createSpan(name).setAttribute(REGION_NAMES_KEY,
6597 Collections.singletonList(getRegionInfo().getRegionNameAsString()));
6600 // will be override in tests
6601 protected RowLock getRowLockInternal(byte[] row, boolean readLock, RowLock prevRowLock)
6602 throws IOException {
6603 // create an object to use a a key in the row lock map
6604 HashedBytes rowKey = new HashedBytes(row);
6606 RowLockContext rowLockContext = null;
6607 RowLockImpl result = null;
6609 boolean success = false;
6610 try {
6611 // Keep trying until we have a lock or error out.
6612 // TODO: do we need to add a time component here?
6613 while (result == null) {
6614 rowLockContext = computeIfAbsent(lockedRows, rowKey, () -> new RowLockContext(rowKey));
6615 // Now try an get the lock.
6616 // This can fail as
6617 if (readLock) {
6618 // For read lock, if the caller has locked the same row previously, it will not try
6619 // to acquire the same read lock. It simply returns the previous row lock.
6620 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
6621 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
6622 rowLockContext.readWriteLock.readLock())) {
6623 success = true;
6624 return prevRowLock;
6626 result = rowLockContext.newReadLock();
6627 } else {
6628 result = rowLockContext.newWriteLock();
6632 int timeout = rowLockWaitDuration;
6633 boolean reachDeadlineFirst = false;
6634 Optional<RpcCall> call = RpcServer.getCurrentCall();
6635 if (call.isPresent()) {
6636 long deadline = call.get().getDeadline();
6637 if (deadline < Long.MAX_VALUE) {
6638 int timeToDeadline = (int) (deadline - EnvironmentEdgeManager.currentTime());
6639 if (timeToDeadline <= this.rowLockWaitDuration) {
6640 reachDeadlineFirst = true;
6641 timeout = timeToDeadline;
6646 if (timeout <= 0 || !result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS)) {
6647 String message = "Timed out waiting for lock for row: " + rowKey + " in region "
6648 + getRegionInfo().getEncodedName();
6649 if (reachDeadlineFirst) {
6650 throw new TimeoutIOException(message);
6651 } else {
6652 // If timeToDeadline is larger than rowLockWaitDuration, we can not drop the request.
6653 throw new IOException(message);
6656 rowLockContext.setThreadName(Thread.currentThread().getName());
6657 success = true;
6658 return result;
6659 } catch (InterruptedException ie) {
6660 if (LOG.isDebugEnabled()) {
6661 LOG.debug("Thread interrupted waiting for lock on row: {}, in region {}", rowKey,
6662 getRegionInfo().getRegionNameAsString());
6664 throw throwOnInterrupt(ie);
6665 } catch (Error error) {
6666 // The maximum lock count for read lock is 64K (hardcoded), when this maximum count
6667 // is reached, it will throw out an Error. This Error needs to be caught so it can
6668 // go ahead to process the minibatch with lock acquired.
6669 LOG.warn("Error to get row lock for {}, in region {}, cause: {}", Bytes.toStringBinary(row),
6670 getRegionInfo().getRegionNameAsString(), error);
6671 IOException ioe = new IOException(error);
6672 throw ioe;
6673 } finally {
6674 // Clean up the counts just in case this was the thing keeping the context alive.
6675 if (!success && rowLockContext != null) {
6676 rowLockContext.cleanUp();
6681 private RowLock getRowLock(byte[] row, boolean readLock, final RowLock prevRowLock)
6682 throws IOException {
6683 return TraceUtil.trace(() -> getRowLockInternal(row, readLock, prevRowLock),
6684 () -> createRegionSpan("Region.getRowLock").setAttribute(ROW_LOCK_READ_LOCK_KEY,
6685 readLock));
6688 private void releaseRowLocks(List<RowLock> rowLocks) {
6689 if (rowLocks != null) {
6690 for (RowLock rowLock : rowLocks) {
6691 rowLock.release();
6693 rowLocks.clear();
6697 public int getReadLockCount() {
6698 return lock.getReadLockCount();
6701 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
6702 return lockedRows;
6705 class RowLockContext {
6706 private final HashedBytes row;
6707 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
6708 final AtomicBoolean usable = new AtomicBoolean(true);
6709 final AtomicInteger count = new AtomicInteger(0);
6710 final Object lock = new Object();
6711 private String threadName;
6713 RowLockContext(HashedBytes row) {
6714 this.row = row;
6717 RowLockImpl newWriteLock() {
6718 Lock l = readWriteLock.writeLock();
6719 return getRowLock(l);
6721 RowLockImpl newReadLock() {
6722 Lock l = readWriteLock.readLock();
6723 return getRowLock(l);
6726 private RowLockImpl getRowLock(Lock l) {
6727 count.incrementAndGet();
6728 synchronized (lock) {
6729 if (usable.get()) {
6730 return new RowLockImpl(this, l);
6731 } else {
6732 return null;
6737 void cleanUp() {
6738 long c = count.decrementAndGet();
6739 if (c <= 0) {
6740 synchronized (lock) {
6741 if (count.get() <= 0 && usable.get()){ // Don't attempt to remove row if already removed
6742 usable.set(false);
6743 RowLockContext removed = lockedRows.remove(row);
6744 assert removed == this: "we should never remove a different context";
6750 public void setThreadName(String threadName) {
6751 this.threadName = threadName;
6754 @Override
6755 public String toString() {
6756 return "RowLockContext{" +
6757 "row=" + row +
6758 ", readWriteLock=" + readWriteLock +
6759 ", count=" + count +
6760 ", threadName=" + threadName +
6761 '}';
6766 * Class used to represent a lock on a row.
6768 public static class RowLockImpl implements RowLock {
6769 private final RowLockContext context;
6770 private final Lock lock;
6772 public RowLockImpl(RowLockContext context, Lock lock) {
6773 this.context = context;
6774 this.lock = lock;
6777 public Lock getLock() {
6778 return lock;
6781 public RowLockContext getContext() {
6782 return context;
6785 @Override
6786 public void release() {
6787 lock.unlock();
6788 context.cleanUp();
6791 @Override
6792 public String toString() {
6793 return "RowLockImpl{" +
6794 "context=" + context +
6795 ", lock=" + lock +
6796 '}';
6801 * Determines whether multiple column families are present
6802 * Precondition: familyPaths is not null
6804 * @param familyPaths List of (column family, hfilePath)
6806 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
6807 boolean multipleFamilies = false;
6808 byte[] family = null;
6809 for (Pair<byte[], String> pair : familyPaths) {
6810 byte[] fam = pair.getFirst();
6811 if (family == null) {
6812 family = fam;
6813 } else if (!Bytes.equals(family, fam)) {
6814 multipleFamilies = true;
6815 break;
6818 return multipleFamilies;
6822 * Attempts to atomically load a group of hfiles. This is critical for loading
6823 * rows with multiple column families atomically.
6825 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
6826 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
6827 * file about to be bulk loaded
6828 * @param assignSeqId
6829 * @return Map from family to List of store file paths if successful, null if failed recoverably
6830 * @throws IOException if failed unrecoverably.
6832 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
6833 BulkLoadListener bulkLoadListener) throws IOException {
6834 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, false,
6835 null, true);
6839 * Listener class to enable callers of
6840 * bulkLoadHFile() to perform any necessary
6841 * pre/post processing of a given bulkload call
6843 public interface BulkLoadListener {
6845 * Called before an HFile is actually loaded
6846 * @param family family being loaded to
6847 * @param srcPath path of HFile
6848 * @return final path to be used for actual loading
6849 * @throws IOException
6851 String prepareBulkLoad(byte[] family, String srcPath, boolean copyFile)
6852 throws IOException;
6855 * Called after a successful HFile load
6856 * @param family family being loaded to
6857 * @param srcPath path of HFile
6858 * @throws IOException
6860 void doneBulkLoad(byte[] family, String srcPath) throws IOException;
6863 * Called after a failed HFile load
6864 * @param family family being loaded to
6865 * @param srcPath path of HFile
6866 * @throws IOException
6868 void failedBulkLoad(byte[] family, String srcPath) throws IOException;
6872 * Attempts to atomically load a group of hfiles. This is critical for loading
6873 * rows with multiple column families atomically.
6875 * @param familyPaths List of Pair&lt;byte[] column family, String hfilePath&gt;
6876 * @param assignSeqId
6877 * @param bulkLoadListener Internal hooks enabling massaging/preparation of a
6878 * file about to be bulk loaded
6879 * @param copyFile always copy hfiles if true
6880 * @param clusterIds ids from clusters that had already handled the given bulkload event.
6881 * @return Map from family to List of store file paths if successful, null if failed recoverably
6882 * @throws IOException if failed unrecoverably.
6884 public Map<byte[], List<Path>> bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths,
6885 boolean assignSeqId, BulkLoadListener bulkLoadListener, boolean copyFile,
6886 List<String> clusterIds, boolean replicate) throws IOException {
6887 long seqId = -1;
6888 Map<byte[], List<Path>> storeFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
6889 Map<String, Long> storeFilesSizes = new HashMap<>();
6890 Preconditions.checkNotNull(familyPaths);
6891 // we need writeLock for multi-family bulk load
6892 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
6893 boolean isSuccessful = false;
6894 try {
6895 this.writeRequestsCount.increment();
6897 // There possibly was a split that happened between when the split keys
6898 // were gathered and before the HRegion's write lock was taken. We need
6899 // to validate the HFile region before attempting to bulk load all of them
6900 IOException ioException = null;
6901 List<Pair<byte[], String>> failures = new ArrayList<>();
6902 for (Pair<byte[], String> p : familyPaths) {
6903 byte[] familyName = p.getFirst();
6904 String path = p.getSecond();
6906 HStore store = getStore(familyName);
6907 if (store == null) {
6908 ioException = new org.apache.hadoop.hbase.DoNotRetryIOException(
6909 "No such column family " + Bytes.toStringBinary(familyName));
6910 } else {
6911 try {
6912 store.assertBulkLoadHFileOk(new Path(path));
6913 } catch (WrongRegionException wre) {
6914 // recoverable (file doesn't fit in region)
6915 failures.add(p);
6916 } catch (IOException ioe) {
6917 // unrecoverable (hdfs problem)
6918 ioException = ioe;
6922 // validation failed because of some sort of IO problem.
6923 if (ioException != null) {
6924 LOG.error("There was IO error when checking if the bulk load is ok in region {}.", this,
6925 ioException);
6926 throw ioException;
6929 // validation failed, bail out before doing anything permanent.
6930 if (failures.size() != 0) {
6931 StringBuilder list = new StringBuilder();
6932 for (Pair<byte[], String> p : failures) {
6933 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
6934 .append(p.getSecond());
6936 // problem when validating
6937 LOG.warn("There was a recoverable bulk load failure likely due to a split. These (family,"
6938 + " HFile) pairs were not loaded: {}, in region {}", list.toString(), this);
6939 return null;
6942 // We need to assign a sequential ID that's in between two memstores in order to preserve
6943 // the guarantee that all the edits lower than the highest sequential ID from all the
6944 // HFiles are flushed on disk. See HBASE-10958. The sequence id returned when we flush is
6945 // guaranteed to be one beyond the file made when we flushed (or if nothing to flush, it is
6946 // a sequence id that we can be sure is beyond the last hfile written).
6947 if (assignSeqId) {
6948 FlushResult fs = flushcache(true, false, FlushLifeCycleTracker.DUMMY);
6949 if (fs.isFlushSucceeded()) {
6950 seqId = ((FlushResultImpl)fs).flushSequenceId;
6951 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
6952 seqId = ((FlushResultImpl)fs).flushSequenceId;
6953 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
6954 // CANNOT_FLUSH may mean that a flush is already on-going
6955 // we need to wait for that flush to complete
6956 waitForFlushes();
6957 } else {
6958 throw new IOException("Could not bulk load with an assigned sequential ID because the "+
6959 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
6963 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
6964 new TreeMap<>(Bytes.BYTES_COMPARATOR);
6965 for (Pair<byte[], String> p : familyPaths) {
6966 byte[] familyName = p.getFirst();
6967 String path = p.getSecond();
6968 HStore store = getStore(familyName);
6969 if (!familyWithFinalPath.containsKey(familyName)) {
6970 familyWithFinalPath.put(familyName, new ArrayList<>());
6972 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
6973 try {
6974 String finalPath = path;
6975 if (bulkLoadListener != null) {
6976 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path, copyFile);
6978 Pair<Path, Path> pair = store.preBulkLoadHFile(finalPath, seqId);
6979 lst.add(pair);
6980 } catch (IOException ioe) {
6981 // A failure here can cause an atomicity violation that we currently
6982 // cannot recover from since it is likely a failed HDFS operation.
6984 LOG.error("There was a partial failure due to IO when attempting to" +
6985 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
6986 if (bulkLoadListener != null) {
6987 try {
6988 bulkLoadListener.failedBulkLoad(familyName, path);
6989 } catch (Exception ex) {
6990 LOG.error("Error while calling failedBulkLoad for family " +
6991 Bytes.toString(familyName) + " with path " + path, ex);
6994 throw ioe;
6998 if (this.getCoprocessorHost() != null) {
6999 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7000 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
7003 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
7004 byte[] familyName = entry.getKey();
7005 for (Pair<Path, Path> p : entry.getValue()) {
7006 String path = p.getFirst().toString();
7007 Path commitedStoreFile = p.getSecond();
7008 HStore store = getStore(familyName);
7009 try {
7010 store.bulkLoadHFile(familyName, path, commitedStoreFile);
7011 // Note the size of the store file
7012 try {
7013 FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
7014 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile)
7015 .getLen());
7016 } catch (IOException e) {
7017 LOG.warn("Failed to find the size of hfile " + commitedStoreFile, e);
7018 storeFilesSizes.put(commitedStoreFile.getName(), 0L);
7021 if(storeFiles.containsKey(familyName)) {
7022 storeFiles.get(familyName).add(commitedStoreFile);
7023 } else {
7024 List<Path> storeFileNames = new ArrayList<>();
7025 storeFileNames.add(commitedStoreFile);
7026 storeFiles.put(familyName, storeFileNames);
7028 if (bulkLoadListener != null) {
7029 bulkLoadListener.doneBulkLoad(familyName, path);
7031 } catch (IOException ioe) {
7032 // A failure here can cause an atomicity violation that we currently
7033 // cannot recover from since it is likely a failed HDFS operation.
7035 // TODO Need a better story for reverting partial failures due to HDFS.
7036 LOG.error("There was a partial failure due to IO when attempting to" +
7037 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
7038 if (bulkLoadListener != null) {
7039 try {
7040 bulkLoadListener.failedBulkLoad(familyName, path);
7041 } catch (Exception ex) {
7042 LOG.error("Error while calling failedBulkLoad for family " +
7043 Bytes.toString(familyName) + " with path " + path, ex);
7046 throw ioe;
7051 isSuccessful = true;
7052 if (conf.getBoolean(COMPACTION_AFTER_BULKLOAD_ENABLE, false)) {
7053 // request compaction
7054 familyWithFinalPath.keySet().forEach(family -> {
7055 HStore store = getStore(family);
7056 try {
7057 if (this.rsServices != null && store.needsCompaction()) {
7058 this.rsServices.getCompactionRequestor().requestSystemCompaction(this, store,
7059 "bulkload hfiles request compaction", true);
7060 LOG.info("Request compaction for region {} family {} after bulk load",
7061 this.getRegionInfo().getEncodedName(), store.getColumnFamilyName());
7063 } catch (IOException e) {
7064 LOG.error("bulkload hfiles request compaction error ", e);
7068 } finally {
7069 if (wal != null && !storeFiles.isEmpty()) {
7070 // Write a bulk load event for hfiles that are loaded
7071 try {
7072 WALProtos.BulkLoadDescriptor loadDescriptor =
7073 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
7074 UnsafeByteOperations.unsafeWrap(this.getRegionInfo().getEncodedNameAsBytes()),
7075 storeFiles, storeFilesSizes, seqId, clusterIds, replicate);
7076 WALUtil.writeBulkLoadMarkerAndSync(this.wal, this.getReplicationScope(), getRegionInfo(),
7077 loadDescriptor, mvcc);
7078 } catch (IOException ioe) {
7079 if (this.rsServices != null) {
7080 // Have to abort region server because some hfiles has been loaded but we can't write
7081 // the event into WAL
7082 isSuccessful = false;
7083 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
7088 closeBulkRegionOperation();
7090 return isSuccessful ? storeFiles : null;
7093 @Override
7094 public boolean equals(Object o) {
7095 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
7096 ((HRegion) o).getRegionInfo().getRegionName());
7099 @Override
7100 public int hashCode() {
7101 return Bytes.hashCode(getRegionInfo().getRegionName());
7104 @Override
7105 public String toString() {
7106 return getRegionInfo().getRegionNameAsString();
7109 // Utility methods
7111 * A utility method to create new instances of HRegion based on the {@link HConstants#REGION_IMPL}
7112 * configuration property.
7113 * @param tableDir qualified path of directory where region should be located, usually the table
7114 * directory.
7115 * @param wal The WAL is the outbound log for any updates to the HRegion The wal file is a logfile
7116 * from the previous execution that's custom-computed for this HRegion. The HRegionServer
7117 * computes and sorts the appropriate wal info for this HRegion. If there is a previous
7118 * file (implying that the HRegion has been written-to before), then read it from the
7119 * supplied path.
7120 * @param fs is the filesystem.
7121 * @param conf is global configuration settings.
7122 * @param regionInfo - RegionInfo that describes the region is new), then read them from the
7123 * supplied path.
7124 * @param htd the table descriptor
7125 * @return the new instance
7127 public static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
7128 Configuration conf, RegionInfo regionInfo, final TableDescriptor htd,
7129 RegionServerServices rsServices) {
7130 try {
7131 @SuppressWarnings("unchecked")
7132 Class<? extends HRegion> regionClass =
7133 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
7135 Constructor<? extends HRegion> c =
7136 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
7137 Configuration.class, RegionInfo.class, TableDescriptor.class,
7138 RegionServerServices.class);
7140 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
7141 } catch (Throwable e) {
7142 // todo: what should I throw here?
7143 throw new IllegalStateException("Could not instantiate a region instance.", e);
7148 * Convenience method creating new HRegions. Used by createTable.
7149 * @param info Info for region to create.
7150 * @param rootDir Root directory for HBase instance
7151 * @param wal shared WAL
7152 * @param initialize - true to initialize the region
7153 * @return new HRegion
7155 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7156 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7157 final boolean initialize) throws IOException {
7158 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, initialize, null);
7162 * Convenience method creating new HRegions. Used by createTable.
7163 * @param info Info for region to create.
7164 * @param rootDir Root directory for HBase instance
7165 * @param wal shared WAL
7166 * @param initialize - true to initialize the region
7167 * @param rsRpcServices An interface we can request flushes against.
7168 * @return new HRegion
7170 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7171 final Configuration conf, final TableDescriptor hTableDescriptor, final WAL wal,
7172 final boolean initialize, RegionServerServices rsRpcServices) throws IOException {
7173 LOG.info("creating " + info + ", tableDescriptor="
7174 + (hTableDescriptor == null ? "null" : hTableDescriptor) + ", regionDir=" + rootDir);
7175 createRegionDir(conf, info, rootDir);
7176 FileSystem fs = rootDir.getFileSystem(conf);
7177 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7178 HRegion region =
7179 HRegion.newHRegion(tableDir, wal, fs, conf, info, hTableDescriptor, rsRpcServices);
7180 if (initialize) {
7181 region.initialize(null);
7183 return region;
7187 * Create a region under the given table directory.
7189 public static HRegion createHRegion(Configuration conf, RegionInfo regionInfo, FileSystem fs,
7190 Path tableDir, TableDescriptor tableDesc) throws IOException {
7191 LOG.info("Creating {}, tableDescriptor={}, under table dir {}", regionInfo, tableDesc,
7192 tableDir);
7193 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, regionInfo);
7194 HRegion region = HRegion.newHRegion(tableDir, null, fs, conf, regionInfo, tableDesc, null);
7195 return region;
7199 * Create the region directory in the filesystem.
7201 public static HRegionFileSystem createRegionDir(Configuration configuration, RegionInfo ri,
7202 Path rootDir)
7203 throws IOException {
7204 FileSystem fs = rootDir.getFileSystem(configuration);
7205 Path tableDir = CommonFSUtils.getTableDir(rootDir, ri.getTable());
7206 // If directory already exists, will log warning and keep going. Will try to create
7207 // .regioninfo. If one exists, will overwrite.
7208 return HRegionFileSystem.createRegionOnFileSystem(configuration, fs, tableDir, ri);
7211 public static HRegion createHRegion(final RegionInfo info, final Path rootDir,
7212 final Configuration conf,
7213 final TableDescriptor hTableDescriptor,
7214 final WAL wal)
7215 throws IOException {
7216 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7221 * Open a Region.
7222 * @param info Info for region to be opened.
7223 * @param wal WAL for region to use. This method will call
7224 * WAL#setSequenceNumber(long) passing the result of the call to
7225 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7226 * up. HRegionStore does this every time it opens a new region.
7227 * @return new HRegion
7229 * @throws IOException
7231 public static HRegion openHRegion(final RegionInfo info,
7232 final TableDescriptor htd, final WAL wal,
7233 final Configuration conf)
7234 throws IOException {
7235 return openHRegion(info, htd, wal, conf, null, null);
7239 * Open a Region.
7240 * @param info Info for region to be opened
7241 * @param htd the table descriptor
7242 * @param wal WAL for region to use. This method will call
7243 * WAL#setSequenceNumber(long) passing the result of the call to
7244 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7245 * up. HRegionStore does this every time it opens a new region.
7246 * @param conf The Configuration object to use.
7247 * @param rsServices An interface we can request flushes against.
7248 * @param reporter An interface we can report progress against.
7249 * @return new HRegion
7251 * @throws IOException
7253 public static HRegion openHRegion(final RegionInfo info,
7254 final TableDescriptor htd, final WAL wal, final Configuration conf,
7255 final RegionServerServices rsServices,
7256 final CancelableProgressable reporter)
7257 throws IOException {
7258 return openHRegion(CommonFSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7262 * Open a Region.
7263 * @param rootDir Root directory for HBase instance
7264 * @param info Info for region to be opened.
7265 * @param htd the table descriptor
7266 * @param wal WAL for region to use. This method will call
7267 * WAL#setSequenceNumber(long) passing the result of the call to
7268 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7269 * up. HRegionStore does this every time it opens a new region.
7270 * @param conf The Configuration object to use.
7271 * @return new HRegion
7272 * @throws IOException
7274 public static HRegion openHRegion(Path rootDir, final RegionInfo info,
7275 final TableDescriptor htd, final WAL wal, final Configuration conf)
7276 throws IOException {
7277 return openHRegion(rootDir, info, htd, wal, conf, null, null);
7281 * Open a Region.
7282 * @param rootDir Root directory for HBase instance
7283 * @param info Info for region to be opened.
7284 * @param htd the table descriptor
7285 * @param wal WAL for region to use. This method will call
7286 * WAL#setSequenceNumber(long) passing the result of the call to
7287 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7288 * up. HRegionStore does this every time it opens a new region.
7289 * @param conf The Configuration object to use.
7290 * @param rsServices An interface we can request flushes against.
7291 * @param reporter An interface we can report progress against.
7292 * @return new HRegion
7293 * @throws IOException
7295 public static HRegion openHRegion(final Path rootDir, final RegionInfo info,
7296 final TableDescriptor htd, final WAL wal, final Configuration conf,
7297 final RegionServerServices rsServices,
7298 final CancelableProgressable reporter)
7299 throws IOException {
7300 FileSystem fs = null;
7301 if (rsServices != null) {
7302 fs = rsServices.getFileSystem();
7304 if (fs == null) {
7305 fs = rootDir.getFileSystem(conf);
7307 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7311 * Open a Region.
7312 * @param conf The Configuration object to use.
7313 * @param fs Filesystem to use
7314 * @param rootDir Root directory for HBase instance
7315 * @param info Info for region to be opened.
7316 * @param htd the table descriptor
7317 * @param wal WAL for region to use. This method will call
7318 * WAL#setSequenceNumber(long) passing the result of the call to
7319 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7320 * up. HRegionStore does this every time it opens a new region.
7321 * @return new HRegion
7323 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7324 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal)
7325 throws IOException {
7326 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7330 * Open a Region.
7331 * @param conf The Configuration object to use.
7332 * @param fs Filesystem to use
7333 * @param rootDir Root directory for HBase instance
7334 * @param info Info for region to be opened.
7335 * @param htd the table descriptor
7336 * @param wal WAL for region to use. This method will call
7337 * WAL#setSequenceNumber(long) passing the result of the call to
7338 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7339 * up. HRegionStore does this every time it opens a new region.
7340 * @param rsServices An interface we can request flushes against.
7341 * @param reporter An interface we can report progress against.
7342 * @return new HRegion
7344 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7345 final Path rootDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7346 final RegionServerServices rsServices, final CancelableProgressable reporter)
7347 throws IOException {
7348 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7349 return openHRegionFromTableDir(conf, fs, tableDir, info, htd, wal, rsServices, reporter);
7353 * Open a Region.
7354 * @param conf The Configuration object to use.
7355 * @param fs Filesystem to use
7356 * @param info Info for region to be opened.
7357 * @param htd the table descriptor
7358 * @param wal WAL for region to use. This method will call
7359 * WAL#setSequenceNumber(long) passing the result of the call to
7360 * HRegion#getMinSequenceId() to ensure the wal id is properly kept
7361 * up. HRegionStore does this every time it opens a new region.
7362 * @param rsServices An interface we can request flushes against.
7363 * @param reporter An interface we can report progress against.
7364 * @return new HRegion
7365 * @throws NullPointerException if {@code info} is {@code null}
7367 public static HRegion openHRegionFromTableDir(final Configuration conf, final FileSystem fs,
7368 final Path tableDir, final RegionInfo info, final TableDescriptor htd, final WAL wal,
7369 final RegionServerServices rsServices, final CancelableProgressable reporter)
7370 throws IOException {
7371 Objects.requireNonNull(info, "RegionInfo cannot be null");
7372 LOG.debug("Opening region: {}", info);
7373 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7374 return r.openHRegion(reporter);
7377 public NavigableMap<byte[], Integer> getReplicationScope() {
7378 return this.replicationScope;
7382 * Useful when reopening a closed region (normally for unit tests)
7383 * @param other original object
7384 * @param reporter An interface we can report progress against.
7385 * @return new HRegion
7387 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7388 throws IOException {
7389 HRegionFileSystem regionFs = other.getRegionFileSystem();
7390 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7391 other.baseConf, other.getRegionInfo(), other.getTableDescriptor(), null);
7392 return r.openHRegion(reporter);
7395 public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7396 throws IOException {
7397 return openHRegion((HRegion)other, reporter);
7401 * Open HRegion.
7402 * <p/>
7403 * Calls initialize and sets sequenceId.
7404 * @return Returns <code>this</code>
7406 private HRegion openHRegion(final CancelableProgressable reporter) throws IOException {
7407 try {
7408 // Refuse to open the region if we are missing local compression support
7409 TableDescriptorChecker.checkCompression(htableDescriptor);
7410 // Refuse to open the region if encryption configuration is incorrect or
7411 // codec support is missing
7412 LOG.debug("checking encryption for " + this.getRegionInfo().getEncodedName());
7413 TableDescriptorChecker.checkEncryption(conf, htableDescriptor);
7414 // Refuse to open the region if a required class cannot be loaded
7415 LOG.debug("checking classloading for " + this.getRegionInfo().getEncodedName());
7416 TableDescriptorChecker.checkClassLoading(conf, htableDescriptor);
7417 this.openSeqNum = initialize(reporter);
7418 this.mvcc.advanceTo(openSeqNum);
7419 // The openSeqNum must be increased every time when a region is assigned, as we rely on it to
7420 // determine whether a region has been successfully reopened. So here we always write open
7421 // marker, even if the table is read only.
7422 if (wal != null && getRegionServerServices() != null &&
7423 RegionReplicaUtil.isDefaultReplica(getRegionInfo())) {
7424 writeRegionOpenMarker(wal, openSeqNum);
7426 } catch (Throwable t) {
7427 // By coprocessor path wrong region will open failed,
7428 // MetricsRegionWrapperImpl is already init and not close,
7429 // add region close when open failed
7430 try {
7431 // It is not required to write sequence id file when region open is failed.
7432 // Passing true to skip the sequence id file write.
7433 this.close(true);
7434 } catch (Throwable e) {
7435 LOG.warn("Open region: {} failed. Try close region but got exception ", this.getRegionInfo(),
7438 throw t;
7440 return this;
7444 * Open a Region on a read-only file-system (like hdfs snapshots)
7445 * @param conf The Configuration object to use.
7446 * @param fs Filesystem to use
7447 * @param info Info for region to be opened.
7448 * @param htd the table descriptor
7449 * @return new HRegion
7450 * @throws NullPointerException if {@code info} is {@code null}
7452 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7453 final Path tableDir, RegionInfo info, final TableDescriptor htd) throws IOException {
7454 Objects.requireNonNull(info, "RegionInfo cannot be null");
7455 if (LOG.isDebugEnabled()) {
7456 LOG.debug("Opening region (readOnly filesystem): " + info);
7458 if (info.getReplicaId() <= 0) {
7459 info = RegionReplicaUtil.getRegionInfoForReplica(info, 1);
7461 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7462 r.writestate.setReadOnly(true);
7463 return r.openHRegion(null);
7466 public static void warmupHRegion(final RegionInfo info,
7467 final TableDescriptor htd, final WAL wal, final Configuration conf,
7468 final RegionServerServices rsServices,
7469 final CancelableProgressable reporter)
7470 throws IOException {
7472 Objects.requireNonNull(info, "RegionInfo cannot be null");
7473 LOG.debug("Warmup {}", info);
7474 Path rootDir = CommonFSUtils.getRootDir(conf);
7475 Path tableDir = CommonFSUtils.getTableDir(rootDir, info.getTable());
7476 FileSystem fs = null;
7477 if (rsServices != null) {
7478 fs = rsServices.getFileSystem();
7480 if (fs == null) {
7481 fs = rootDir.getFileSystem(conf);
7483 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7484 r.initializeWarmup(reporter);
7488 * Computes the Path of the HRegion
7490 * @param tabledir qualified path for table
7491 * @param name ENCODED region name
7492 * @return Path of HRegion directory
7493 * @deprecated For tests only; to be removed.
7495 @Deprecated
7496 public static Path getRegionDir(final Path tabledir, final String name) {
7497 return new Path(tabledir, name);
7501 * Determines if the specified row is within the row range specified by the
7502 * specified RegionInfo
7504 * @param info RegionInfo that specifies the row range
7505 * @param row row to be checked
7506 * @return true if the row is within the range specified by the RegionInfo
7508 public static boolean rowIsInRange(RegionInfo info, final byte [] row) {
7509 return ((info.getStartKey().length == 0) ||
7510 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
7511 ((info.getEndKey().length == 0) ||
7512 (Bytes.compareTo(info.getEndKey(), row) > 0));
7515 public static boolean rowIsInRange(RegionInfo info, final byte [] row, final int offset,
7516 final short length) {
7517 return ((info.getStartKey().length == 0) ||
7518 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
7519 row, offset, length) <= 0)) &&
7520 ((info.getEndKey().length == 0) ||
7521 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
7524 @Override
7525 public Result get(final Get get) throws IOException {
7526 prepareGet(get);
7527 List<Cell> results = get(get, true);
7528 boolean stale = this.getRegionInfo().getReplicaId() != 0;
7529 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7532 void prepareGet(final Get get) throws IOException {
7533 checkRow(get.getRow(), "Get");
7534 // Verify families are all valid
7535 if (get.hasFamilies()) {
7536 for (byte[] family : get.familySet()) {
7537 checkFamily(family);
7539 } else { // Adding all families to scanner
7540 for (byte[] family : this.htableDescriptor.getColumnFamilyNames()) {
7541 get.addFamily(family);
7546 @Override
7547 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7548 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7551 private List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7552 throws IOException {
7553 return TraceUtil.trace(() -> getInternal(get, withCoprocessor, nonceGroup, nonce),
7554 () -> createRegionSpan("Region.get"));
7557 private List<Cell> getInternal(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7558 throws IOException {
7559 List<Cell> results = new ArrayList<>();
7560 long before = EnvironmentEdgeManager.currentTime();
7562 // pre-get CP hook
7563 if (withCoprocessor && (coprocessorHost != null)) {
7564 if (coprocessorHost.preGet(get, results)) {
7565 metricsUpdateForGet(results, before);
7566 return results;
7569 Scan scan = new Scan(get);
7570 if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
7571 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
7573 try (RegionScanner scanner = getScanner(scan, null, nonceGroup, nonce)) {
7574 List<Cell> tmp = new ArrayList<>();
7575 scanner.next(tmp);
7576 // Copy EC to heap, then close the scanner.
7577 // This can be an EXPENSIVE call. It may make an extra copy from offheap to onheap buffers.
7578 // See more details in HBASE-26036.
7579 for (Cell cell : tmp) {
7580 results.add(cell instanceof ByteBufferExtendedCell ?
7581 ((ByteBufferExtendedCell) cell).deepClone(): cell);
7585 // post-get CP hook
7586 if (withCoprocessor && (coprocessorHost != null)) {
7587 coprocessorHost.postGet(get, results);
7590 metricsUpdateForGet(results, before);
7592 return results;
7595 void metricsUpdateForGet(List<Cell> results, long before) {
7596 if (this.metricsRegion != null) {
7597 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before);
7599 if (this.rsServices != null && this.rsServices.getMetrics() != null) {
7600 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable(), 1);
7605 @Override
7606 public Result mutateRow(RowMutations rm) throws IOException {
7607 return mutateRow(rm, HConstants.NO_NONCE, HConstants.NO_NONCE);
7610 public Result mutateRow(RowMutations rm, long nonceGroup, long nonce) throws IOException {
7611 final List<Mutation> m = rm.getMutations();
7612 OperationStatus[] statuses = batchMutate(m.toArray(new Mutation[0]), true, nonceGroup, nonce);
7614 List<Result> results = new ArrayList<>();
7615 for (OperationStatus status : statuses) {
7616 if (status.getResult() != null) {
7617 results.add(status.getResult());
7621 if (results.isEmpty()) {
7622 return null;
7625 // Merge the results of the Increment/Append operations
7626 List<Cell> cells = new ArrayList<>();
7627 for (Result result : results) {
7628 if (result.rawCells() != null) {
7629 cells.addAll(Arrays.asList(result.rawCells()));
7632 return Result.create(cells);
7636 * Perform atomic (all or none) mutations within the region.
7637 * @param mutations The list of mutations to perform.
7638 * <code>mutations</code> can contain operations for multiple rows.
7639 * Caller has to ensure that all rows are contained in this region.
7640 * @param rowsToLock Rows to lock
7641 * @param nonceGroup Optional nonce group of the operation (client Id)
7642 * @param nonce Optional nonce of the operation (unique random id to ensure "more idempotence")
7643 * If multiple rows are locked care should be taken that
7644 * <code>rowsToLock</code> is sorted in order to avoid deadlocks.
7645 * @throws IOException
7647 @Override
7648 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7649 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
7650 batchMutate(new MutationBatchOperation(this, mutations.toArray(new Mutation[mutations.size()]),
7651 true, nonceGroup, nonce) {
7652 @Override
7653 public MiniBatchOperationInProgress<Mutation> lockRowsAndBuildMiniBatch(
7654 List<RowLock> acquiredRowLocks) throws IOException {
7655 RowLock prevRowLock = null;
7656 for (byte[] row : rowsToLock) {
7657 try {
7658 RowLock rowLock = region.getRowLock(row, false, prevRowLock); // write lock
7659 if (rowLock != prevRowLock) {
7660 acquiredRowLocks.add(rowLock);
7661 prevRowLock = rowLock;
7663 } catch (IOException ioe) {
7664 LOG.warn("Failed getting lock, row={}, in region {}", Bytes.toStringBinary(row), this,
7665 ioe);
7666 throw ioe;
7669 return createMiniBatch(size(), size());
7675 * @return statistics about the current load of the region
7677 public ClientProtos.RegionLoadStats getLoadStatistics() {
7678 if (!regionStatsEnabled) {
7679 return null;
7681 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
7682 stats.setMemStoreLoad((int) (Math.min(100,
7683 (this.memStoreSizing.getMemStoreSize().getHeapSize() * 100) / this.memstoreFlushSize)));
7684 if (rsServices.getHeapMemoryManager() != null) {
7685 // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM,
7686 // so we could just do the calculation below and we'll get a 0.
7687 // treating it as a special case analogous to no HMM instead so that it can be
7688 // programatically treated different from using <1% of heap.
7689 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
7690 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
7691 stats.setHeapOccupancy((int)(occupancy * 100));
7694 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100
7695 : rsServices.getCompactionPressure() * 100));
7696 return stats.build();
7699 @Override
7700 public Result append(Append append) throws IOException {
7701 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7704 public Result append(Append append, long nonceGroup, long nonce) throws IOException {
7705 return TraceUtil.trace(() -> {
7706 checkReadOnly();
7707 checkResources();
7708 startRegionOperation(Operation.APPEND);
7709 try {
7710 // All edits for the given row (across all column families) must happen atomically.
7711 return mutate(append, true, nonceGroup, nonce).getResult();
7712 } finally {
7713 closeRegionOperation(Operation.APPEND);
7715 }, () -> createRegionSpan("Region.append"));
7718 @Override
7719 public Result increment(Increment increment) throws IOException {
7720 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
7723 public Result increment(Increment increment, long nonceGroup, long nonce) throws IOException {
7724 return TraceUtil.trace(() -> {
7725 checkReadOnly();
7726 checkResources();
7727 startRegionOperation(Operation.INCREMENT);
7728 try {
7729 // All edits for the given row (across all column families) must happen atomically.
7730 return mutate(increment, true, nonceGroup, nonce).getResult();
7731 } finally {
7732 closeRegionOperation(Operation.INCREMENT);
7734 }, () -> createRegionSpan("Region.increment"));
7738 * @return writeEntry associated with this append
7740 private WriteEntry doWALAppend(WALEdit walEdit, Durability durability, List<UUID> clusterIds,
7741 long now, long nonceGroup, long nonce, long origLogSeqNum) throws IOException {
7742 Preconditions.checkArgument(walEdit != null && !walEdit.isEmpty(),
7743 "WALEdit is null or empty!");
7744 Preconditions.checkArgument(!walEdit.isReplay() || origLogSeqNum != SequenceId.NO_SEQUENCE_ID,
7745 "Invalid replay sequence Id for replay WALEdit!");
7746 // Using default cluster id, as this can only happen in the originating cluster.
7747 // A slave cluster receives the final value (not the delta) as a Put. We use HLogKey
7748 // here instead of WALKeyImpl directly to support legacy coprocessors.
7749 WALKeyImpl walKey = walEdit.isReplay()?
7750 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
7751 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
7752 nonceGroup, nonce, mvcc) :
7753 new WALKeyImpl(this.getRegionInfo().getEncodedNameAsBytes(),
7754 this.htableDescriptor.getTableName(), SequenceId.NO_SEQUENCE_ID, now, clusterIds,
7755 nonceGroup, nonce, mvcc, this.getReplicationScope());
7756 if (walEdit.isReplay()) {
7757 walKey.setOrigLogSeqNum(origLogSeqNum);
7759 //don't call the coproc hook for writes to the WAL caused by
7760 //system lifecycle events like flushes or compactions
7761 if (this.coprocessorHost != null && !walEdit.isMetaEdit()) {
7762 this.coprocessorHost.preWALAppend(walKey, walEdit);
7764 WriteEntry writeEntry = null;
7765 try {
7766 long txid = this.wal.appendData(this.getRegionInfo(), walKey, walEdit);
7767 // Call sync on our edit.
7768 if (txid != 0) {
7769 sync(txid, durability);
7771 writeEntry = walKey.getWriteEntry();
7772 } catch (IOException ioe) {
7773 if (walKey != null && walKey.getWriteEntry() != null) {
7774 mvcc.complete(walKey.getWriteEntry());
7776 throw ioe;
7778 return writeEntry;
7781 public static final long FIXED_OVERHEAD = ClassSize.estimateBase(HRegion.class, false);
7783 // woefully out of date - currently missing:
7784 // 1 x HashMap - coprocessorServiceHandlers
7785 // 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
7786 // checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount,
7787 // writeRequestsCount, cpRequestsCount
7788 // 1 x HRegion$WriteState - writestate
7789 // 1 x RegionCoprocessorHost - coprocessorHost
7790 // 1 x RegionSplitPolicy - splitPolicy
7791 // 1 x MetricsRegion - metricsRegion
7792 // 1 x MetricsRegionWrapperImpl - metricsRegionWrapper
7793 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
7794 ClassSize.OBJECT + // closeLock
7795 (2 * ClassSize.ATOMIC_BOOLEAN) + // closed, closing
7796 (3 * ClassSize.ATOMIC_LONG) + // numPutsWithoutWAL, dataInMemoryWithoutWAL,
7797 // compactionsFailed
7798 (3 * ClassSize.CONCURRENT_HASHMAP) + // lockedRows, scannerReadPoints, regionLockHolders
7799 WriteState.HEAP_SIZE + // writestate
7800 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY + // stores
7801 (2 * ClassSize.REENTRANT_LOCK) + // lock, updatesLock
7802 MultiVersionConcurrencyControl.FIXED_SIZE // mvcc
7803 + 2 * ClassSize.TREEMAP // maxSeqIdInStores, replicationScopes
7804 + 2 * ClassSize.ATOMIC_INTEGER // majorInProgress, minorInProgress
7805 + ClassSize.STORE_SERVICES // store services
7806 + StoreHotnessProtector.FIXED_SIZE
7809 @Override
7810 public long heapSize() {
7811 // this does not take into account row locks, recent flushes, mvcc entries, and more
7812 return DEEP_OVERHEAD + stores.values().stream().mapToLong(HStore::heapSize).sum();
7816 * Registers a new protocol buffer {@link Service} subclass as a coprocessor endpoint to be
7817 * available for handling {@link #execService(RpcController, CoprocessorServiceCall)} calls.
7818 * <p/>
7819 * Only a single instance may be registered per region for a given {@link Service} subclass (the
7820 * instances are keyed on {@link ServiceDescriptor#getFullName()}.. After the first registration,
7821 * subsequent calls with the same service name will fail with a return value of {@code false}.
7822 * @param instance the {@code Service} subclass instance to expose as a coprocessor endpoint
7823 * @return {@code true} if the registration was successful, {@code false} otherwise
7825 public boolean registerService(Service instance) {
7826 // No stacking of instances is allowed for a single service name
7827 ServiceDescriptor serviceDesc = instance.getDescriptorForType();
7828 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
7829 if (coprocessorServiceHandlers.containsKey(serviceName)) {
7830 LOG.error("Coprocessor service {} already registered, rejecting request from {} in region {}",
7831 serviceName, instance, this);
7832 return false;
7835 coprocessorServiceHandlers.put(serviceName, instance);
7836 if (LOG.isDebugEnabled()) {
7837 LOG.debug("Registered coprocessor service: region=" +
7838 Bytes.toStringBinary(getRegionInfo().getRegionName()) + " service=" + serviceName);
7840 return true;
7844 * Executes a single protocol buffer coprocessor endpoint {@link Service} method using
7845 * the registered protocol handlers. {@link Service} implementations must be registered via the
7846 * {@link #registerService(Service)}
7847 * method before they are available.
7849 * @param controller an {@code RpcContoller} implementation to pass to the invoked service
7850 * @param call a {@code CoprocessorServiceCall} instance identifying the service, method,
7851 * and parameters for the method invocation
7852 * @return a protocol buffer {@code Message} instance containing the method's result
7853 * @throws IOException if no registered service handler is found or an error
7854 * occurs during the invocation
7855 * @see #registerService(Service)
7857 public Message execService(RpcController controller, CoprocessorServiceCall call)
7858 throws IOException {
7859 String serviceName = call.getServiceName();
7860 Service service = coprocessorServiceHandlers.get(serviceName);
7861 if (service == null) {
7862 throw new UnknownProtocolException(null, "No registered coprocessor service found for " +
7863 serviceName + " in region " + Bytes.toStringBinary(getRegionInfo().getRegionName()));
7865 ServiceDescriptor serviceDesc = service.getDescriptorForType();
7867 cpRequestsCount.increment();
7868 String methodName = call.getMethodName();
7869 MethodDescriptor methodDesc =
7870 CoprocessorRpcUtils.getMethodDescriptor(methodName, serviceDesc);
7872 Message.Builder builder =
7873 service.getRequestPrototype(methodDesc).newBuilderForType();
7875 ProtobufUtil.mergeFrom(builder,
7876 call.getRequest().toByteArray());
7877 Message request =
7878 CoprocessorRpcUtils.getRequest(service, methodDesc, call.getRequest());
7880 if (coprocessorHost != null) {
7881 request = coprocessorHost.preEndpointInvocation(service, methodName, request);
7884 final Message.Builder responseBuilder =
7885 service.getResponsePrototype(methodDesc).newBuilderForType();
7886 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
7887 @Override
7888 public void run(Message message) {
7889 if (message != null) {
7890 responseBuilder.mergeFrom(message);
7895 if (coprocessorHost != null) {
7896 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
7898 IOException exception =
7899 org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils.getControllerException(controller);
7900 if (exception != null) {
7901 throw exception;
7904 return responseBuilder.build();
7907 public Optional<byte[]> checkSplit() {
7908 return checkSplit(false);
7912 * Return the split point. An empty result indicates the region isn't splittable.
7914 public Optional<byte[]> checkSplit(boolean force) {
7915 // Can't split META
7916 if (this.getRegionInfo().isMetaRegion()) {
7917 return Optional.empty();
7920 // Can't split a region that is closing.
7921 if (this.isClosing()) {
7922 return Optional.empty();
7925 if (!force && !splitPolicy.shouldSplit()) {
7926 return Optional.empty();
7929 byte[] ret = splitPolicy.getSplitPoint();
7930 if (ret != null && ret.length > 0) {
7931 ret = splitRestriction.getRestrictedSplitPoint(ret);
7934 if (ret != null) {
7935 try {
7936 checkRow(ret, "calculated split");
7937 } catch (IOException e) {
7938 LOG.error("Ignoring invalid split for region {}", this, e);
7939 return Optional.empty();
7941 return Optional.of(ret);
7942 } else {
7943 return Optional.empty();
7948 * @return The priority that this region should have in the compaction queue
7950 public int getCompactPriority() {
7951 return stores.values().stream().mapToInt(HStore::getCompactPriority).min()
7952 .orElse(Store.NO_PRIORITY);
7955 /** @return the coprocessor host */
7956 public RegionCoprocessorHost getCoprocessorHost() {
7957 return coprocessorHost;
7960 /** @param coprocessorHost the new coprocessor host */
7961 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
7962 this.coprocessorHost = coprocessorHost;
7965 @Override
7966 public void startRegionOperation() throws IOException {
7967 startRegionOperation(Operation.ANY);
7970 @Override
7971 public void startRegionOperation(Operation op) throws IOException {
7972 boolean isInterruptableOp = false;
7973 switch (op) {
7974 case GET: // interruptible read operations
7975 case SCAN:
7976 isInterruptableOp = true;
7977 checkReadsEnabled();
7978 break;
7979 case INCREMENT: // interruptible write operations
7980 case APPEND:
7981 case PUT:
7982 case DELETE:
7983 case BATCH_MUTATE:
7984 case CHECK_AND_MUTATE:
7985 isInterruptableOp = true;
7986 break;
7987 default: // all others
7988 break;
7990 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION
7991 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) {
7992 // split, merge or compact region doesn't need to check the closing/closed state or lock the
7993 // region
7994 return;
7996 if (this.closing.get()) {
7997 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
7999 lock(lock.readLock());
8000 // Update regionLockHolders ONLY for any startRegionOperation call that is invoked from
8001 // an RPC handler
8002 Thread thisThread = Thread.currentThread();
8003 if (isInterruptableOp) {
8004 regionLockHolders.put(thisThread, true);
8006 if (this.closed.get()) {
8007 lock.readLock().unlock();
8008 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8010 // The unit for snapshot is a region. So, all stores for this region must be
8011 // prepared for snapshot operation before proceeding.
8012 if (op == Operation.SNAPSHOT) {
8013 stores.values().forEach(HStore::preSnapshotOperation);
8015 try {
8016 if (coprocessorHost != null) {
8017 coprocessorHost.postStartRegionOperation(op);
8019 } catch (Exception e) {
8020 if (isInterruptableOp) {
8021 // would be harmless to remove what we didn't add but we know by 'isInterruptableOp'
8022 // if we added this thread to regionLockHolders
8023 regionLockHolders.remove(thisThread);
8025 lock.readLock().unlock();
8026 throw new IOException(e);
8030 @Override
8031 public void closeRegionOperation() throws IOException {
8032 closeRegionOperation(Operation.ANY);
8035 @Override
8036 public void closeRegionOperation(Operation operation) throws IOException {
8037 if (operation == Operation.SNAPSHOT) {
8038 stores.values().forEach(HStore::postSnapshotOperation);
8040 Thread thisThread = Thread.currentThread();
8041 regionLockHolders.remove(thisThread);
8042 lock.readLock().unlock();
8043 if (coprocessorHost != null) {
8044 coprocessorHost.postCloseRegionOperation(operation);
8049 * This method needs to be called before any public call that reads or
8050 * modifies stores in bulk. It has to be called just before a try.
8051 * #closeBulkRegionOperation needs to be called in the try's finally block
8052 * Acquires a writelock and checks if the region is closing or closed.
8053 * @throws NotServingRegionException when the region is closing or closed
8054 * @throws RegionTooBusyException if failed to get the lock in time
8055 * @throws InterruptedIOException if interrupted while waiting for a lock
8057 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8058 if (this.closing.get()) {
8059 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8061 if (writeLockNeeded) lock(lock.writeLock());
8062 else lock(lock.readLock());
8063 if (this.closed.get()) {
8064 if (writeLockNeeded) lock.writeLock().unlock();
8065 else lock.readLock().unlock();
8066 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8068 regionLockHolders.put(Thread.currentThread(), true);
8072 * Closes the lock. This needs to be called in the finally block corresponding
8073 * to the try block of #startRegionOperation
8075 private void closeBulkRegionOperation(){
8076 regionLockHolders.remove(Thread.currentThread());
8077 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8078 else lock.readLock().unlock();
8082 * Update LongAdders for number of puts without wal and the size of possible data loss.
8083 * These information are exposed by the region server metrics.
8085 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
8086 numMutationsWithoutWAL.increment();
8087 if (numMutationsWithoutWAL.sum() <= 1) {
8088 LOG.info("writing data to region " + this +
8089 " with WAL disabled. Data may be lost in the event of a crash.");
8092 long mutationSize = 0;
8093 for (List<Cell> cells: familyMap.values()) {
8094 // Optimization: 'foreach' loop is not used. See:
8095 // HBASE-12023 HRegion.applyFamilyMapToMemstore creates too many iterator objects
8096 assert cells instanceof RandomAccess;
8097 int listSize = cells.size();
8098 for (int i=0; i < listSize; i++) {
8099 Cell cell = cells.get(i);
8100 mutationSize += cell.getSerializedSize();
8104 dataInMemoryWithoutWAL.add(mutationSize);
8107 private void lock(final Lock lock) throws IOException {
8108 lock(lock, 1);
8112 * Try to acquire a lock. Throw RegionTooBusyException
8113 * if failed to get the lock in time. Throw InterruptedIOException
8114 * if interrupted while waiting for the lock.
8116 private void lock(final Lock lock, final int multiplier) throws IOException {
8117 try {
8118 final long waitTime = Math.min(maxBusyWaitDuration,
8119 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
8120 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
8121 // Don't print millis. Message is used as a key over in
8122 // RetriesExhaustedWithDetailsException processing.
8123 final String regionName =
8124 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
8125 final String serverName = this.getRegionServerServices() == null ?
8126 "unknown" : (this.getRegionServerServices().getServerName() == null ?
8127 "unknown" : this.getRegionServerServices().getServerName().toString());
8128 RegionTooBusyException rtbe = new RegionTooBusyException(
8129 "Failed to obtain lock; regionName=" + regionName + ", server=" + serverName);
8130 LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
8131 throw rtbe;
8133 } catch (InterruptedException ie) {
8134 if (LOG.isDebugEnabled()) {
8135 LOG.debug("Interrupted while waiting for a lock in region {}", this);
8137 throw throwOnInterrupt(ie);
8142 * Calls sync with the given transaction ID
8143 * @param txid should sync up to which transaction
8144 * @throws IOException If anything goes wrong with DFS
8146 private void sync(long txid, Durability durability) throws IOException {
8147 if (this.getRegionInfo().isMetaRegion()) {
8148 this.wal.sync(txid);
8149 } else {
8150 switch(durability) {
8151 case USE_DEFAULT:
8152 // do what table defaults to
8153 if (shouldSyncWAL()) {
8154 this.wal.sync(txid);
8156 break;
8157 case SKIP_WAL:
8158 // nothing do to
8159 break;
8160 case ASYNC_WAL:
8161 // nothing do to
8162 break;
8163 case SYNC_WAL:
8164 this.wal.sync(txid, false);
8165 break;
8166 case FSYNC_WAL:
8167 this.wal.sync(txid, true);
8168 break;
8169 default:
8170 throw new RuntimeException("Unknown durability " + durability);
8176 * Check whether we should sync the wal from the table's durability settings
8178 private boolean shouldSyncWAL() {
8179 return regionDurability.ordinal() > Durability.ASYNC_WAL.ordinal();
8182 /** @return the latest sequence number that was read from storage when this region was opened */
8183 public long getOpenSeqNum() {
8184 return this.openSeqNum;
8187 @Override
8188 public Map<byte[], Long> getMaxStoreSeqId() {
8189 return this.maxSeqIdInStores;
8192 public long getOldestSeqIdOfStore(byte[] familyName) {
8193 return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
8196 @Override
8197 public CompactionState getCompactionState() {
8198 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
8199 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
8200 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
8203 public void reportCompactionRequestStart(boolean isMajor){
8204 (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
8207 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
8208 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
8210 // metrics
8211 compactionsFinished.increment();
8212 compactionNumFilesCompacted.add(numFiles);
8213 compactionNumBytesCompacted.add(filesSizeCompacted);
8215 assert newValue >= 0;
8218 public void reportCompactionRequestFailure() {
8219 compactionsFailed.increment();
8222 public void incrementCompactionsQueuedCount() {
8223 compactionsQueued.increment();
8226 public void decrementCompactionsQueuedCount() {
8227 compactionsQueued.decrement();
8230 public void incrementFlushesQueuedCount() {
8231 flushesQueued.increment();
8234 protected void decrementFlushesQueuedCount() {
8235 flushesQueued.decrement();
8239 * If a handler thread is eligible for interrupt, make it ineligible. Should be paired
8240 * with {{@link #enableInterrupts()}.
8242 void disableInterrupts() {
8243 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> false);
8247 * If a handler thread was made ineligible for interrupt via {{@link #disableInterrupts()},
8248 * make it eligible again. No-op if interrupts are already enabled.
8250 void enableInterrupts() {
8251 regionLockHolders.computeIfPresent(Thread.currentThread(), (t,b) -> true);
8255 * Interrupt any region options that have acquired the region lock via
8256 * {@link #startRegionOperation(org.apache.hadoop.hbase.regionserver.Region.Operation)},
8257 * or {@link #startBulkRegionOperation(boolean)}.
8259 private void interruptRegionOperations() {
8260 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) {
8261 // An entry in this map will have a boolean value indicating if it is currently
8262 // eligible for interrupt; if so, we should interrupt it.
8263 if (entry.getValue().booleanValue()) {
8264 entry.getKey().interrupt();
8270 * Check thread interrupt status and throw an exception if interrupted.
8271 * @throws NotServingRegionException if region is closing
8272 * @throws InterruptedIOException if interrupted but region is not closing
8274 // Package scope for tests
8275 void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
8276 if (Thread.interrupted()) {
8277 if (this.closing.get()) {
8278 throw new NotServingRegionException(
8279 getRegionInfo().getRegionNameAsString() + " is closing");
8281 throw new InterruptedIOException();
8286 * Throw the correct exception upon interrupt
8287 * @param t cause
8289 // Package scope for tests
8290 IOException throwOnInterrupt(Throwable t) {
8291 if (this.closing.get()) {
8292 return (NotServingRegionException) new NotServingRegionException(
8293 getRegionInfo().getRegionNameAsString() + " is closing")
8294 .initCause(t);
8296 return (InterruptedIOException) new InterruptedIOException().initCause(t);
8300 * {@inheritDoc}
8302 @Override
8303 public void onConfigurationChange(Configuration conf) {
8304 this.storeHotnessProtector.update(conf);
8308 * {@inheritDoc}
8310 @Override
8311 public void registerChildren(ConfigurationManager manager) {
8312 configurationManager = manager;
8313 stores.values().forEach(manager::registerObserver);
8317 * {@inheritDoc}
8319 @Override
8320 public void deregisterChildren(ConfigurationManager manager) {
8321 stores.values().forEach(configurationManager::deregisterObserver);
8324 @Override
8325 public CellComparator getCellComparator() {
8326 return cellComparator;
8329 public long getMemStoreFlushSize() {
8330 return this.memstoreFlushSize;
8334 //// method for debugging tests
8335 void throwException(String title, String regionName) {
8336 StringBuilder buf = new StringBuilder();
8337 buf.append(title + ", ");
8338 buf.append(getRegionInfo().toString());
8339 buf.append(getRegionInfo().isMetaRegion() ? " meta region " : " ");
8340 buf.append("stores: ");
8341 for (HStore s : stores.values()) {
8342 buf.append(s.getColumnFamilyDescriptor().getNameAsString());
8343 buf.append(" size: ");
8344 buf.append(s.getMemStoreSize().getDataSize());
8345 buf.append(" ");
8347 buf.append("end-of-stores");
8348 buf.append(", memstore size ");
8349 buf.append(getMemStoreDataSize());
8350 if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) {
8351 throw new RuntimeException(buf.toString());
8355 @Override
8356 public void requestCompaction(String why, int priority, boolean major,
8357 CompactionLifeCycleTracker tracker) throws IOException {
8358 if (major) {
8359 stores.values().forEach(HStore::triggerMajorCompaction);
8361 rsServices.getCompactionRequestor().requestCompaction(this, why, priority, tracker,
8362 RpcServer.getRequestUser().orElse(null));
8365 @Override
8366 public void requestCompaction(byte[] family, String why, int priority, boolean major,
8367 CompactionLifeCycleTracker tracker) throws IOException {
8368 HStore store = stores.get(family);
8369 if (store == null) {
8370 throw new NoSuchColumnFamilyException("column family " + Bytes.toString(family) +
8371 " does not exist in region " + getRegionInfo().getRegionNameAsString());
8373 if (major) {
8374 store.triggerMajorCompaction();
8376 rsServices.getCompactionRequestor().requestCompaction(this, store, why, priority, tracker,
8377 RpcServer.getRequestUser().orElse(null));
8380 private void requestFlushIfNeeded() throws RegionTooBusyException {
8381 if(isFlushSize(this.memStoreSizing.getMemStoreSize())) {
8382 requestFlush();
8386 private void requestFlush() {
8387 if (this.rsServices == null) {
8388 return;
8390 requestFlush0(FlushLifeCycleTracker.DUMMY);
8393 private void requestFlush0(FlushLifeCycleTracker tracker) {
8394 boolean shouldFlush = false;
8395 synchronized (writestate) {
8396 if (!this.writestate.isFlushRequested()) {
8397 shouldFlush = true;
8398 writestate.flushRequested = true;
8401 if (shouldFlush) {
8402 // Make request outside of synchronize block; HBASE-818.
8403 this.rsServices.getFlushRequester().requestFlush(this, tracker);
8404 if (LOG.isDebugEnabled()) {
8405 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
8407 } else {
8408 tracker.notExecuted("Flush already requested on " + this);
8412 @Override
8413 public void requestFlush(FlushLifeCycleTracker tracker) throws IOException {
8414 requestFlush0(tracker);
8418 * This method modifies the region's configuration in order to inject replication-related
8419 * features
8420 * @param conf region configurations
8422 private static void decorateRegionConfiguration(Configuration conf) {
8423 if (ReplicationUtils.isReplicationForBulkLoadDataEnabled(conf)) {
8424 String plugins = conf.get(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,"");
8425 String replicationCoprocessorClass = ReplicationObserver.class.getCanonicalName();
8426 if (!plugins.contains(replicationCoprocessorClass)) {
8427 conf.set(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY,
8428 (plugins.equals("") ? "" : (plugins + ",")) + replicationCoprocessorClass);
8433 public void addReadRequestsCount(long readRequestsCount) {
8434 this.readRequestsCount.add(readRequestsCount);
8437 public void addWriteRequestsCount(long writeRequestsCount) {
8438 this.writeRequestsCount.add(writeRequestsCount);